diff --git a/benchmarks/cdk/bin/datafusion-bench.ts b/benchmarks/cdk/bin/datafusion-bench.ts index 55e7d3fe..aff363b8 100644 --- a/benchmarks/cdk/bin/datafusion-bench.ts +++ b/benchmarks/cdk/bin/datafusion-bench.ts @@ -12,8 +12,7 @@ async function main() { program .requiredOption('--dataset ', 'Dataset to run queries on') .option('-i, --iterations ', 'Number of iterations', '3') - .option('--files-per-task ', 'Files per task', '8') - .option('--cardinality-task-sf ', 'Cardinality task scale factor', '1') + .option('--bytes-processed-per-partition ', 'How many bytes each partition is expected to process', '8388608') // 8 Mb default .option('--batch-size ', 'Standard Batch coalescing size (number of rows)', '32768') .option('--shuffle-batch-size ', 'Shuffle batch coalescing size (number of rows)', '32768') .option('--children-isolator-unions ', 'Use children isolator unions', 'true') @@ -29,8 +28,7 @@ async function main() { const dataset: string = options.dataset const iterations = parseInt(options.iterations); - const filesPerTask = parseInt(options.filesPerTask); - const cardinalityTaskSf = parseInt(options.cardinalityTaskSf); + const bytesProcessedPerPartition = parseInt(options.bytesProcessedPerPartition) const batchSize = parseInt(options.batchSize); const shuffleBatchSize = parseInt(options.shuffleBatchSize); const compression = options.compression; @@ -42,8 +40,7 @@ async function main() { const warmup = options.warmup === 'true' || options.debug === 1 const runner = new DataFusionRunner({ - filesPerTask, - cardinalityTaskSf, + bytesProcessedPerPartition, batchSize, shuffleBatchSize, collectMetrics, @@ -72,8 +69,7 @@ class DataFusionRunner implements BenchmarkRunner { private url = 'http://localhost:9000'; constructor(private readonly options: { - filesPerTask: number; - cardinalityTaskSf: number; + bytesProcessedPerPartition: number; batchSize: number; shuffleBatchSize: number; collectMetrics: boolean; @@ -124,8 +120,7 @@ class DataFusionRunner implements BenchmarkRunner { } await this.query(stmt); await this.query(` - SET distributed.files_per_task=${this.options.filesPerTask}; - SET distributed.cardinality_task_count_factor=${this.options.cardinalityTaskSf}; + SET distributed.bytes_processed_per_partition=${this.options.bytesProcessedPerPartition}; SET datafusion.execution.batch_size=${this.options.batchSize}; SET distributed.shuffle_batch_size=${this.options.shuffleBatchSize}; SET distributed.collect_metrics=${this.options.collectMetrics}; diff --git a/benchmarks/src/run.rs b/benchmarks/src/run.rs index d1672cc1..abf61e17 100644 --- a/benchmarks/src/run.rs +++ b/benchmarks/src/run.rs @@ -75,13 +75,9 @@ pub struct RunOpt { #[structopt(long)] threads: Option, - /// Number of files per each distributed task. - #[structopt(long)] - files_per_task: Option, - - /// Task count scale factor for when nodes in stages change the cardinality of the data - #[structopt(long)] - cardinality_task_sf: Option, + /// Number of bytes each partition is expected to process. + #[structopt(long, default_value = "8388608")] // 8 Mb + bytes_processed_per_partition: usize, /// Use children isolator UNIONs for distributing UNION operations. #[structopt(long)] @@ -176,12 +172,7 @@ impl RunOpt { .with_config(self.config()?) .with_distributed_worker_resolver(LocalHostWorkerResolver::new(self.workers.clone())) .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) - .with_distributed_files_per_task( - self.files_per_task.unwrap_or(get_available_parallelism()), - )? - .with_distributed_cardinality_effect_task_scale_factor( - self.cardinality_task_sf.unwrap_or(1.0), - )? + .with_distributed_bytes_processed_per_partition(self.bytes_processed_per_partition)? .with_distributed_compression(match self.compression.as_str() { "zstd" => Some(CompressionType::ZSTD), "lz4" => Some(CompressionType::LZ4_FRAME), diff --git a/examples/custom_execution_plan.md b/examples/custom_execution_plan.md index 49f2366a..7984e6d1 100644 --- a/examples/custom_execution_plan.md +++ b/examples/custom_execution_plan.md @@ -12,12 +12,8 @@ Uses `DistributedTaskContext` to determine which range to generate. **NumbersExecCodec** – Protobuf-based serialization implementing `PhysicalExtensionCodec`. Must be registered in the `SessionStateBuilder` that initiates the query as well as the one used by `Worker`s. -**NumbersTaskEstimator** – Controls distributed parallelism: - -- `task_estimation()` - Returns how many tasks needed based on range size and config -- `scale_up_leaf_node()` - Splits single range of numbers into N per-task ranges - -**NumbersConfig** – Custom config extension for controlling distributed parallelism (`numbers_per_task: usize`) +**NumbersDistributedPlannerExtension** – Controls distributed parallelism: + - `scale_up_leaf_node()` - Splits single range of numbers into N per-task ranges ## Usage @@ -49,16 +45,20 @@ SortPreservingMergeExec: [number@0 ASC NULLS LAST] This will print a non-distributed plan, as the range of numbers we are querying (`numbers(0, 10)`) is small. -The config parameter `numbers.numbers_per_task` is the one that controls how many distributed tasks are used in the -query, and it's default value is `10`, so querying 10 numbers will not distribute the plan. +Distributed DataFusion has a config parameter that allows controlling the parallelism of a distributed query: +`distributed.bytes_processed_per_partition`. + +It determines how many bytes each partition is expected to handle, and if handling the query would require more +partitions than CPUs the machine has, then the query will get distributed across workers. -However, if we try to query 11 numbers: +For example, if we set `distributed.bytes_processed_per_partition` to something very low, like 10 bytes, +the query will get distributed: ```bash cargo run \ --features integration \ --example custom_execution_plan \ - "SELECT DISTINCT number FROM numbers(0, 11) ORDER BY number" \ + "SET distributed.bytes_processed_per_partition=10;SELECT DISTINCT number FROM numbers(0, 11) ORDER BY number" \ --show-distributed-plan ``` @@ -70,58 +70,55 @@ cargo run \ ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p0..p15] │ SortExec: expr=[number@0 ASC NULLS LAST], preserve_partitioning=[true] │ AggregateExec: mode=FinalPartitioned, gby=[number@0 as number], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=16, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=16, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p31] t1:[p0..p31] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([number@0], 32), input_partitions=16 - │ AggregateExec: mode=Partial, gby=[number@0 as number], aggr=[] - │ RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 - │ CooperativeExec - │ NumbersExec: t0:[0-6), t1:[6-11) + ┌───── Stage 1 ── Tasks: t0:[p0..p31] t1:[p0..p31] t2:[p0..p31] + │ RepartitionExec: partitioning=Hash([number@0], 32), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[number@0 as number], aggr=[] + │ CooperativeExec + │ NumbersExec: t0:[0-4), t1:[4-8), t2:[8-11) └────────────────────────────────────────────────── ``` The distribution rule kicks in, and the plan gets distributed. -Note that the parallelism in the plan has an upper threshold, so for example, if we query 100 numbers: +Note that the parallelism in the plan has an upper threshold, so for example, if we query 100 numbers so that +more rows flow through the query: ```bash cargo run \ --features integration \ --example custom_execution_plan \ - "SELECT DISTINCT number FROM numbers(0, 100) ORDER BY number" \ + "SET distributed.bytes_processed_per_partition=10;SELECT DISTINCT number FROM numbers(0, 100) ORDER BY number" \ --show-distributed-plan ``` ``` ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [number@0 ASC NULLS LAST] -│ [Stage 2] => NetworkCoalesceExec: output_partitions=48, input_tasks=3 +│ [Stage 2] => NetworkCoalesceExec: output_partitions=64, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p0..p15] t2:[p0..p15] + ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p0..p15] t2:[p0..p15] t3:[p0..p15] │ SortExec: expr=[number@0 ASC NULLS LAST], preserve_partitioning=[true] │ AggregateExec: mode=FinalPartitioned, gby=[number@0 as number], aggr=[] │ [Stage 1] => NetworkShuffleExec: output_partitions=16, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p47] t1:[p0..p47] t2:[p0..p47] t3:[p0..p47] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([number@0], 48), input_partitions=16 - │ AggregateExec: mode=Partial, gby=[number@0 as number], aggr=[] - │ RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 - │ CooperativeExec - │ NumbersExec: t0:[0-25), t1:[25-50), t2:[50-75), t3:[75-100) + ┌───── Stage 1 ── Tasks: t0:[p0..p63] t1:[p0..p63] t2:[p0..p63] t3:[p0..p63] + │ RepartitionExec: partitioning=Hash([number@0], 64), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[number@0 as number], aggr=[] + │ CooperativeExec + │ NumbersExec: t0:[0-25), t1:[25-50), t2:[50-75), t3:[75-100) └────────────────────────────────────────────────── ``` -We do not get 100/10 = 10 distributed tasks, we just get 4. This is because the example is configured by default to -simulate a 4-worker cluster. If we increase the worker count, we get a highly distributed plan out with 10 tasks: +We do not get many more distributed tasks, we just get 4. This is because the example is configured by default to +simulate a 4-worker cluster. If we increase the worker count, we get a highly distributed plan with more parallelism: ```bash cargo run \ --features integration \ --example custom_execution_plan \ - "SELECT DISTINCT number FROM numbers(0, 100) ORDER BY number" \ + "SET distributed.bytes_processed_per_partition=10;SELECT DISTINCT number FROM numbers(0, 100) ORDER BY number" \ --workers 10 \ --show-distributed-plan ``` @@ -129,20 +126,18 @@ cargo run \ ``` ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [number@0 ASC NULLS LAST] -│ [Stage 2] => NetworkCoalesceExec: output_partitions=112, input_tasks=7 +│ [Stage 2] => NetworkCoalesceExec: output_partitions=160, input_tasks=10 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p0..p15] t2:[p0..p15] t3:[p0..p15] t4:[p0..p15] t5:[p0..p15] t6:[p0..p15] + ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p0..p15] t2:[p0..p15] t3:[p0..p15] t4:[p0..p15] t5:[p0..p15] t6:[p0..p15] t7:[p0..p15] t8:[p0..p15] t9:[p0..p15] │ SortExec: expr=[number@0 ASC NULLS LAST], preserve_partitioning=[true] │ AggregateExec: mode=FinalPartitioned, gby=[number@0 as number], aggr=[] │ [Stage 1] => NetworkShuffleExec: output_partitions=16, input_tasks=10 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p111] t1:[p0..p111] t2:[p0..p111] t3:[p0..p111] t4:[p0..p111] t5:[p0..p111] t6:[p0..p111] t7:[p0..p111] t8:[p0..p111] t9:[p0..p111] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([number@0], 112), input_partitions=16 - │ AggregateExec: mode=Partial, gby=[number@0 as number], aggr=[] - │ RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 - │ CooperativeExec - │ NumbersExec: t0:[0-10), t1:[10-20), t2:[20-30), t3:[30-40), t4:[40-50), t5:[50-60), t6:[60-70), t7:[70-80), t8:[80-90), t9:[90-100) + ┌───── Stage 1 ── Tasks: t0:[p0..p159] t1:[p0..p159] t2:[p0..p159] t3:[p0..p159] t4:[p0..p159] t5:[p0..p159] t6:[p0..p159] t7:[p0..p159] t8:[p0..p159] t9:[p0..p159] + │ RepartitionExec: partitioning=Hash([number@0], 160), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[number@0 as number], aggr=[] + │ CooperativeExec + │ NumbersExec: t0:[0-10), t1:[10-20), t2:[20-30), t3:[30-40), t4:[40-50), t5:[50-60), t6:[60-70), t7:[70-80), t8:[80-90), t9:[90-100) └────────────────────────────────────────────────── ``` diff --git a/examples/custom_execution_plan.rs b/examples/custom_execution_plan.rs index 10c16d28..cbc7d669 100644 --- a/examples/custom_execution_plan.rs +++ b/examples/custom_execution_plan.rs @@ -7,7 +7,7 @@ //! - Custom TableProvider for mapping the table function to an execution plan //! - Custom ExecutionPlan for returning the requested number range //! - Custom PhysicalExtensionCodec for serialization across the network -//! - Custom TaskEstimator to control parallelism +//! - Custom DistributedPlannerExtension to control how to scale up a custom node //! //! Run this example with: //! ```bash @@ -23,24 +23,26 @@ use arrow::record_batch::RecordBatchOptions; use arrow::util::pretty::pretty_format_batches; use async_trait::async_trait; use datafusion::catalog::{Session, TableFunctionImpl}; +use datafusion::common::stats::Precision; use datafusion::common::{ - DataFusionError, Result, ScalarValue, exec_err, extensions_options, internal_err, plan_err, + DataFusionError, Result, ScalarValue, Statistics, exec_err, internal_err, plan_err, }; -use datafusion::config::ConfigExtension; use datafusion::datasource::{TableProvider, TableType}; use datafusion::execution::{SendableRecordBatchStream, SessionStateBuilder, TaskContext}; use datafusion::logical_expr::Expr; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; -use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; -use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion::physical_plan::{ + ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; +use datafusion::prelude::SessionContext; use datafusion_distributed::test_utils::in_memory_channel_resolver::{ InMemoryChannelResolver, InMemoryWorkerResolver, }; use datafusion_distributed::{ - DistributedExt, DistributedPhysicalOptimizerRule, DistributedTaskContext, TaskEstimation, - TaskEstimator, WorkerQueryContext, display_plan_ascii, + DistributedExt, DistributedPhysicalOptimizerRule, DistributedPlannerExtension, + DistributedTaskContext, WorkerQueryContext, display_plan_ascii, }; use datafusion_proto::physical_plan::PhysicalExtensionCodec; use datafusion_proto::protobuf; @@ -219,6 +221,23 @@ impl ExecutionPlan for NumbersExec { stream::once(async { Ok(batch) }), ))) } + + /// Implementing [ExecutionPlan::partition_statistics] is essential for Distributed DataFusion + /// to infer how much compute is going to be needed across the plan. + fn partition_statistics(&self, _: Option) -> Result { + let mut stats = Statistics::default(); + let num_rows = self + .ranges_per_task + .iter() + .map(|v| v.end - v.start) + .sum::(); + + stats.num_rows = Precision::Exact(num_rows as usize); + stats.column_statistics = + vec![ColumnStatistics::new_unknown().with_distinct_count(stats.num_rows)]; + + Ok(stats) + } } /// Custom codec for serializing/deserializing NumbersExec across the network. As the NumbersExec @@ -261,7 +280,7 @@ impl PhysicalExtensionCodec for NumbersExecCodec { .schema .as_ref() .map(|s| s.try_into()) - .ok_or(proto_error("NetworkShuffleExec is missing schema"))??; + .ok_or(proto_error("NumbersExec is missing schema"))??; Ok(Arc::new(NumbersExec::new( proto.ranges.iter().map(|v| v.start..v.end), @@ -292,37 +311,11 @@ impl PhysicalExtensionCodec for NumbersExecCodec { } } -extensions_options! { - /// Custom ConfigExtension for configuring NumbersExec distributed task estimation behavior - /// at runtime with SET statements. - struct NumbersConfig { - /// how many numbers each task will produce - numbers_per_task: usize, default = 10 - } -} - -impl ConfigExtension for NumbersConfig { - const PREFIX: &'static str = "numbers"; -} - -/// Custom TaskEstimator that tells the planner how to distribute NumbersExec. +/// Custom [DistributedPlannerExtension] that tells the planner how to distribute [NumbersExec]. #[derive(Debug)] -struct NumbersTaskEstimator; - -impl TaskEstimator for NumbersTaskEstimator { - fn task_estimation( - &self, - plan: &Arc, - cfg: &datafusion::config::ConfigOptions, - ) -> Option { - let plan = plan.as_any().downcast_ref::()?; - let cfg: &NumbersConfig = cfg.extensions.get()?; - let task_count = (plan.ranges_per_task[0].end - plan.ranges_per_task[0].start) as f64 - / cfg.numbers_per_task as f64; - - Some(TaskEstimation::desired(task_count.ceil() as usize)) - } +struct NumbersDistributedPlannerExtension; +impl DistributedPlannerExtension for NumbersDistributedPlannerExtension { fn scale_up_leaf_node( &self, plan: &Arc, @@ -375,16 +368,13 @@ async fn main() -> Result<(), Box> { .build()) }); - let config = SessionConfig::new().with_option_extension(NumbersConfig::default()); - let state = SessionStateBuilder::new() .with_default_features() - .with_config(config) .with_distributed_worker_resolver(worker_resolver) .with_distributed_channel_resolver(channel_resolver) .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) .with_distributed_user_codec(NumbersExecCodec) - .with_distributed_task_estimator(NumbersTaskEstimator) + .with_distributed_planner_extension(NumbersDistributedPlannerExtension) .build(); let ctx = SessionContext::from(state); diff --git a/examples/in_memory_cluster.rs b/examples/in_memory_cluster.rs index e5f02fdc..ddbe733c 100644 --- a/examples/in_memory_cluster.rs +++ b/examples/in_memory_cluster.rs @@ -38,8 +38,9 @@ async fn main() -> Result<(), Box> { .with_default_features() .with_distributed_worker_resolver(InMemoryWorkerResolver) .with_distributed_channel_resolver(InMemoryChannelResolver::new()) + // Set to something very low so that we see some distribution. + .with_distributed_bytes_processed_per_partition(100)? .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) - .with_distributed_files_per_task(1)? .build(); let ctx = SessionContext::from(state); diff --git a/examples/localhost_run.rs b/examples/localhost_run.rs index eae9b6cd..be3ad08a 100644 --- a/examples/localhost_run.rs +++ b/examples/localhost_run.rs @@ -39,8 +39,9 @@ async fn main() -> Result<(), Box> { let state = SessionStateBuilder::new() .with_default_features() .with_distributed_worker_resolver(localhost_resolver) + // Set to something very low so that we see some distribution. + .with_distributed_bytes_processed_per_partition(100)? .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) - .with_distributed_files_per_task(1)? .build(); let ctx = SessionContext::from(state); diff --git a/src/distributed_ext.rs b/src/distributed_ext.rs index 7efea431..cba8d546 100644 --- a/src/distributed_ext.rs +++ b/src/distributed_ext.rs @@ -1,11 +1,11 @@ use crate::config_extension_ext::{ set_distributed_option_extension, set_distributed_option_extension_from_headers, }; -use crate::distributed_planner::set_distributed_task_estimator; +use crate::distributed_planner::set_distributed_planner_extension; use crate::networking::{set_distributed_channel_resolver, set_distributed_worker_resolver}; use crate::passthrough_headers::set_passthrough_headers; use crate::protobuf::{set_distributed_user_codec, set_distributed_user_codec_arc}; -use crate::{ChannelResolver, DistributedConfig, TaskEstimator, WorkerResolver}; +use crate::{ChannelResolver, DistributedConfig, DistributedPlannerExtension, WorkerResolver}; use arrow_ipc::CompressionType; use datafusion::common::DataFusionError; use datafusion::config::ConfigExtension; @@ -285,12 +285,11 @@ pub trait DistributedExt: Sized { resolver: T, ); - /// Adds a distributed task count estimator. [TaskEstimator]s are executed on each node - /// sequentially until one returns an estimation on the number of tasks that should be - /// used for the stage containing that node. + /// Adds a [DistributedPlannerExtension] to the session that handles custom `ExecutionPlan`s + /// during distributed planning. /// - /// Many nodes might decide to provide an estimation, so a reconciliation between all of them - /// is performed internally during planning. + /// Multiple [DistributedPlannerExtension] can be provided, and they will be called sequentially + /// for each node in the plan until one returns a way of handling the node. /// /// ```text /// ┌───────────────────────┐ @@ -309,107 +308,22 @@ pub trait DistributedExt: Sized { /// ┌───────────────────────┐ │ /// │ │ FilterExec │ /// └───────────────────────┘ │ - /// │ ┌───────────────────────┐ a TaskEstimator estimates the amount of tasks - /// │ SomeExec │◀───┼── based on how much data will be pulled. - /// │ └───────────────────────┘ + /// │ ┌───────────────────────┐ a DistributedPlannerExtension determines how this node + /// │ SomeExec │◀───┼── should be "scaled up" in case of getting executed in + /// │ └───────────────────────┘ a distributed manner. /// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ /// ``` - fn with_distributed_task_estimator( + fn with_distributed_planner_extension( self, estimator: T, ) -> Self; - /// Same as [DistributedExt::with_distributed_task_estimator] but with an in-place mutation. - fn set_distributed_task_estimator( + /// Same as [DistributedExt::with_distributed_planner_extension] but with an in-place mutation. + fn set_distributed_planner_extension( &mut self, estimator: T, ); - /// Sets the maximum number of files each task in a stage with a FileScanConfig node will - /// handle. Reducing this number will increment the amount of tasks. By default, this - /// is close to the number of cores in the machine. - /// - /// ```text - /// ┌───────────────────────┐ - /// │SortPreservingMergeExec│ - /// └───────────────────────┘ - /// ▲ - /// ┌ ─ ─ ─ ─ ─ ─ ─ ┼ ─ ─ ─ ─ ─ ─ ─ ─ Stage 2 - /// ┌───────────┴───────────┐ │ - /// │ │ SortExec │ - /// └───────────────────────┘ │ - /// │ ┌───────────────────────┐ - /// │ AggregateExec │ │ - /// │ └───────────────────────┘ - /// ─ ─ ─ ─ ─ ─ ─ ─▲─ ─ ─ ─ ─ ─ ─ ─ ┘ - /// ┌ ─ ─ ─ ─ ─ ─ ─ ┴ ─ ─ ─ ─ ─ ─ ─ ─ Stage 1 - /// ┌───────────────────────┐ │ - /// │ │ FilterExec │ - /// └───────────────────────┘ │ - /// │ ┌───────────────────────┐ Sets the max number of files - /// │ FileScanConfig │◀───┼─ each task will handle. Less - /// │ └───────────────────────┘ files_per_task == more tasks - /// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ - ///``` - fn with_distributed_files_per_task( - self, - files_per_task: usize, - ) -> Result; - - /// Same as [DistributedExt::with_distributed_files_per_task] but with an in-place mutation. - fn set_distributed_files_per_task( - &mut self, - files_per_task: usize, - ) -> Result<(), DataFusionError>; - - /// The number of tasks in each stage is calculated in a bottom-to-top fashion. - /// - /// Bottom stages containing leaf nodes will provide an estimation of the amount of tasks - /// for those stages, but upper stages might see a reduction (or increment) in the amount - /// of tasks based on the cardinality effect bottom stages have in the data. - /// - /// For example: If there are two stages, and the leaf stage is estimated to use 10 tasks, - /// the upper stage might use less (e.g. 5) if it sees that the leaf stage is returning - /// less data because of filters or aggregations. - /// - /// This function sets the scale factor for when encountering these nodes that change the - /// cardinality of the data. For example, if a stage with 10 tasks contains an AggregateExec - /// node, and the scale factor is 2.0, the following stage will use 10 / 2.0 = 5 tasks. - /// - /// ```text - /// ┌───────────────────────┐ - /// │SortPreservingMergeExec│ - /// └───────────────────────┘ - /// ▲ - /// ┌ ─ ─ ─ ─ ─ ─ ─ ┼ ─ ─ ─ ─ ─ ─ ─ ─ Stage 2 (N/scale_factor tasks) - /// ┌───────────┴───────────┐ │ - /// │ │ SortExec │ - /// └───────────────────────┘ │ - /// │ ┌───────────────────────┐ - /// │ AggregateExec │ │ - /// │ └───────────────────────┘ - /// ─ ─ ─ ─ ─ ─ ─ ─▲─ ─ ─ ─ ─ ─ ─ ─ ┘ - /// ┌ ─ ─ ─ ─ ─ ─ ─ ┴ ─ ─ ─ ─ ─ ─ ─ ─ Stage 1 (N tasks) - /// ┌───────────────────────┐ │ A filter reduces cardinality, - /// │ │ FilterExec │◀────────therefore the next stage will have - /// └───────────────────────┘ │ less tasks according to this factor - /// │ ┌───────────────────────┐ - /// │ FileScanConfig │ │ - /// │ └───────────────────────┘ - /// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ - /// ``` - fn with_distributed_cardinality_effect_task_scale_factor( - self, - factor: f64, - ) -> Result; - - /// Same as [DistributedExt::with_distributed_cardinality_effect_task_scale_factor] but with - /// an in-place mutation. - fn set_distributed_cardinality_effect_task_scale_factor( - &mut self, - factor: f64, - ) -> Result<(), DataFusionError>; - /// Enables metrics collection across network boundaries so that all the metrics gather in /// each node are accessible from the head stage that started running the query. fn with_distributed_metrics_collection(self, enabled: bool) -> Result; @@ -520,6 +434,20 @@ pub trait DistributedExt: Sized { &mut self, headers: HeaderMap, ) -> Result<(), DataFusionError>; + + /// The amount of bytes each partition is expected to process. + /// + /// TODO: longer explanation. + fn with_distributed_bytes_processed_per_partition( + self, + bytes_processed_per_partition: usize, + ) -> Result; + + /// Same as [DistributedExt::with_distributed_bytes_processed_per_partition] but with an in-place mutation. + fn set_distributed_bytes_processed_per_partition( + &mut self, + bytes_processed_per_partition: usize, + ) -> Result<(), DataFusionError>; } impl DistributedExt for SessionConfig { @@ -557,29 +485,11 @@ impl DistributedExt for SessionConfig { set_distributed_channel_resolver(self, resolver); } - fn set_distributed_task_estimator( + fn set_distributed_planner_extension( &mut self, estimator: T, ) { - set_distributed_task_estimator(self, estimator) - } - - fn set_distributed_files_per_task( - &mut self, - files_per_task: usize, - ) -> Result<(), DataFusionError> { - let d_cfg = DistributedConfig::from_config_options_mut(self.options_mut())?; - d_cfg.files_per_task = files_per_task; - Ok(()) - } - - fn set_distributed_cardinality_effect_task_scale_factor( - &mut self, - factor: f64, - ) -> Result<(), DataFusionError> { - let d_cfg = DistributedConfig::from_config_options_mut(self.options_mut())?; - d_cfg.cardinality_task_count_factor = factor; - Ok(()) + set_distributed_planner_extension(self, estimator) } fn set_distributed_metrics_collection(&mut self, enabled: bool) -> Result<(), DataFusionError> { @@ -632,6 +542,15 @@ impl DistributedExt for SessionConfig { set_passthrough_headers(self, headers) } + fn set_distributed_bytes_processed_per_partition( + &mut self, + bytes_processed_per_partition: usize, + ) -> Result<(), DataFusionError> { + let d_cfg = DistributedConfig::from_config_options_mut(self.options_mut())?; + d_cfg.bytes_processed_per_partition = bytes_processed_per_partition; + Ok(()) + } + delegate! { to self { #[call(set_distributed_option_extension)] @@ -658,17 +577,9 @@ impl DistributedExt for SessionConfig { #[expr($;self)] fn with_distributed_channel_resolver(mut self, resolver: T) -> Self; - #[call(set_distributed_task_estimator)] + #[call(set_distributed_planner_extension)] #[expr($;self)] - fn with_distributed_task_estimator(mut self, estimator: T) -> Self; - - #[call(set_distributed_files_per_task)] - #[expr($?;Ok(self))] - fn with_distributed_files_per_task(mut self, files_per_task: usize) -> Result; - - #[call(set_distributed_cardinality_effect_task_scale_factor)] - #[expr($?;Ok(self))] - fn with_distributed_cardinality_effect_task_scale_factor(mut self, factor: f64) -> Result; + fn with_distributed_planner_extension(mut self, estimator: T) -> Self; #[call(set_distributed_metrics_collection)] #[expr($?;Ok(self))] @@ -693,6 +604,10 @@ impl DistributedExt for SessionConfig { #[call(set_distributed_passthrough_headers)] #[expr($?;Ok(self))] fn with_distributed_passthrough_headers(mut self, headers: HeaderMap) -> Result; + + #[call(set_distributed_bytes_processed_per_partition)] + #[expr($?;Ok(self))] + fn with_distributed_bytes_processed_per_partition(mut self, processed_per_partition: usize) -> Result; } } } @@ -730,20 +645,10 @@ impl DistributedExt for SessionStateBuilder { #[expr($;self)] fn with_distributed_channel_resolver(mut self, resolver: T) -> Self; - fn set_distributed_task_estimator(&mut self, estimator: T); - #[call(set_distributed_task_estimator)] + fn set_distributed_planner_extension(&mut self, estimator: T); + #[call(set_distributed_planner_extension)] #[expr($;self)] - fn with_distributed_task_estimator(mut self, estimator: T) -> Self; - - fn set_distributed_files_per_task(&mut self, files_per_task: usize) -> Result<(), DataFusionError>; - #[call(set_distributed_files_per_task)] - #[expr($?;Ok(self))] - fn with_distributed_files_per_task(mut self, files_per_task: usize) -> Result; - - fn set_distributed_cardinality_effect_task_scale_factor(&mut self, factor: f64) -> Result<(), DataFusionError>; - #[call(set_distributed_cardinality_effect_task_scale_factor)] - #[expr($?;Ok(self))] - fn with_distributed_cardinality_effect_task_scale_factor(mut self, factor: f64) -> Result; + fn with_distributed_planner_extension(mut self, estimator: T) -> Self; fn set_distributed_metrics_collection(&mut self, enabled: bool) -> Result<(), DataFusionError>; #[call(set_distributed_metrics_collection)] @@ -774,6 +679,11 @@ impl DistributedExt for SessionStateBuilder { #[call(set_distributed_passthrough_headers)] #[expr($?;Ok(self))] fn with_distributed_passthrough_headers(mut self, headers: HeaderMap) -> Result; + + fn set_distributed_bytes_processed_per_partition(&mut self, bytes_processed_per_partition: usize) -> Result<(), DataFusionError>; + #[call(set_distributed_bytes_processed_per_partition)] + #[expr($?;Ok(self))] + fn with_distributed_bytes_processed_per_partition(mut self, bytes_processed_per_partition: usize) -> Result; } } } @@ -811,20 +721,10 @@ impl DistributedExt for SessionState { #[expr($;self)] fn with_distributed_channel_resolver(mut self, resolver: T) -> Self; - fn set_distributed_task_estimator(&mut self, estimator: T); - #[call(set_distributed_task_estimator)] + fn set_distributed_planner_extension(&mut self, estimator: T); + #[call(set_distributed_planner_extension)] #[expr($;self)] - fn with_distributed_task_estimator(mut self, estimator: T) -> Self; - - fn set_distributed_files_per_task(&mut self, files_per_task: usize) -> Result<(), DataFusionError>; - #[call(set_distributed_files_per_task)] - #[expr($?;Ok(self))] - fn with_distributed_files_per_task(mut self, files_per_task: usize) -> Result; - - fn set_distributed_cardinality_effect_task_scale_factor(&mut self, factor: f64) -> Result<(), DataFusionError>; - #[call(set_distributed_cardinality_effect_task_scale_factor)] - #[expr($?;Ok(self))] - fn with_distributed_cardinality_effect_task_scale_factor(mut self, factor: f64) -> Result; + fn with_distributed_planner_extension(mut self, estimator: T) -> Self; fn set_distributed_metrics_collection(&mut self, enabled: bool) -> Result<(), DataFusionError>; #[call(set_distributed_metrics_collection)] @@ -855,6 +755,11 @@ impl DistributedExt for SessionState { #[call(set_distributed_passthrough_headers)] #[expr($?;Ok(self))] fn with_distributed_passthrough_headers(mut self, headers: HeaderMap) -> Result; + + fn set_distributed_bytes_processed_per_partition(&mut self, bytes_processed_per_partition: usize) -> Result<(), DataFusionError>; + #[call(set_distributed_bytes_processed_per_partition)] + #[expr($?;Ok(self))] + fn with_distributed_bytes_processed_per_partition(mut self, bytes_processed_per_partition: usize) -> Result; } } } @@ -892,20 +797,10 @@ impl DistributedExt for SessionContext { #[expr($;self)] fn with_distributed_channel_resolver(self, resolver: T) -> Self; - fn set_distributed_task_estimator(&mut self, estimator: T); - #[call(set_distributed_task_estimator)] + fn set_distributed_planner_extension(&mut self, estimator: T); + #[call(set_distributed_planner_extension)] #[expr($;self)] - fn with_distributed_task_estimator(self, estimator: T) -> Self; - - fn set_distributed_files_per_task(&mut self, files_per_task: usize) -> Result<(), DataFusionError>; - #[call(set_distributed_files_per_task)] - #[expr($?;Ok(self))] - fn with_distributed_files_per_task(self, files_per_task: usize) -> Result; - - fn set_distributed_cardinality_effect_task_scale_factor(&mut self, factor: f64) -> Result<(), DataFusionError>; - #[call(set_distributed_cardinality_effect_task_scale_factor)] - #[expr($?;Ok(self))] - fn with_distributed_cardinality_effect_task_scale_factor(self, factor: f64) -> Result; + fn with_distributed_planner_extension(self, estimator: T) -> Self; fn set_distributed_metrics_collection(&mut self, enabled: bool) -> Result<(), DataFusionError>; #[call(set_distributed_metrics_collection)] @@ -936,6 +831,11 @@ impl DistributedExt for SessionContext { #[call(set_distributed_passthrough_headers)] #[expr($?;Ok(self))] fn with_distributed_passthrough_headers(self, headers: HeaderMap) -> Result; + + fn set_distributed_bytes_processed_per_partition(&mut self, bytes_processed_per_partition: usize) -> Result<(), DataFusionError>; + #[call(set_distributed_bytes_processed_per_partition)] + #[expr($?;Ok(self))] + fn with_distributed_bytes_processed_per_partition(self, bytes_processed_per_partition: usize) -> Result; } } } diff --git a/src/distributed_planner/batch_coalescing_below_network_boundaries.rs b/src/distributed_planner/batch_coalescing_below_network_boundaries.rs index 80da921a..20c3157c 100644 --- a/src/distributed_planner/batch_coalescing_below_network_boundaries.rs +++ b/src/distributed_planner/batch_coalescing_below_network_boundaries.rs @@ -80,14 +80,14 @@ mod tests { assert_snapshot!(explain, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[p0..p3] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 12), input_partitions=4 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] @@ -108,14 +108,14 @@ mod tests { assert_snapshot!(explain, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[p0..p3] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 12), input_partitions=4 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] @@ -136,16 +136,16 @@ mod tests { assert_snapshot!(explain, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[p0..p3] │ CoalesceBatchesExec: target_batch_size=101 │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] │ CoalesceBatchesExec: target_batch_size=101 - │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4 + │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 12), input_partitions=4 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] @@ -160,6 +160,8 @@ mod tests { .with_config(SessionConfig::new().with_target_partitions(4)) .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) .with_distributed_worker_resolver(InMemoryWorkerResolver::new(3)) + .with_distributed_bytes_processed_per_partition(1000) + .unwrap() .build(); let ctx = SessionContext::new_with_state(state); diff --git a/src/distributed_planner/children_isolator_union_split.rs b/src/distributed_planner/children_isolator_union_split.rs new file mode 100644 index 00000000..15b22b23 --- /dev/null +++ b/src/distributed_planner/children_isolator_union_split.rs @@ -0,0 +1,384 @@ +use crate::DistributedTaskContext; +use crate::distributed_planner::plan_annotator::AnnotatedPlan; +use datafusion::common::{Result, internal_err}; + +/// Given a list of children with a different compute cost each, and a restriction about the maximum +/// tasks in which they are allowed to run, it assigns tasks counts to them so that the following +/// conditions are met: +/// 1. The amount of tasks used for each child is proportional to their cost. +/// 2. The restriction about the maximum tasks in which a child can run is respected. +/// (e.g., a child with high cost and a "maximum 1 task" restriction runs in 1 task) +/// 3. The children are distributed across tasks so that the total compute cost running within +/// each task is as evenly distributed as possible. +/// +/// For example, given these inputs: +/// `children`: [{cost: 1, max: None}, {cost: 1, max: None}, {cost: 1, max: None}] +/// `task_count`: 3 +/// It returns the following output: +/// `[[(0, 0/1)], [(1, 0/1)], [(2, 0/1)]]` +/// That means that: +/// - Task 0 will execute task 0 from child 0 +/// - Task 1 will execute task 0 from child 1 +/// - Task 2 will execute task 0 from child 2 +/// +/// If there is one child with a very high cost, preference is given to it to run distributed, even +/// if that implies squeezing two cheap nodes in 1 task in order to give room to the expensive one. +/// `children`: [{cost: 4, max: None}, {cost: 1, max: None}, {cost: 1, max: None}] +/// `task_count`: 3 +/// It returns the following output: +/// `[[(0, 0/2)], [(0, 1/2)], [(1, 0/1), (2, 0/1)]]` +/// That means that: +/// - Task 0 will execute task 0 from child 0 (the expensive one) +/// - Task 1 will execute task 1 from child 0 (the expensive one) +/// - Task 2 will execute task 0 from child 1 and task 0 from child 2 +/// +/// If one child declares that it can only run in a specific number of tasks at most, this needs +/// to be respected: +/// `children`: [{cost: 4, max: Some(1)}, {cost: 1, max: None}, {cost: 1, max: None}] +/// `task_count`: 3 +/// It returns the following output: +/// `[[(0, 0/1)], [(1, 0/1)], [(2, 0/1)]]` +/// - Task 0 will execute task 0 from child 0. This is expensive, but nothing we can do about it. +/// - Task 1 will execute task 0 from child 1 +/// - Task 2 will execute task 0 from child 2 +/// +/// The function looks pretty much like the solution of a LeetCode problem, so it's not super +/// easy to understand just be looking at the code. The best way to get a grasp of what its doing +/// is by looking at the tests. +pub(super) fn children_isolator_union_split( + children: &[impl CostChild], + task_count_budget: usize, +) -> Result< + // Task idx. This Vec will have `task_count_budget` length. + Vec< + // For this task, the child indexes and DistributedTaskContext that should be executed. + Vec<( + /* Child index */ usize, + /* Distributed task ctx for the child */ DistributedTaskContext, + )>, + >, +> { + if task_count_budget == 0 { + return internal_err!( + "ChildrenIsolatorUnionExec had a task count {task_count_budget}. This is a bug in the distributed planning logic, please report it" + ); + } + + if children.is_empty() { + return Ok(vec![vec![]; task_count_budget]); + } + + // ==================== Phase 1: Calculate subtask count for each child ==================== + // Goal: distribute task_count_budget subtasks among children proportionally to their cost, + // respecting max_tasks constraints and ensuring at least 1 subtask per child. + + let costs: Vec = children.iter().map(|c| c.cost()).collect(); + let max_tasks: Vec> = children.iter().map(|c| c.max_tasks()).collect(); + let total_cost: usize = costs.iter().sum(); + + // Calculate ideal subtask count for each child based on cost proportion + let avg_budget_per_task = if total_cost > 0 { + total_cost as f64 / task_count_budget as f64 + } else { + 1.0 + }; + + let mut child_subtask_counts: Vec = costs + .iter() + .enumerate() + .map(|(i, &cost)| { + // Ideal subtasks proportional to cost + let ideal = if total_cost > 0 { + (cost as f64 / avg_budget_per_task).ceil() as usize + } else { + 1 + }; + // At least 1 subtask per child + let count = ideal.max(1); + // Apply max_tasks constraint + max_tasks[i].map_or(count, |max| count.min(max.max(1))) + }) + .collect(); + + // Adjust total subtasks to match budget (or get as close as possible) + let mut total_subtasks: usize = child_subtask_counts.iter().sum(); + + // Trim if over budget: reduce from children with the highest subtask count + // Tiebreaker: prefer reducing lower-cost children (keep expensive ones parallel) + while total_subtasks > task_count_budget { + let mut best_idx = None; + let mut best_count = 1; // Can't reduce below 1 + + for (i, &count) in child_subtask_counts.iter().enumerate() { + // Find child with the highest count (above 1), tiebreak by lower cost + if count > best_count { + best_idx = Some(i); + best_count = count; + } + } + + if let Some(idx) = best_idx { + child_subtask_counts[idx] -= 1; + total_subtasks -= 1; + } else { + break; // All children at minimum (1), can't reduce further + } + } + + // Expand if under budget: add to children with the highest cost that can expand + while total_subtasks < task_count_budget { + let mut best_idx = None; + let mut best_cost = 0; + + for (i, &cost) in costs.iter().enumerate() { + let current = child_subtask_counts[i]; + let can_expand = max_tasks[i].is_none_or(|max| current < max); + if can_expand && cost > best_cost { + best_idx = Some(i); + best_cost = cost; + } + } + + if let Some(idx) = best_idx { + child_subtask_counts[idx] += 1; + total_subtasks += 1; + } else { + break; // All children at max, can't expand further + } + } + + // ==================== Phase 2: Pack subtasks into output tasks ==================== + // Distribute child subtasks evenly across output tasks by count. + // Process children in order for deterministic output. + + let total_subtasks: usize = child_subtask_counts.iter().sum(); + let base_per_task = total_subtasks / task_count_budget; + let mut extra = total_subtasks % task_count_budget; + + let mut result = vec![vec![]; task_count_budget]; + let mut task_idx = 0; + let mut current_count = 0; + let mut task_capacity = base_per_task + if extra > 0 { 1 } else { 0 }; + + for (child_idx, &subtask_count) in child_subtask_counts.iter().enumerate() { + for subtask_i in 0..subtask_count { + result[task_idx].push(( + child_idx, + DistributedTaskContext { + task_index: subtask_i, + task_count: subtask_count, + }, + )); + current_count += 1; + + // Move to next output task if capacity reached (unless we're at the last task) + if current_count >= task_capacity && task_idx < task_count_budget - 1 { + task_idx += 1; + current_count = 0; + extra = extra.saturating_sub(1); + task_capacity = base_per_task + if extra > 0 { 1 } else { 0 }; + } + } + } + + Ok(result) +} + +/// private trait just for testability purposes. +pub(super) trait CostChild { + fn cost(&self) -> usize; + fn max_tasks(&self) -> Option; +} + +impl CostChild for AnnotatedPlan { + fn cost(&self) -> usize { + self.cost_aggregated_until_network_boundary() + } + + fn max_tasks(&self) -> Option { + self.max_task_count_restriction + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn children_split_all_1_task() -> Result<()> { + assert_eq!( + children_isolator_union_split(&[(1, None), (1, None), (1, None)], 3)?, + vec![ + vec![(0, ctx(0, 1))], + vec![(1, ctx(0, 1))], + vec![(2, ctx(0, 1))] + ] + ); + assert_eq!( + children_isolator_union_split(&[(1, None), (1, None), (1, None)], 2)?, + vec![vec![(0, ctx(0, 1)), (1, ctx(0, 1))], vec![(2, ctx(0, 1))]] + ); + assert_eq!( + children_isolator_union_split(&[(1, None), (1, None), (1, None)], 1)?, + vec![vec![(0, ctx(0, 1)), (1, ctx(0, 1)), (2, ctx(0, 1))]] + ); + Ok(()) + } + + #[test] + fn children_isolator_union_split_different_tasks() -> Result<()> { + assert_eq!( + children_isolator_union_split(&[(1, None), (2, None), (3, None)], 6)?, + vec![ + vec![(0, ctx(0, 1))], + vec![(1, ctx(0, 2))], + vec![(1, ctx(1, 2))], + vec![(2, ctx(0, 3))], + vec![(2, ctx(1, 3))], + vec![(2, ctx(2, 3))] + ] + ); + assert_eq!( + children_isolator_union_split(&[(1, None), (2, None), (3, None)], 5)?, + vec![ + vec![(0, ctx(0, 1))], + vec![(1, ctx(0, 2))], + vec![(1, ctx(1, 2))], + vec![(2, ctx(0, 2))], + vec![(2, ctx(1, 2))], + ] + ); + assert_eq!( + children_isolator_union_split(&[(1, None), (2, None), (3, None)], 4)?, + vec![ + vec![(0, ctx(0, 1))], + vec![(1, ctx(0, 1))], + vec![(2, ctx(0, 2))], + vec![(2, ctx(1, 2))], + ] + ); + assert_eq!( + children_isolator_union_split(&[(1, None), (2, None), (3, None)], 3)?, + vec![ + vec![(0, ctx(0, 1))], + vec![(1, ctx(0, 1))], + vec![(2, ctx(0, 1))], + ] + ); + assert_eq!( + children_isolator_union_split(&[(1, None), (2, None), (3, None)], 2)?, + vec![vec![(0, ctx(0, 1)), (1, ctx(0, 1))], vec![(2, ctx(0, 1))]] + ); + assert_eq!( + children_isolator_union_split(&[(1, None), (2, None), (3, None)], 1)?, + vec![vec![(0, ctx(0, 1)), (1, ctx(0, 1)), (2, ctx(0, 1))]] + ); + Ok(()) + } + + /// Tests the doc example: expensive child with max_tasks=1 restriction. + /// Even though child 0 has cost=4 (would ideally get 2 tasks), it's capped at 1. + #[test] + fn expensive_child_with_max_tasks_restriction() -> Result<()> { + // From doc: [{cost: 4, max: Some(1)}, {cost: 1, max: None}, {cost: 1, max: None}] + // With budget=3, child 0 is forced to 1 task despite high cost. + assert_eq!( + children_isolator_union_split(&[(4, Some(1)), (1, None), (1, None)], 3)?, + vec![ + vec![(0, ctx(0, 1))], + vec![(1, ctx(0, 1))], + vec![(2, ctx(0, 1))] + ] + ); + Ok(()) + } + + /// Tests that max_tasks caps expansion when budget exceeds what children can use. + #[test] + fn max_tasks_prevents_over_expansion() -> Result<()> { + // All children capped at 1 task each, but budget is 5. + // Should result in 3 used tasks with 2 empty ones. + assert_eq!( + children_isolator_union_split(&[(10, Some(1)), (10, Some(1)), (10, Some(1))], 5)?, + vec![ + vec![(0, ctx(0, 1))], + vec![(1, ctx(0, 1))], + vec![(2, ctx(0, 1))], + vec![], + vec![] + ] + ); + Ok(()) + } + + #[test] + fn unbounded_child_absorbs_budget() -> Result<()> { + // Children 0,1,2 are capped at 1 task each. Child 3 is expensive and unbounded. + // With budget=6, child 3 should expand to fill the remaining 3 tasks. + assert_eq!( + children_isolator_union_split( + &[(1, Some(1)), (1, Some(1)), (1, Some(1)), (10, None)], + 6 + )?, + vec![ + vec![(0, ctx(0, 1))], + vec![(1, ctx(0, 1))], + vec![(2, ctx(0, 1))], + vec![(3, ctx(0, 3))], + vec![(3, ctx(1, 3))], + vec![(3, ctx(2, 3))], + ] + ); + Ok(()) + } + + /// NOTE(gabotechs): This test proves how this approach has some limitations. + /// + /// One could argue that the correct thing to do here is to actually try to spread + /// the last child across tasks, but this algorithm doesn't. + #[test] + fn unbounded_expensive_child_just_uses_1_task() -> Result<()> { + assert_eq!( + children_isolator_union_split( + &[(1, Some(1)), (1, Some(1)), (1, Some(1)), (10, None)], + 4 + )?, + vec![ + vec![(0, ctx(0, 1))], + vec![(1, ctx(0, 1))], + vec![(2, ctx(0, 1))], + vec![(3, ctx(0, 1))], + ] + ); + Ok(()) + } + + #[test] + fn many_free_children_does_not_prevent_expensive_one_from_being_distributed() -> Result<()> { + assert_eq!( + children_isolator_union_split(&[(0, None), (0, None), (0, None), (10, None)], 2)?, + vec![ + vec![(0, ctx(0, 1)), (1, ctx(0, 1))], + vec![(2, ctx(0, 1)), (3, ctx(0, 1))], + ] + ); + Ok(()) + } + + fn ctx(task_index: usize, task_count: usize) -> DistributedTaskContext { + DistributedTaskContext { + task_index, + task_count, + } + } + + impl CostChild for (usize, Option) { + fn cost(&self) -> usize { + self.0 + } + + fn max_tasks(&self) -> Option { + self.1 + } + } +} diff --git a/src/distributed_planner/distributed_config.rs b/src/distributed_planner/distributed_config.rs index f3c3f8c6..c845f70b 100644 --- a/src/distributed_planner/distributed_config.rs +++ b/src/distributed_planner/distributed_config.rs @@ -1,26 +1,12 @@ -use crate::TaskEstimator; -use crate::distributed_planner::task_estimator::CombinedTaskEstimator; +use crate::distributed_planner::distributed_planner_extension::CombinedDistributedPlannerExtension; use crate::networking::{ChannelResolverExtension, WorkerResolverExtension}; -use datafusion::common::utils::get_available_parallelism; use datafusion::common::{DataFusionError, extensions_options, not_impl_err, plan_err}; use datafusion::config::{ConfigExtension, ConfigField, ConfigOptions, Visit}; use std::fmt::{Debug, Formatter}; -use std::sync::Arc; extensions_options! { /// Configuration for the distributed planner. pub struct DistributedConfig { - /// Sets the maximum amount of files that will be assigned to each task. Reducing this - /// number will spawn more tasks for the same number of files. This only applies when - /// estimating tasks for stages containing `DataSourceExec` nodes with `FileScanConfig` - /// implementations. - pub files_per_task: usize, default = files_per_task_default() - /// Task multiplying factor for when a node declares that it changes the cardinality - /// of the data: - /// - If a node is increasing the cardinality of the data, this factor will increase. - /// - If a node reduces the cardinality of the data, this factor will decrease. - /// - In any other situation, this factor is left intact. - pub cardinality_task_count_factor: f64, default = cardinality_task_count_factor_default() /// Upon shuffling over the network, data streams need to be disassembled in a lot of output /// partitions, which means the resulting streams might contain a lot of tiny record batches /// to be sent over the wire. This parameter controls the batch size in number of rows for @@ -48,9 +34,18 @@ extensions_options! { /// The compression used for sending data over the network between workers. /// It can be set to either `zstd`, `lz4` or `none`. pub compression: String, default = "lz4".to_string() - /// Collection of [TaskEstimator]s that will be applied to leaf nodes in order to + /// How many bytes is each partition expected to process. This is used to calculate how many + /// partitions are suitable to be used for driving the query forward. If the result of the + /// calculation is that more partitions than `datafusion.execution.target_partitions` are + /// needed, the query will use more workers. + pub bytes_processed_per_partition: usize, default = 8 * 1024 * 1024 + /// Distributed DataFusion relies on row count estimation in order to infer how many workers + /// should be used in serving the query. Some plans might not implement any kind of row count + /// estimation, and this parameter sets the default estimated row count for those plans. + pub default_estimated_row_count: Option, default = Some(0) + /// Collection of [DistributedPlannerExtension]s that will be applied to leaf nodes in order to /// estimate how many tasks should be spawned for the [Stage] containing the leaf node. - pub(crate) __private_task_estimator: CombinedTaskEstimator, default = CombinedTaskEstimator::default() + pub(crate) __private_distributed_planner_extension: CombinedDistributedPlannerExtension, default = CombinedDistributedPlannerExtension::default() /// [ChannelResolver] implementation that tells the distributed planner information about /// the available workers ready to execute distributed tasks. pub(crate) __private_channel_resolver: ChannelResolverExtension, default = ChannelResolverExtension::default() @@ -60,36 +55,7 @@ extensions_options! { } } -fn files_per_task_default() -> usize { - if cfg!(test) || cfg!(feature = "integration") { - 1 - } else { - get_available_parallelism() - } -} - -fn cardinality_task_count_factor_default() -> f64 { - if cfg!(test) || cfg!(feature = "integration") { - 1.5 - } else { - 1.0 - } -} - impl DistributedConfig { - /// Appends a [TaskEstimator] to the list. [TaskEstimator] will be executed sequentially in - /// order on leaf nodes, and the first one to provide a value is the one that gets to decide - /// how many tasks are used for that [Stage]. - pub fn with_task_estimator( - mut self, - task_estimator: impl TaskEstimator + Send + Sync + 'static, - ) -> Self { - self.__private_task_estimator - .user_provided - .push(Arc::new(task_estimator)); - self - } - /// Gets the [DistributedConfig] from the [ConfigOptions]'s extensions. pub fn from_config_options(cfg: &ConfigOptions) -> Result<&Self, DataFusionError> { let Some(distributed_cfg) = cfg.extensions.get::() else { @@ -111,7 +77,7 @@ impl ConfigExtension for DistributedConfig { const PREFIX: &'static str = "distributed"; } -// FIXME: Ideally, both ChannelResolverExtension and TaskEstimators would be passed as +// FIXME: Ideally, all this structs would be passed as // extensions in SessionConfig's AnyMap instead of the ConfigOptions. However, we need // to pass this as ConfigOptions as we need these two fields to be present during // planning in the DistributedPhysicalOptimizerRule, and the signature of the optimize() @@ -151,7 +117,7 @@ impl Debug for WorkerResolverExtension { } } -impl ConfigField for CombinedTaskEstimator { +impl ConfigField for CombinedDistributedPlannerExtension { fn visit(&self, _: &mut V, _: &str, _: &'static str) { //nothing to do. } @@ -161,8 +127,8 @@ impl ConfigField for CombinedTaskEstimator { } } -impl Debug for CombinedTaskEstimator { +impl Debug for CombinedDistributedPlannerExtension { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "TaskEstimators") + write!(f, "CombinedDistributedPlannerExtension") } } diff --git a/src/distributed_planner/distributed_physical_optimizer_rule.rs b/src/distributed_planner/distributed_physical_optimizer_rule.rs index 1f31e171..116864c5 100644 --- a/src/distributed_planner/distributed_physical_optimizer_rule.rs +++ b/src/distributed_planner/distributed_physical_optimizer_rule.rs @@ -4,8 +4,8 @@ use crate::distributed_planner::plan_annotator::{ AnnotatedPlan, PlanOrNetworkBoundary, annotate_plan, }; use crate::{ - DistributedConfig, DistributedExec, NetworkBroadcastExec, NetworkCoalesceExec, - NetworkShuffleExec, TaskEstimator, + DistributedConfig, DistributedExec, DistributedPlannerExtension, NetworkBroadcastExec, + NetworkCoalesceExec, NetworkShuffleExec, }; use datafusion::config::ConfigOptions; use datafusion::error::DataFusionError; @@ -81,7 +81,7 @@ impl PhysicalOptimizerRule for DistributedPhysicalOptimizerRule { /// appropriately placed. This step performs the following modifications to the original /// [ExecutionPlan]: /// - The leaf nodes are scaled up in parallelism based on the number of distributed tasks in -/// which they are going to run. This is configurable by the user via the [TaskEstimator] trait. +/// which they are going to run. This is configurable by the user via the [DistributedPlannerExtension] trait. /// - The appropriate network boundaries are placed in the plan depending on how it was annotated, /// so new nodes like [NetworkBroadcastExec], [NetworkCoalesceExec] and [NetworkShuffleExec] will be present. fn distribute_plan( @@ -91,9 +91,12 @@ fn distribute_plan( stage_id: &mut usize, ) -> Result, DataFusionError> { let d_cfg = DistributedConfig::from_config_options(cfg)?; + let task_count = annotated_plan.task_count()?; let children = annotated_plan.children; - let task_count = annotated_plan.task_count.as_usize(); - let max_child_task_count = children.iter().map(|v| v.task_count.as_usize()).max(); + let mut max_child_task_count = 0; + for child in &children { + max_child_task_count = std::cmp::max(max_child_task_count, child.task_count()?) + } let new_children = children .into_iter() .map(|child| distribute_plan(child, cfg, query_id, stage_id)) @@ -102,11 +105,9 @@ fn distribute_plan( // This is a leaf node. It needs to be scaled up in order to account for it running in // multiple tasks. PlanOrNetworkBoundary::Plan(plan) if plan.children().is_empty() => { - let scaled_up = d_cfg.__private_task_estimator.scale_up_leaf_node( - &plan, - annotated_plan.task_count.as_usize(), - cfg, - ); + let scaled_up = d_cfg + .__private_distributed_planner_extension + .scale_up_leaf_node(&plan, task_count, cfg); Ok(scaled_up.unwrap_or(plan)) } // This is a normal intermediate plan, just pass it through with the mapped children. @@ -115,7 +116,7 @@ fn distribute_plan( PlanOrNetworkBoundary::Shuffle => { // It would need a network boundary, but on both sides of the boundary there is just 1 task, // so we are fine with not introducing any network boundary. - if task_count == 1 && max_child_task_count == Some(1) { + if task_count == 1 && max_child_task_count == 1 { return require_one_child(new_children); } let node = Arc::new(NetworkShuffleExec::try_new( @@ -123,7 +124,7 @@ fn distribute_plan( query_id, *stage_id, task_count, - max_child_task_count.unwrap_or(1), + max_child_task_count, )?); stage_id.add_assign(1); Ok(node) @@ -133,7 +134,7 @@ fn distribute_plan( PlanOrNetworkBoundary::Coalesce => { // It would need a network boundary, but on both sides of the boundary there is just 1 task, // so we are fine with not introducing any network boundary. - if task_count == 1 && max_child_task_count == Some(1) { + if task_count == 1 && max_child_task_count == 1 { return require_one_child(new_children); } let node = Arc::new(NetworkCoalesceExec::try_new( @@ -141,7 +142,7 @@ fn distribute_plan( query_id, *stage_id, task_count, - max_child_task_count.unwrap_or(1), + max_child_task_count, )?); stage_id.add_assign(1); Ok(node) @@ -151,7 +152,7 @@ fn distribute_plan( PlanOrNetworkBoundary::Broadcast => { // It would need a network boundary, but on both sides of the boundary there is just 1 task, // so we are fine with not introducing any network boundary. - if task_count == 1 && max_child_task_count == Some(1) { + if task_count == 1 && max_child_task_count == 1 { return require_one_child(new_children); } let node = Arc::new(NetworkBroadcastExec::try_new( @@ -159,7 +160,7 @@ fn distribute_plan( query_id, *stage_id, task_count, - max_child_task_count.unwrap_or(1), + max_child_task_count, )?); stage_id.add_assign(1); Ok(node) @@ -171,8 +172,7 @@ fn distribute_plan( mod tests { use crate::test_utils::in_memory_channel_resolver::InMemoryWorkerResolver; use crate::test_utils::plans::{ - BuildSideOneTaskEstimator, TestPlanOptions, base_session_builder, context_with_query, - sql_to_physical_plan, + TestPlanOptions, base_session_builder, context_with_query, sql_to_physical_plan, }; use crate::{ DistributedExt, DistributedPhysicalOptimizerRule, assert_snapshot, display_plan_ascii, @@ -231,16 +231,16 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[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))] │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=1 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([RainToday@0], 12), input_partitions=1 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet @@ -300,54 +300,32 @@ mod tests { } #[tokio::test] - async fn test_aggregation_with_high_cardinality_factor() { + async fn test_aggregation_with_a_lot_of_files_per_task() { let query = r#" SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*) "#; let plan = sql_to_explain(query, |b| { b.with_distributed_worker_resolver(InMemoryWorkerResolver::new(3)) - .with_distributed_cardinality_effect_task_scale_factor(3.0) - .unwrap() }) .await; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[p0..p3] - │ RepartitionExec: partitioning=Hash([RainToday@0], 4), input_partitions=1 - │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[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))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── - "); - } - - #[tokio::test] - async fn test_aggregation_with_a_lot_of_files_per_task() { - let query = r#" - SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*) - "#; - let plan = sql_to_explain(query, |b| { - b.with_distributed_worker_resolver(InMemoryWorkerResolver::new(3)) - .with_distributed_files_per_task(3) - .unwrap() - }) - .await; - assert_snapshot!(plan, @r" - ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] - SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] - ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] - AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - RepartitionExec: partitioning=Hash([RainToday@0], 4), input_partitions=3 - AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([RainToday@0], 12), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet + └────────────────────────────────────────────────── "); } @@ -364,16 +342,16 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[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))] │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=1 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([RainToday@0], 12), input_partitions=1 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet @@ -495,14 +473,14 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[p0..p3] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=1 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 12), input_partitions=1 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday, WindGustDir], file_type=parquet @@ -720,13 +698,13 @@ mod tests { ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] + │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + │ 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], file_type=parquet └────────────────────────────────────────────────── ") @@ -757,13 +735,13 @@ mod tests { │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2, RainToday@3] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] + │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + │ 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], file_type=parquet └────────────────────────────────────────────────── ") @@ -794,13 +772,13 @@ mod tests { ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] + │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + │ 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], file_type=parquet └────────────────────────────────────────────────── ") @@ -826,26 +804,38 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=2 │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=2 │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=2 │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 + │ 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], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 + │ 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], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 + │ 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], file_type=parquet + └────────────────────────────────────────────────── "); } @@ -856,7 +846,7 @@ mod tests { FROM weather a INNER JOIN weather b ON a."RainToday" = b."RainToday" "#; - let plan = sql_to_explain_with_broadcast_one_to_many(query, 3).await; + let plan = sql_to_explain_with_broadcast(query, 3, true).await; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec @@ -865,13 +855,14 @@ mod tests { ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=1 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] + │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + │ 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], file_type=parquet └────────────────────────────────────────────────── "); } @@ -891,18 +882,6 @@ mod tests { sql_to_plan_with_options(query, num_workers, broadcast_enabled, true).await } - async fn sql_to_explain_with_broadcast_one_to_many(query: &str, num_workers: usize) -> String { - let options = TestPlanOptions { - target_partitions: 4, - num_workers, - broadcast_enabled: true, - }; - explain_test_plan(query, options, true, |b| { - b.with_distributed_task_estimator(BuildSideOneTaskEstimator) - }) - .await - } - async fn sql_to_plan_with_options( query: &str, num_workers: usize, @@ -933,7 +912,9 @@ mod tests { builder.with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)); } let builder = configure(builder); - let (ctx, query) = context_with_query(builder, query).await; + let (mut ctx, query) = context_with_query(builder, query).await; + ctx.set_distributed_bytes_processed_per_partition(1000) + .unwrap(); let df = ctx.sql(&query).await.unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); diff --git a/src/distributed_planner/distributed_planner_extension.rs b/src/distributed_planner/distributed_planner_extension.rs new file mode 100644 index 00000000..45c21c86 --- /dev/null +++ b/src/distributed_planner/distributed_planner_extension.rs @@ -0,0 +1,156 @@ +use crate::config_extension_ext::set_distributed_option_extension; +use crate::{DistributedConfig, PartitionIsolatorExec}; +use datafusion::catalog::memory::DataSourceExec; +use datafusion::config::ConfigOptions; +use datafusion::datasource::physical_plan::FileScanConfig; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::SessionConfig; +use delegate::delegate; +use std::fmt::Debug; +use std::sync::Arc; + +/// Extends the behavior of the distributed planner while interacting with one or more specific +/// [ExecutionPlan]s. Typically used for handling custom user defined [ExecutionPlan]s in a +/// distributed context. +pub trait DistributedPlannerExtension { + /// Signals the distributed planner that the provided node must run in at most N tasks. + /// Returning Some(1) effectively tells the planner that the node cannot be distributed. + /// + /// This is useful when users have custom nodes that cannot be executed in more than one task + /// because of technical limitations. These nodes might still have the chance to be executed + /// in a distributed query, but the stage that handles them will be force-limited to one task. + fn max_tasks(&self, _plan: &Arc, _cfg: &ConfigOptions) -> Option { + None + } + + /// Once the distributed planner has decided how many tasks should be used in each stage, it + /// will call this method so that [ExecutionPlan]s have the chance to get "scaled up" to account + /// for the provided `task_count`. For example, when reading from multiple partitioned files, + /// these might get repartitioned further in order to account for it being run in multiple + /// machines. + fn scale_up_leaf_node( + &self, + plan: &Arc, + task_count: usize, + cfg: &ConfigOptions, + ) -> Option>; +} + +impl DistributedPlannerExtension for Arc { + delegate! { + to self.as_ref() { + fn max_tasks(&self, plan: &Arc, cfg: &ConfigOptions) -> Option; + fn scale_up_leaf_node(&self, plan: &Arc, task_count: usize, cfg: &ConfigOptions) -> Option>; + } + } +} + +impl DistributedPlannerExtension for Arc { + delegate! { + to self.as_ref() { + fn max_tasks(&self, plan: &Arc, cfg: &ConfigOptions) -> Option; + fn scale_up_leaf_node(&self, plan: &Arc, task_count: usize, cfg: &ConfigOptions) -> Option>; + } + } +} + +pub(crate) fn set_distributed_planner_extension( + cfg: &mut SessionConfig, + estimator: impl DistributedPlannerExtension + Send + Sync + 'static, +) { + let opts = cfg.options_mut(); + if let Some(distributed_cfg) = opts.extensions.get_mut::() { + distributed_cfg + .__private_distributed_planner_extension + .user_provided + .push(Arc::new(estimator)); + } else { + let mut estimators = CombinedDistributedPlannerExtension::default(); + estimators.user_provided.push(Arc::new(estimator)); + set_distributed_option_extension( + cfg, + DistributedConfig { + __private_distributed_planner_extension: estimators, + ..Default::default() + }, + ) + } +} + +/// [DistributedPlannerExtension] implementation that acts on [DataSourceExec] nodes that contain +/// [FileScanConfig]s data sources (e.g., Parquet or CSV files). it will read the +/// [DistributedConfig].`files_per_task` field and assigns as many tasks as needed so that +/// no task handles more than the configured files. +#[derive(Debug)] +struct FileScanConfigDistributedPlannerExtension; + +impl DistributedPlannerExtension for FileScanConfigDistributedPlannerExtension { + fn scale_up_leaf_node( + &self, + plan: &Arc, + task_count: usize, + _cfg: &ConfigOptions, + ) -> Option> { + if task_count == 1 { + return Some(Arc::clone(plan)); + } + // Based on the task count, attempt to scale up the partitions in the DataSourceExec by + // repartitioning it. This will result in a DataSourceExec with potentially a lot of + // partitions, but as we are going to wrap it with PartitionIsolatorExec, that's fine. + let dse: &DataSourceExec = plan.as_any().downcast_ref()?; + let file_scan: &FileScanConfig = dse.data_source().as_any().downcast_ref()?; + + let mut new_file_scan = file_scan.clone(); + new_file_scan.file_groups.clear(); + for file_group in file_scan.file_groups.clone() { + new_file_scan + .file_groups + .extend(file_group.split_files(task_count)); + } + let plan = DataSourceExec::from_data_source(new_file_scan); + Some(Arc::new(PartitionIsolatorExec::new(plan, task_count))) + } +} + +/// Tries multiple user-provided [DistributedPlannerExtension]s until one returns an estimation. If none +/// returns an estimation, a set of default [DistributedPlannerExtension] implementations is tried. Right +/// now the only default [DistributedPlannerExtension] is [FileScanConfigDistributedPlannerExtension]. +#[derive(Clone, Default)] +pub(crate) struct CombinedDistributedPlannerExtension { + pub(crate) user_provided: Vec>, +} + +impl DistributedPlannerExtension for CombinedDistributedPlannerExtension { + fn max_tasks(&self, plan: &Arc, cfg: &ConfigOptions) -> Option { + for estimator in &self.user_provided { + if let Some(max_tasks) = estimator.max_tasks(plan, cfg) { + return Some(max_tasks); + } + } + None + } + + fn scale_up_leaf_node( + &self, + plan: &Arc, + task_count: usize, + cfg: &ConfigOptions, + ) -> Option> { + for estimator in &self.user_provided { + if let Some(result) = estimator.scale_up_leaf_node(plan, task_count, cfg) { + return Some(result); + } + } + // We want to execute the default estimators last so that the user-provided ones have + // a chance of providing an estimation. + // If none of the user-provided returned an estimation, the default ones are used. + for default_estimator in + [&FileScanConfigDistributedPlannerExtension as &dyn DistributedPlannerExtension] + { + if let Some(result) = default_estimator.scale_up_leaf_node(plan, task_count, cfg) { + return Some(result); + } + } + None + } +} diff --git a/src/distributed_planner/mod.rs b/src/distributed_planner/mod.rs index 2f9b2895..1f31be9d 100644 --- a/src/distributed_planner/mod.rs +++ b/src/distributed_planner/mod.rs @@ -1,14 +1,17 @@ mod batch_coalescing_below_network_boundaries; +mod children_isolator_union_split; mod distributed_config; mod distributed_physical_optimizer_rule; +mod distributed_planner_extension; mod insert_broadcast; mod network_boundary; mod plan_annotator; -mod task_estimator; +mod statistics; pub(crate) use batch_coalescing_below_network_boundaries::batch_coalescing_below_network_boundaries; pub use distributed_config::DistributedConfig; pub use distributed_physical_optimizer_rule::DistributedPhysicalOptimizerRule; +pub use distributed_planner_extension::DistributedPlannerExtension; +pub(crate) use distributed_planner_extension::set_distributed_planner_extension; pub use network_boundary::{NetworkBoundary, NetworkBoundaryExt}; -pub(crate) use task_estimator::set_distributed_task_estimator; -pub use task_estimator::{TaskCountAnnotation, TaskEstimation, TaskEstimator}; +pub use statistics::plan_statistics; diff --git a/src/distributed_planner/plan_annotator.rs b/src/distributed_planner/plan_annotator.rs index f54b4271..fc5fe6d7 100644 --- a/src/distributed_planner/plan_annotator.rs +++ b/src/distributed_planner/plan_annotator.rs @@ -1,16 +1,19 @@ -use crate::TaskCountAnnotation::{Desired, Maximum}; +use crate::distributed_planner::children_isolator_union_split::children_isolator_union_split; +use crate::distributed_planner::statistics::{ + Complexity, LinearComplexity, calculate_compute_complexity, plan_statistics, +}; use crate::execution_plans::ChildrenIsolatorUnionExec; -use crate::{BroadcastExec, DistributedConfig, TaskCountAnnotation, TaskEstimator}; -use datafusion::common::{DataFusionError, plan_datafusion_err}; +use crate::{BroadcastExec, DistributedConfig, DistributedPlannerExtension}; +use datafusion::common::{DataFusionError, plan_datafusion_err, plan_err}; use datafusion::config::ConfigOptions; use datafusion::physical_expr::Partitioning; -use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion::physical_plan::execution_plan::CardinalityEffect; use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode}; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion::physical_plan::union::UnionExec; +use datafusion::physical_plan::{ExecutionPlan, Statistics}; +use itertools::Itertools; use std::fmt::{Debug, Formatter}; use std::sync::Arc; @@ -50,22 +53,126 @@ pub(super) struct AnnotatedPlan { pub(super) children: Vec, // annotation fields - /// How many distributed tasks this plan should run on. - pub(super) task_count: TaskCountAnnotation, + /// This node can only run in exactly 1 task. + pub(super) task_count: Option, + + /// What's the runtime complexity for the operations that need to happen in this node. + pub(super) complexity: Complexity, + + /// The maximum amount of tasks in which this node is allowed to run. + pub(super) max_task_count_restriction: Option, + + /// Stats about how many rows will this node return. + pub(super) stats: Statistics, +} + +impl AnnotatedPlan { + fn cost(&self) -> usize { + let output_stat = &self.stats; + let input_stats = self.children.iter().map(|v| &v.stats).collect::>(); + self.complexity.cost(output_stat, &input_stats) + } + + pub(super) fn cost_aggregated_until_network_boundary(&self) -> usize { + if self.plan_or_nb.is_network_boundary() { + return self.cost(); + } + let mut accumulated_compute_cost = self.cost(); + for input_child in &self.children { + accumulated_compute_cost += input_child.cost_aggregated_until_network_boundary(); + } + accumulated_compute_cost + } + + pub(super) fn task_count(&self) -> Result { + self.task_count.ok_or_else(|| { + plan_datafusion_err!( + "AnnotatedPlan {:?} does not have a task count assigned", + self.plan_or_nb + ) + }) + } + + // The plan needs a NetworkBoundary. At this point we have all the info we need for choosing + // the right size for the stage below, so what we need to do is take the calculated final + // task count and propagate to all the children that will eventually be part of the stage. + fn propagate_task_count_until_network_boundary( + &mut self, + task_count: usize, + d_cfg: &DistributedConfig, + ) -> Result<(), DataFusionError> { + self.task_count = Some(task_count); + let plan = match &self.plan_or_nb { + // If it's a normal plan, continue with the propagation. + PlanOrNetworkBoundary::Plan(plan) => plan, + // This is a network boundary. + // + // Nothing to propagate here, all the nodes below the network boundary were already + // assigned a task count, we do not want to overwrite it. + PlanOrNetworkBoundary::Broadcast => return Ok(()), + PlanOrNetworkBoundary::Shuffle => return Ok(()), + PlanOrNetworkBoundary::Coalesce => return Ok(()), + }; + + if d_cfg.children_isolator_unions && plan.as_any().is::() { + // Propagating through ChildrenIsolatorUnionExec is not that easy, each child will + // be executed in its own task, and therefore, they will act as if they were in executing + // in a non-distributed context. The ChildrenIsolatorUnionExec itself will make sure to + // determine which children to run and which to exclude depending on the task index in + // which it's running. + let task_idx_map = children_isolator_union_split(&self.children, task_count)?; + + for children_and_tasks in &task_idx_map { + for (child_i, task_ctx) in children_and_tasks { + let Some(child) = self.children.get_mut(*child_i) else { + return plan_err!( + "Error propagating task count from ChildrenIsolatorUnionExec: {child_i} index out of range for {} children.", + children_and_tasks.len() + ); + }; + + child.propagate_task_count_until_network_boundary(task_ctx.task_count, d_cfg)? + } + } + let c_i_union = ChildrenIsolatorUnionExec::from_children_and_task_counts( + plan.children().into_iter().cloned().collect(), + task_idx_map, + )?; + self.plan_or_nb = PlanOrNetworkBoundary::Plan(Arc::new(c_i_union)); + } else { + for child in &mut self.children { + child.propagate_task_count_until_network_boundary(task_count, d_cfg)?; + } + } + Ok(()) + } } impl Debug for AnnotatedPlan { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - fn fmt_dbg(f: &mut Formatter<'_>, plan: &AnnotatedPlan, depth: usize) -> std::fmt::Result { + fn fmt_dbg( + f: &mut Formatter<'_>, + annotation: &AnnotatedPlan, + depth: usize, + ) -> std::fmt::Result { write!( f, - "{}{:?}: task_count={:?}", + "{}{:?}: task_count={:?} output_rows={:?}", " ".repeat(depth * 2), - plan.plan_or_nb, - plan.task_count + annotation.plan_or_nb, + annotation.task_count, + annotation.stats.num_rows.get_value().unwrap_or(&0), )?; + write!(f, " compute_complexity=O({:?})", annotation.complexity)?; + write!( + f, + " accumulated_cost={:?}", + annotation.cost_aggregated_until_network_boundary() + )?; + let output_bytes = annotation.stats.total_byte_size.get_value().unwrap_or(&0); + write!(f, " output_bytes={output_bytes:?}")?; writeln!(f)?; - for child in plan.children.iter() { + for child in annotation.children.iter() { fmt_dbg(f, child, depth + 1)?; } Ok(()) @@ -81,26 +188,21 @@ impl Debug for AnnotatedPlan { /// This is the first step of the distribution process, where the plan structure is still left /// untouched and the existing nodes are just annotated for future steps to perform the distribution. /// -/// The plans are annotated in a bottom-to-top manner, starting with the leaf nodes all the way +/// The plans are annotated in a bottom-up manner, starting with the leaf nodes all the way /// to the head of the plan: /// -/// 1. Leaf nodes have the opportunity to provide an estimation of how many distributed tasks should -/// be used for the whole stage that will execute them. +/// 1. Leaf nodes have the opportunity to declare some statistics about how much data is going to +/// be pulled by them, along with other statistics like the number of distinct values per column. /// -/// 2. If a stage contains multiple leaf nodes, and all provide a task count estimation, the -/// biggest is taken. +/// 2. The function recurses up propagating statistics, until a network boundary is reached. /// -/// 3. When traversing the plan in a bottom-to-top fashion, this function looks for nodes that -/// either increase or reduce cardinality: -/// - If there's a node that increases cardinality, the next stage will spawn more tasks than -/// the current one. -/// - If there's a node that reduces cardinality, the next stage will spawn fewer tasks than the -/// current one. +/// 3. Upon reaching a network boundary, the function looks at the nodes below and calculates an +/// estimation of the compute cost of the plan below. The compute estimation is based on the +/// amount of rows that are expected to flow through it, the estimated size of each row, and how +/// compute hungry each node is. /// -/// 4. At a certain point, the function will reach a node that needs a network boundary below; in -/// that case, the node is annotated with a [PlanOrNetworkBoundary] value. At this point, all -/// the nodes below must reach a consensus about the final task count for the stage below the -/// network boundary. +/// 4. Based on the total aggregated compute cost for the plan below, a specific number of tasks +/// is assigned to all those nodes, as they all will run in the same stage. /// /// 5. This process is repeated recursively until all nodes are annotated. /// @@ -108,142 +210,113 @@ impl Debug for AnnotatedPlan { /// /// Following the process above, an annotated plan will look like this: /// -/// ```text -/// -/// ┌──────────────────────┐ task_count: Desired(3) (inherited from child) -/// │ CoalesceBatches │ network_boundary: None -/// └───────────▲──────────┘ -/// │ -/// ┌───────────┴──────────┐ task_count: Desired(3) (inherits from probe child) -/// │ HashJoin │ network_boundary: None -/// │ (CollectLeft Inner) │ -/// └─▲──────────────────▲─┘ -/// │ │ -/// │ Probe Side -/// │ │ -/// │ ┌───────────┴──────────┐ task_count: Desired(3) (inherited from child) -/// │ │ Projection │ network_boundary: None -/// │ └───────────▲──────────┘ -/// │ │ -/// │ ┌───────────┴──────────┐ task_count: Desired(3) (as this node requires a network boundary below, -/// │ │ Aggregation │ and the stage below reduces the cardinality of the data because of the -/// │ │ (Final) │ partial aggregation, we can choose a smaller amount of tasks) -/// │ └───────────▲──────────┘ network_boundary: Some(Shuffle) (because the child is a repartition) -/// │ │ -/// │ ┌───────────┴──────────┐ task_count: Desired(4) (inherited from child) -/// │ │ Repartition │ network_boundary: None -/// Build Side └───────────▲──────────┘ -/// │ │ -/// │ ┌───────────┴──────────┐ -/// │ │ Aggregation │ task_count: Desired(4) (inherited from child) -/// │ │ (Partial) │ network_boundary: None -/// │ └───────────▲──────────┘ -/// │ │ -/// │ ┌───────────┴──────────┐ task_count: Desired(4) (this was set by a TaskEstimator implementation) -/// │ │ DataSource │ network_boundary: None -/// │ └──────────────────────┘ -/// │ -/// │ -/// ┌───────────┴──────────┐ task_count: Desired(3) (inherits from the probe side) -/// │ CoalescePartitions │ network_boundary: Broadcast -/// └───────────▲──────────┘ -/// │ -/// ┌───────────┴──────────┐ task_count: Desired(2) (inherited from child) -/// │ BroadcastExec │ network_boundary: None -/// └───────────▲──────────┘ -/// │ -/// ┌───────────┴──────────┐ task_count: Desired(2) (this was set by a TaskEstimator implementation) -/// │ DataSource │ network_boundary: None -/// └──────────────────────┘ └──────────────────────┘ -/// ``` -/// -/// ``` +/// TODO: create a graphical example pub(super) fn annotate_plan( plan: Arc, cfg: &ConfigOptions, ) -> Result { - _annotate_plan(plan, None, cfg, true) + let mut annotation = _annotate_plan(plan, None, cfg)?; + let d_cfg = DistributedConfig::from_config_options(cfg)?; + + let stage_below_task_count = if let Some(n) = annotation.max_task_count_restriction { + n + } else { + let compute_cost = annotation.cost_aggregated_until_network_boundary(); + cost_based_task_count(compute_cost, cfg, d_cfg)? + }; + // This is the root node, it means that we have just finished annotating nodes for the + // subplan belonging to the head stage, so propagate the task count to all children. + annotation.propagate_task_count_until_network_boundary(stage_below_task_count, d_cfg)?; + Ok(annotation) } fn _annotate_plan( plan: Arc, parent: Option<&Arc>, cfg: &ConfigOptions, - root: bool, ) -> Result { let d_cfg = DistributedConfig::from_config_options(cfg)?; let broadcast_joins = d_cfg.broadcast_joins; - let estimator = &d_cfg.__private_task_estimator; - let n_workers = d_cfg.__private_worker_resolver.0.get_urls()?.len().max(1); + let estimator = &d_cfg.__private_distributed_planner_extension; let annotated_children = plan .children() .iter() - .map(|child| _annotate_plan(Arc::clone(child), Some(&plan), cfg, false)) + .map(|child| _annotate_plan(Arc::clone(child), Some(&plan), cfg)) .collect::, _>>()?; - if plan.children().is_empty() { + let mut max_task_count_restriction = estimator.max_tasks(&plan, cfg); + if annotated_children.is_empty() { // This is a leaf node, maybe a DataSourceExec, or maybe something else custom from the // user. We need to estimate how many tasks are needed for this leaf node, and we'll take // this decision into account when deciding how many tasks will be actually used. - return if let Some(estimate) = estimator.task_estimation(&plan, cfg) { - Ok(AnnotatedPlan { - plan_or_nb: PlanOrNetworkBoundary::Plan(plan), - children: Vec::new(), - task_count: estimate.task_count.limit(n_workers), - }) - } else { - // We could not determine how many tasks this leaf node should run on, so - // assume it cannot be distributed and use just 1 task. - Ok(AnnotatedPlan { + // We could not determine how many tasks this leaf node should run on, so + // assume it cannot be distributed and use just 1 task. + return Ok(match plan_statistics(&plan, &[], d_cfg.into()) { + Ok(output_row_stats) => AnnotatedPlan { + stats: output_row_stats, + complexity: calculate_compute_complexity(&plan), plan_or_nb: PlanOrNetworkBoundary::Plan(plan), children: Vec::new(), - task_count: Maximum(1), - }) - }; + + max_task_count_restriction, + task_count: None, + }, + Err(_) => { + // We know nothing about this plan. It has no row statistics, so we cannot assume + // that is fine to distribute it. + AnnotatedPlan { + stats: Statistics::new_unknown(&plan.schema()), + complexity: calculate_compute_complexity(&plan), + plan_or_nb: PlanOrNetworkBoundary::Plan(plan), + children: Vec::new(), + + max_task_count_restriction: Some(1), + task_count: None, + } + } + }); } - let mut task_count = estimator - .task_estimation(&plan, cfg) - .map_or(Desired(1), |v| v.task_count); if d_cfg.children_isolator_unions && plan.as_any().is::() { - // Unions have the chance to decide how many tasks they should run on. If there's a union - // with a bunch of children, the user might want to increase parallelism and increase the - // task count for the stage running that. - let mut count = 0; - for annotated_child in annotated_children.iter() { - count += annotated_child.task_count.as_usize(); - } - task_count = Desired(count); + // A UNION might have some children declaring a max_task_count_restriction, but if it's + // going to be converted into a ChildrenIsolatorUnionExec, it might be able to satisfy + // those restrictions while still distributing the stage, so we can skip the restriction + // propagation. } else if let Some(node) = plan.as_any().downcast_ref::() && node.mode == PartitionMode::CollectLeft && !broadcast_joins { - // Only distriubte CollectLeft HashJoins after we broadcast more intelligently or when it + // Only distribute CollectLeft HashJoins after we broadcast more intelligently or when it // is explicitly enabled. - task_count = Maximum(1); + max_task_count_restriction = Some(1) } else { // The task count for this plan is decided by the biggest task count from the children; unless // a child specifies a maximum task count, in that case, the maximum is respected. Some // nodes can only run in one task. If there is a subplan with a single node declaring that // it can only run in one task, all the rest of the nodes in the stage need to respect it. for annotated_child in annotated_children.iter() { - task_count = match (task_count, &annotated_child.task_count) { - (Desired(desired), Desired(child)) => Desired(desired.max(*child)), - (Maximum(max), Desired(_)) => Maximum(max), - (Desired(_), Maximum(max)) => Maximum(*max), - (Maximum(max_1), Maximum(max_2)) => Maximum(max_1.min(*max_2)), + max_task_count_restriction = match ( + max_task_count_restriction, + annotated_child.max_task_count_restriction, + ) { + (None, None) => None, + (Some(max), None) => Some(max), + (None, Some(max)) => Some(max), + (Some(max_1), Some(max_2)) => Some(max_1.min(max_2)), }; } } - task_count = task_count.limit(n_workers); - - // Wrap the node with a boundary node if the parent marks it. + let input_row_stats = annotated_children.iter().map(|v| &v.stats).collect_vec(); let mut annotation = AnnotatedPlan { plan_or_nb: PlanOrNetworkBoundary::Plan(Arc::clone(&plan)), + stats: plan_statistics(&plan, &input_row_stats, d_cfg.into())?, children: annotated_children, - task_count: task_count.clone(), + + max_task_count_restriction, + complexity: calculate_compute_complexity(&plan), + task_count: None, }; // Upon reaching a hash repartition, we need to introduce a shuffle right above it. @@ -251,8 +324,12 @@ fn _annotate_plan( if matches!(r_exec.partitioning(), Partitioning::Hash(_, _)) { annotation = AnnotatedPlan { plan_or_nb: PlanOrNetworkBoundary::Shuffle, + stats: annotation.stats.clone(), children: vec![annotation], - task_count, + + max_task_count_restriction: None, + complexity: Complexity::Linear(LinearComplexity::AllColumns), + task_count: None, }; } } else if let Some(parent) = parent @@ -268,127 +345,48 @@ fn _annotate_plan( if plan.as_any().is::() { annotation = AnnotatedPlan { plan_or_nb: PlanOrNetworkBoundary::Broadcast, + stats: annotation.stats.clone(), children: vec![annotation], - task_count, + + max_task_count_restriction: None, + complexity: Complexity::Linear(LinearComplexity::AllColumns), + task_count: None, }; } else { annotation = AnnotatedPlan { plan_or_nb: PlanOrNetworkBoundary::Coalesce, + stats: annotation.stats.clone(), children: vec![annotation], - task_count, + + max_task_count_restriction: None, + complexity: Complexity::Linear(LinearComplexity::AllColumns), + task_count: None, }; } } - // The plan needs a NetworkBoundary. At this point we have all the info we need for choosing - // the right size for the stage below, so what we need to do is take the calculated final - // task count and propagate to all the children that will eventually be part of the stage. - fn propagate_task_count( - annotation: &mut AnnotatedPlan, - task_count: &TaskCountAnnotation, - d_cfg: &DistributedConfig, - ) -> Result<(), DataFusionError> { - annotation.task_count = task_count.clone(); - let plan = match &annotation.plan_or_nb { - // If it's a normal plan, continue with the propagation. - PlanOrNetworkBoundary::Plan(plan) => plan, - // Broadcast is a stage split only propagate a Maximum cap into the build stage. - PlanOrNetworkBoundary::Broadcast => { - if let Maximum(max) = task_count { - for child in annotation.children.iter_mut() { - let child_task_count = child.task_count.clone().limit(*max); - propagate_task_count(child, &child_task_count, d_cfg)?; - } - } - return Ok(()); - } - // This is a network boundary. - // - // Nothing to propagate here, all the nodes below the network boundary were already - // assigned a task count, we do not want to overwrite it. - PlanOrNetworkBoundary::Shuffle => return Ok(()), - PlanOrNetworkBoundary::Coalesce => return Ok(()), + if annotation.plan_or_nb.is_network_boundary() { + let Some(input) = annotation.children.first_mut() else { + return plan_err!("Found a network boundary without input"); }; - - if d_cfg.children_isolator_unions && plan.as_any().is::() { - // Propagating through ChildrenIsolatorUnionExec is not that easy, each child will - // be executed in its own task, and therefore, they will act as if they were in executing - // in a non-distributed context. The ChildrenIsolatorUnionExec itself will make sure to - // determine which children to run and which to exclude depending on the task index in - // which it's running. - let c_i_union = ChildrenIsolatorUnionExec::from_children_and_task_counts( - plan.children().into_iter().cloned(), - annotation.children.iter().map(|v| v.task_count.as_usize()), - task_count.as_usize(), - )?; - for children_and_tasks in c_i_union.task_idx_map.iter() { - for (child_i, task_ctx) in children_and_tasks { - if let Some(child) = annotation.children.get_mut(*child_i) { - propagate_task_count(child, &Maximum(task_ctx.task_count), d_cfg)? - }; - } - } - annotation.plan_or_nb = PlanOrNetworkBoundary::Plan(Arc::new(c_i_union)); + let stage_below_task_count = if let Some(n) = input.max_task_count_restriction { + n } else { - for child in &mut annotation.children { - propagate_task_count(child, task_count, d_cfg)?; - } - } - Ok(()) - } + let compute_cost = input.cost_aggregated_until_network_boundary(); + cost_based_task_count(compute_cost, cfg, d_cfg)? + }; - if annotation.plan_or_nb.is_network_boundary() { - // The plan is a network boundary, so everything below it belongs to the same stage. This + // The plan is a network boundary, so everything below belongs to the same stage. This // means that we need to propagate the task count to all the nodes in that stage. - for annotated_child in annotation.children.iter_mut() { - propagate_task_count(annotated_child, &annotation.task_count, d_cfg)?; - } + input.propagate_task_count_until_network_boundary(stage_below_task_count, d_cfg)?; // If the current plan that needs a NetworkBoundary boundary below is either a // CoalescePartitionsExec or a SortPreservingMergeExec, then we are sure that all the stage // that they are going to be part of needs to run in exactly one task. if matches!(annotation.plan_or_nb, PlanOrNetworkBoundary::Coalesce) { - annotation.task_count = Maximum(1); - return Ok(annotation); + annotation.max_task_count_restriction = Some(1); } - // From now and up in the plan, a new task count needs to be calculated for the next stage. - // Depending on the number of nodes that reduce/increase cardinality, the task count will be - // calculated based on the previous task count multiplied by a factor. - fn calculate_scale_factor(annotation: &AnnotatedPlan, f: f64) -> f64 { - let PlanOrNetworkBoundary::Plan(plan) = &annotation.plan_or_nb else { - return 1.0; - }; - - let mut sf = None; - for plan in &annotation.children { - sf = match sf { - None => Some(calculate_scale_factor(plan, f)), - Some(sf) => Some(sf.max(calculate_scale_factor(plan, f))), - } - } - - let sf = sf.unwrap_or(1.0); - match plan.cardinality_effect() { - CardinalityEffect::LowerEqual => sf / f, - CardinalityEffect::GreaterEqual => sf * f, - _ => sf, - } - } - let sf = calculate_scale_factor( - annotation.children.first().ok_or_else(|| { - plan_datafusion_err!("missing child in a plan annotated with a network boundary") - })?, - d_cfg.cardinality_task_count_factor, - ); - let prev_task_count = annotation.task_count.as_usize() as f64; - annotation.task_count = Desired((prev_task_count * sf).ceil() as usize); - Ok(annotation) - } else if root { - // If this is the root node, it means that we have just finished annotating nodes for the - // subplan belonging to the head stage, so propagate the task count to all children. - let task_count = annotation.task_count.clone(); - propagate_task_count(&mut annotation, &task_count, d_cfg)?; Ok(annotation) } else { // If this is not the root node, and it's also not a network boundary, then we don't need @@ -397,19 +395,35 @@ fn _annotate_plan( } } +fn cost_based_task_count( + cost: usize, + cfg: &ConfigOptions, + d_cfg: &DistributedConfig, +) -> Result { + if cost == 0 { + return Ok(1); + } + + let partitions_needed = + (cost as f64 / d_cfg.bytes_processed_per_partition as f64).ceil() as usize; + let partitions_available = cfg.execution.target_partitions; + let workers_needed = (partitions_needed as f64 / partitions_available as f64).ceil() as usize; + let workers_available = d_cfg.__private_worker_resolver.0.get_urls()?.len().max(1); + Ok(std::cmp::min(workers_needed, workers_available)) +} + #[cfg(test)] mod tests { use super::*; use crate::distributed_planner::insert_broadcast::insert_broadcast_execs; use crate::test_utils::plans::{ - BuildSideOneTaskEstimator, TestPlanOptions, base_session_builder, context_with_query, - sql_to_physical_plan, + BuildSideOneDistributedPlannerExtension, TestPlanOptions, base_session_builder, + context_with_query, sql_to_physical_plan, }; - use crate::{DistributedExt, TaskEstimation, TaskEstimator, assert_snapshot}; + use crate::{DistributedExt, DistributedPlannerExtension, assert_snapshot}; use datafusion::config::ConfigOptions; use datafusion::execution::SessionStateBuilder; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use datafusion::physical_plan::filter::FilterExec; /* schema for the "weather" table MinTemp [type=DOUBLE] [repetitiontype=OPTIONAL] @@ -442,7 +456,7 @@ mod tests { SELECT * FROM weather "#; let annotated = sql_to_annotated(query).await; - assert_snapshot!(annotated, @"DataSourceExec: task_count=Desired(3)") + assert_snapshot!(annotated, @"DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=75918 output_bytes=75918") } #[tokio::test] @@ -452,16 +466,16 @@ mod tests { "#; let annotated = sql_to_annotated(query).await; assert_snapshot!(annotated, @r" - ProjectionExec: task_count=Maximum(1) - SortPreservingMergeExec: task_count=Maximum(1) - [NetworkBoundary] Coalesce: task_count=Maximum(1) - SortExec: task_count=Desired(2) - ProjectionExec: task_count=Desired(2) - AggregateExec: task_count=Desired(2) - [NetworkBoundary] Shuffle: task_count=Desired(2) - RepartitionExec: task_count=Desired(3) - AggregateExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) + ProjectionExec: task_count=Some(1) output_rows=366 compute_complexity=O(1) accumulated_cost=8806 output_bytes=4402 + SortPreservingMergeExec: task_count=Some(1) output_rows=366 compute_complexity=O((Cols+1)) accumulated_cost=8805 output_bytes=4402 + [NetworkBoundary] Coalesce: task_count=Some(1) output_rows=366 compute_complexity=O(Cols) accumulated_cost=4402 output_bytes=4402 + SortExec: task_count=Some(4) output_rows=366 compute_complexity=O((Cols+1)*Log((Cols+1))) accumulated_cost=67863 output_bytes=4402 + ProjectionExec: task_count=Some(4) output_rows=366 compute_complexity=O(1) accumulated_cost=15027 output_bytes=4402 + AggregateExec: task_count=Some(4) output_rows=366 compute_complexity=O((Cols+Col0)) accumulated_cost=15026 output_bytes=4402 + [NetworkBoundary] Shuffle: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=4402 output_bytes=4402 + RepartitionExec: task_count=Some(4) output_rows=366 compute_complexity=O((Cols+1)) accumulated_cost=17609 output_bytes=4402 + AggregateExec: task_count=Some(4) output_rows=366 compute_complexity=O((Cols+Col0)) accumulated_cost=13206 output_bytes=4402 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=4402 output_bytes=4402 ") } @@ -472,10 +486,10 @@ mod tests { "#; let annotated = sql_to_annotated(query).await; assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) + HashJoinExec: task_count=Some(1) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(1) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 ") } @@ -506,25 +520,25 @@ mod tests { "#; let annotated = sql_to_annotated(query).await; assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - [NetworkBoundary] Coalesce: task_count=Maximum(1) - ProjectionExec: task_count=Desired(2) - AggregateExec: task_count=Desired(2) - [NetworkBoundary] Shuffle: task_count=Desired(2) - RepartitionExec: task_count=Desired(3) - AggregateExec: task_count=Desired(3) - FilterExec: task_count=Desired(3) - RepartitionExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) - ProjectionExec: task_count=Maximum(1) - AggregateExec: task_count=Maximum(1) - [NetworkBoundary] Shuffle: task_count=Maximum(1) - RepartitionExec: task_count=Desired(3) - AggregateExec: task_count=Desired(3) - FilterExec: task_count=Desired(3) - RepartitionExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) + HashJoinExec: task_count=Some(1) output_rows=74 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=17846 output_bytes=1332 + CoalescePartitionsExec: task_count=Some(1) output_rows=74 compute_complexity=O(1) accumulated_cost=2346 output_bytes=2345 + [NetworkBoundary] Coalesce: task_count=Some(1) output_rows=74 compute_complexity=O(Cols) accumulated_cost=2345 output_bytes=2345 + ProjectionExec: task_count=Some(2) output_rows=74 compute_complexity=O(1) accumulated_cost=5949 output_bytes=2345 + AggregateExec: task_count=Some(2) output_rows=74 compute_complexity=O((Cols+Col0)) accumulated_cost=5948 output_bytes=2345 + [NetworkBoundary] Shuffle: task_count=Some(2) output_rows=74 compute_complexity=O(Cols) accumulated_cost=2345 output_bytes=2345 + RepartitionExec: task_count=Some(4) output_rows=74 compute_complexity=O((Cols+1)) accumulated_cost=39280 output_bytes=2345 + AggregateExec: task_count=Some(4) output_rows=74 compute_complexity=O((Cols+Col1)) accumulated_cost=36934 output_bytes=2345 + FilterExec: task_count=Some(4) output_rows=74 compute_complexity=O((out_Cols+Col1)) accumulated_cost=30195 output_bytes=2345 + RepartitionExec: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=23448 output_bytes=11724 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=11724 output_bytes=11724 + ProjectionExec: task_count=Some(1) output_rows=74 compute_complexity=O(1) accumulated_cost=5949 output_bytes=2345 + AggregateExec: task_count=Some(1) output_rows=74 compute_complexity=O((Cols+Col0)) accumulated_cost=5948 output_bytes=2345 + [NetworkBoundary] Shuffle: task_count=Some(1) output_rows=74 compute_complexity=O(Cols) accumulated_cost=2345 output_bytes=2345 + RepartitionExec: task_count=Some(4) output_rows=74 compute_complexity=O((Cols+1)) accumulated_cost=39280 output_bytes=2345 + AggregateExec: task_count=Some(4) output_rows=74 compute_complexity=O((Cols+Col1)) accumulated_cost=36934 output_bytes=2345 + FilterExec: task_count=Some(4) output_rows=74 compute_complexity=O((out_Cols+Col1)) accumulated_cost=30195 output_bytes=2345 + RepartitionExec: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=23448 output_bytes=11724 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=11724 output_bytes=11724 ") } @@ -537,10 +551,10 @@ mod tests { "#; let annotated = sql_to_annotated(query).await; assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) + HashJoinExec: task_count=Some(1) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(1) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 ") } @@ -551,11 +565,11 @@ mod tests { "#; let annotated = sql_to_annotated(query).await; assert_snapshot!(annotated, @r" - AggregateExec: task_count=Desired(2) - [NetworkBoundary] Shuffle: task_count=Desired(2) - RepartitionExec: task_count=Desired(3) - AggregateExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) + AggregateExec: task_count=Some(4) output_rows=366 compute_complexity=O((Cols+Col0)) accumulated_cost=15026 output_bytes=4402 + [NetworkBoundary] Shuffle: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=4402 output_bytes=4402 + RepartitionExec: task_count=Some(4) output_rows=366 compute_complexity=O((Cols+1)) accumulated_cost=17609 output_bytes=4402 + AggregateExec: task_count=Some(4) output_rows=366 compute_complexity=O((Cols+Col0)) accumulated_cost=13206 output_bytes=4402 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=4402 output_bytes=4402 ") } @@ -568,14 +582,14 @@ mod tests { "#; let annotated = sql_to_annotated(query).await; assert_snapshot!(annotated, @r" - ChildrenIsolatorUnionExec: task_count=Desired(4) - FilterExec: task_count=Maximum(2) - RepartitionExec: task_count=Maximum(2) - DataSourceExec: task_count=Maximum(2) - ProjectionExec: task_count=Maximum(2) - FilterExec: task_count=Maximum(2) - RepartitionExec: task_count=Maximum(2) - DataSourceExec: task_count=Maximum(2) + ChildrenIsolatorUnionExec: task_count=Some(4) output_rows=148 compute_complexity=O(1) accumulated_cost=41058 output_bytes=2058 + FilterExec: task_count=Some(2) output_rows=74 compute_complexity=O((out_Cols+Col1)) accumulated_cost=20528 output_bytes=1466 + RepartitionExec: task_count=Some(2) output_rows=366 compute_complexity=O(Cols) accumulated_cost=14660 output_bytes=7330 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + ProjectionExec: task_count=Some(2) output_rows=74 compute_complexity=O(1) accumulated_cost=20529 output_bytes=592 + FilterExec: task_count=Some(2) output_rows=74 compute_complexity=O((out_Cols+Col1)) accumulated_cost=20528 output_bytes=1466 + RepartitionExec: task_count=Some(2) output_rows=366 compute_complexity=O(Cols) accumulated_cost=14660 output_bytes=7330 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 ") } @@ -588,9 +602,9 @@ mod tests { "#; let annotated = sql_to_annotated(query).await; assert_snapshot!(annotated, @r" - FilterExec: task_count=Desired(3) - RepartitionExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) + FilterExec: task_count=Some(4) output_rows=74 compute_complexity=O((out_Cols+(Col0+Col2))) accumulated_cost=29898 output_bytes=2052 + RepartitionExec: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=20516 output_bytes=10258 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=10258 output_bytes=10258 ") } @@ -602,20 +616,19 @@ mod tests { "#; let annotated = sql_to_annotated(query).await; assert_snapshot!(annotated, @r" - ProjectionExec: task_count=Desired(3) - BoundedWindowAggExec: task_count=Desired(3) - SortExec: task_count=Desired(3) - [NetworkBoundary] Shuffle: task_count=Desired(3) - RepartitionExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) + ProjectionExec: task_count=Some(4) output_rows=366 compute_complexity=O(1) accumulated_cost=107047 output_bytes=5856 + BoundedWindowAggExec: task_count=Some(4) output_rows=366 compute_complexity=O((Cols+Col1)) accumulated_cost=107046 output_bytes=10624 + SortExec: task_count=Some(4) output_rows=366 compute_complexity=O((1+1)*Log((1+1))) accumulated_cost=95314 output_bytes=7330 + [NetworkBoundary] Shuffle: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=7330 output_bytes=7330 + RepartitionExec: task_count=Some(4) output_rows=366 compute_complexity=O((Cols+1)) accumulated_cost=14661 output_bytes=7330 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 ") } #[tokio::test] async fn test_children_isolator_union() { let query = r#" - SET distributed.children_isolator_unions = true; - SET distributed.files_per_task = 1; + SELECT "MinTemp" FROM weather WHERE "RainToday" = 'yes' UNION ALL SELECT "MaxTemp" FROM weather WHERE "RainToday" = 'no' @@ -624,59 +637,18 @@ mod tests { "#; let annotated = sql_to_annotated(query).await; assert_snapshot!(annotated, @r" - ChildrenIsolatorUnionExec: task_count=Desired(4) - FilterExec: task_count=Maximum(1) - RepartitionExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) - ProjectionExec: task_count=Maximum(1) - FilterExec: task_count=Maximum(1) - RepartitionExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) - ProjectionExec: task_count=Maximum(2) - FilterExec: task_count=Maximum(2) - RepartitionExec: task_count=Maximum(2) - DataSourceExec: task_count=Maximum(2) - ") - } - - #[tokio::test] - async fn test_intermediate_task_estimator() { - let query = r#" - SELECT DISTINCT "RainToday" FROM weather - "#; - let annotated = sql_to_annotated_with_estimator(query, |_: &RepartitionExec| { - Some(TaskEstimation::maximum(1)) - }) - .await; - assert_snapshot!(annotated, @r" - AggregateExec: task_count=Desired(1) - [NetworkBoundary] Shuffle: task_count=Desired(1) - RepartitionExec: task_count=Maximum(1) - AggregateExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) - ") - } - - #[tokio::test] - async fn test_union_all_limited_by_intermediate_estimator() { - let query = r#" - SELECT "MinTemp" FROM weather WHERE "RainToday" = 'yes' - UNION ALL - SELECT "MaxTemp" FROM weather WHERE "RainToday" = 'no' - "#; - let annotated = sql_to_annotated_with_estimator(query, |_: &FilterExec| { - Some(TaskEstimation::maximum(1)) - }) - .await; - assert_snapshot!(annotated, @r" - ChildrenIsolatorUnionExec: task_count=Desired(2) - FilterExec: task_count=Maximum(1) - RepartitionExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) - ProjectionExec: task_count=Maximum(1) - FilterExec: task_count=Maximum(1) - RepartitionExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) + ChildrenIsolatorUnionExec: task_count=Some(4) output_rows=222 compute_complexity=O(1) accumulated_cost=61562 output_bytes=2650 + FilterExec: task_count=Some(1) output_rows=74 compute_complexity=O((out_Cols+Col1)) accumulated_cost=20528 output_bytes=1466 + RepartitionExec: task_count=Some(1) output_rows=366 compute_complexity=O(Cols) accumulated_cost=14660 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + ProjectionExec: task_count=Some(1) output_rows=74 compute_complexity=O(1) accumulated_cost=20529 output_bytes=592 + FilterExec: task_count=Some(1) output_rows=74 compute_complexity=O((out_Cols+Col1)) accumulated_cost=20528 output_bytes=1466 + RepartitionExec: task_count=Some(1) output_rows=366 compute_complexity=O(Cols) accumulated_cost=14660 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + ProjectionExec: task_count=Some(2) output_rows=74 compute_complexity=O(1) accumulated_cost=20504 output_bytes=592 + FilterExec: task_count=Some(2) output_rows=74 compute_complexity=O((out_Cols+Col1)) accumulated_cost=20503 output_bytes=1465 + RepartitionExec: task_count=Some(2) output_rows=366 compute_complexity=O(Cols) accumulated_cost=14644 output_bytes=7322 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7322 output_bytes=7322 ") } @@ -689,12 +661,12 @@ mod tests { "#; let annotated = sql_to_annotated_broadcast(query, 4, 4, true).await; assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) + HashJoinExec: task_count=Some(4) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(4) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + [NetworkBoundary] Broadcast: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=7330 output_bytes=7330 + BroadcastExec: task_count=Some(2) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 ") } @@ -719,12 +691,12 @@ mod tests { let annotated = sql_to_annotated_broadcast(query, 1, 4, true).await; assert!(annotated.contains("Broadcast")); assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) + HashJoinExec: task_count=Some(4) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(4) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + [NetworkBoundary] Broadcast: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=7330 output_bytes=7330 + BroadcastExec: task_count=Some(4) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 "); } @@ -735,15 +707,19 @@ mod tests { FROM weather a INNER JOIN weather b ON a."RainToday" = b."RainToday" "#; - let annotated = - sql_to_annotated_broadcast_with_estimator(query, 3, BuildSideOneTaskEstimator).await; + let annotated = sql_to_annotated_broadcast_with_estimator( + query, + 3, + BuildSideOneDistributedPlannerExtension, + ) + .await; assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) - DataSourceExec: task_count=Desired(3) + HashJoinExec: task_count=Some(3) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(3) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + [NetworkBoundary] Broadcast: task_count=Some(3) output_rows=366 compute_complexity=O(Cols) accumulated_cost=7330 output_bytes=7330 + BroadcastExec: task_count=Some(1) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(3) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 "); } @@ -758,12 +734,12 @@ mod tests { sql_to_annotated_broadcast_with_estimator(query, 3, BroadcastBuildCoalesceMaxEstimator) .await; assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - [NetworkBoundary] Broadcast: task_count=Maximum(1) - BroadcastExec: task_count=Desired(1) - DataSourceExec: task_count=Desired(1) - DataSourceExec: task_count=Maximum(1) + HashJoinExec: task_count=Some(1) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(1) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + [NetworkBoundary] Broadcast: task_count=Some(1) output_rows=366 compute_complexity=O(Cols) accumulated_cost=7330 output_bytes=7330 + BroadcastExec: task_count=Some(2) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 "); } @@ -778,10 +754,10 @@ mod tests { // With broadcast disabled, no broadcast annotation should appear assert!(!annotated.contains("Broadcast")); assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) - DataSourceExec: task_count=Maximum(1) + HashJoinExec: task_count=Some(1) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(1) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 ") } @@ -795,17 +771,17 @@ mod tests { "#; let annotated = sql_to_annotated_broadcast(query, 4, 4, true).await; assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Desired(3) - HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) + HashJoinExec: task_count=Some(4) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=54239 output_bytes=8784 + CoalescePartitionsExec: task_count=Some(4) output_rows=366 compute_complexity=O(1) accumulated_cost=10259 output_bytes=10258 + [NetworkBoundary] Broadcast: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=10258 output_bytes=10258 + BroadcastExec: task_count=Some(4) output_rows=366 compute_complexity=O(1) accumulated_cost=45456 output_bytes=10258 + HashJoinExec: task_count=Some(4) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=10258 + CoalescePartitionsExec: task_count=Some(4) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + [NetworkBoundary] Broadcast: task_count=Some(4) output_rows=366 compute_complexity=O(Cols) accumulated_cost=7330 output_bytes=7330 + BroadcastExec: task_count=Some(2) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(4) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 ") } @@ -813,6 +789,7 @@ mod tests { async fn test_broadcast_union_children_isolator_annotation() { let query = r#" SET distributed.children_isolator_unions = true; + SELECT a."MinTemp", b."MaxTemp" FROM weather a INNER JOIN weather b ON a."RainToday" = b."RainToday" @@ -829,79 +806,36 @@ mod tests { // With ChildrenIsolatorUnionExec, each broadcast task_count should be limited to their // context. assert_snapshot!(annotated, @r" - ChildrenIsolatorUnionExec: task_count=Desired(4) - HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - [NetworkBoundary] Broadcast: task_count=Maximum(1) - BroadcastExec: task_count=Desired(1) - DataSourceExec: task_count=Desired(1) - DataSourceExec: task_count=Maximum(1) - HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - [NetworkBoundary] Broadcast: task_count=Maximum(1) - BroadcastExec: task_count=Desired(1) - DataSourceExec: task_count=Desired(1) - DataSourceExec: task_count=Maximum(1) - HashJoinExec: task_count=Maximum(2) - CoalescePartitionsExec: task_count=Maximum(2) - [NetworkBoundary] Broadcast: task_count=Maximum(2) - BroadcastExec: task_count=Desired(2) - DataSourceExec: task_count=Desired(2) - DataSourceExec: task_count=Maximum(2) + ChildrenIsolatorUnionExec: task_count=Some(4) output_rows=1098 compute_complexity=O(1) accumulated_cost=136366 output_bytes=17568 + HashJoinExec: task_count=Some(1) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(1) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + [NetworkBoundary] Broadcast: task_count=Some(1) output_rows=366 compute_complexity=O(Cols) accumulated_cost=7330 output_bytes=7330 + BroadcastExec: task_count=Some(2) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + HashJoinExec: task_count=Some(1) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(1) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + [NetworkBoundary] Broadcast: task_count=Some(1) output_rows=366 compute_complexity=O(Cols) accumulated_cost=7330 output_bytes=7330 + BroadcastExec: task_count=Some(2) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(1) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + HashJoinExec: task_count=Some(2) output_rows=366 compute_complexity=O((right_Col1+right_Col1)) accumulated_cost=45455 output_bytes=5856 + CoalescePartitionsExec: task_count=Some(2) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + [NetworkBoundary] Broadcast: task_count=Some(2) output_rows=366 compute_complexity=O(Cols) accumulated_cost=7330 output_bytes=7330 + BroadcastExec: task_count=Some(2) output_rows=366 compute_complexity=O(1) accumulated_cost=7331 output_bytes=7330 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 + DataSourceExec: task_count=Some(2) output_rows=366 compute_complexity=O(out_Cols) accumulated_cost=7330 output_bytes=7330 "); } - #[allow(clippy::type_complexity)] - struct CallbackEstimator { - f: Arc Option + Send + Sync>, - } - - impl CallbackEstimator { - fn new( - f: impl Fn(&T) -> Option + Send + Sync + 'static, - ) -> Self { - let f = Arc::new(move |plan: &dyn ExecutionPlan| -> Option { - if let Some(plan) = plan.as_any().downcast_ref::() { - f(plan) - } else { - None - } - }); - Self { f } - } - } - - impl TaskEstimator for CallbackEstimator { - fn task_estimation( - &self, - plan: &Arc, - _: &ConfigOptions, - ) -> Option { - (self.f)(plan.as_ref()) - } - - fn scale_up_leaf_node( - &self, - _: &Arc, - _: usize, - _: &ConfigOptions, - ) -> Option> { - None - } - } - #[derive(Debug)] struct BroadcastBuildCoalesceMaxEstimator; - impl TaskEstimator for BroadcastBuildCoalesceMaxEstimator { - fn task_estimation( - &self, - plan: &Arc, - _: &ConfigOptions, - ) -> Option { + impl DistributedPlannerExtension for BroadcastBuildCoalesceMaxEstimator { + fn max_tasks(&self, plan: &Arc, _: &ConfigOptions) -> Option { let coalesce = plan.as_any().downcast_ref::()?; if coalesce.input().as_any().is::() { - Some(TaskEstimation::maximum(1)) + Some(1) } else { None } @@ -935,21 +869,10 @@ mod tests { annotate_test_plan(query, options, |b| b).await } - async fn sql_to_annotated_with_estimator( - query: &str, - estimator: impl Fn(&T) -> Option + Send + Sync + 'static, - ) -> String { - let options = TestPlanOptions::default(); - annotate_test_plan(query, options, |b| { - b.with_distributed_task_estimator(CallbackEstimator::new(estimator)) - }) - .await - } - async fn sql_to_annotated_broadcast_with_estimator( query: &str, num_workers: usize, - estimator: impl TaskEstimator + Send + Sync + 'static, + estimator: impl DistributedPlannerExtension + Send + Sync + 'static, ) -> String { let options = TestPlanOptions { target_partitions: 4, @@ -957,7 +880,7 @@ mod tests { broadcast_enabled: true, }; annotate_test_plan(query, options, |b| { - b.with_distributed_task_estimator(estimator) + b.with_distributed_planner_extension(estimator) }) .await } @@ -973,7 +896,9 @@ mod tests { options.broadcast_enabled, ); let builder = configure(builder); - let (ctx, query) = context_with_query(builder, query).await; + let (mut ctx, query) = context_with_query(builder, query).await; + ctx.set_distributed_bytes_processed_per_partition(1000) + .unwrap(); let df = ctx.sql(&query).await.unwrap(); let mut plan = df.create_physical_plan().await.unwrap(); diff --git a/src/distributed_planner/statistics/compute_per_node.rs b/src/distributed_planner/statistics/compute_per_node.rs new file mode 100644 index 00000000..22b8c5f1 --- /dev/null +++ b/src/distributed_planner/statistics/compute_per_node.rs @@ -0,0 +1,512 @@ +use crate::BroadcastExec; +use crate::execution_plans::ChildrenIsolatorUnionExec; +use datafusion::catalog::memory::DataSourceExec; +use datafusion::common::Statistics; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_plan::aggregates::AggregateExec; +use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion::physical_plan::empty::EmptyExec; +use datafusion::physical_plan::expressions::{Column, Literal}; +use datafusion::physical_plan::filter::FilterExec; +use datafusion::physical_plan::joins::{ + CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, SymmetricHashJoinExec, +}; +use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion::physical_plan::projection::ProjectionExec; +use datafusion::physical_plan::repartition::RepartitionExec; +use datafusion::physical_plan::sorts::sort::SortExec; +use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; +use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use std::fmt::{Debug, Formatter}; +use std::sync::Arc; + +#[derive(Clone)] +pub enum Complexity { + /// Constant complexity + Constant, + /// Linear with a specific column from a specific child. + Linear(LinearComplexity), + /// NLogM + Log(Box, Box), + /// N+M + Plus(Box, Box), + /// N*M + Multiply(Box, Box), +} + +#[derive(Clone)] +pub enum LinearComplexity { + /// Depends on linearly with the input column with the provided index + Column(usize), + /// Depends on linearly with the all the input columns + AllColumns, + /// Depends on linearly with the input column with the provided index from the left child + ColumnFromLeft(usize), + /// Depends on linearly with the all the input columns from the left child + AllColumnsFromLeft, + /// Depends on linearly with the input column with the provided index from the right child + ColumnFromRight(usize), + /// Depends on linearly with the all the input columns from the right child + AllColumnsFromRight, + /// Depends on linearly with the output column with the provided index + OutputColumn(usize), + /// Depends on linearly with the all the output columns + AllOutputColumns, +} + +impl Complexity { + fn log(self, other: Self) -> Self { + if matches!(self, Self::Constant) && matches!(other, Self::Constant) { + Self::Constant + } else { + Self::Log(Box::new(self), Box::new(other)) + } + } + + fn plus(self, other: Self) -> Self { + if matches!(self, Self::Constant) && matches!(other, Self::Constant) { + Self::Constant + } else { + Self::Plus(Box::new(self), Box::new(other)) + } + } + + fn multiply(self, other: Self) -> Self { + if matches!(self, Self::Constant) && matches!(other, Self::Constant) { + Self::Constant + } else { + Self::Multiply(Box::new(self), Box::new(other)) + } + } + + /// Computes the total bytes processed given per-child row counts. + pub(crate) fn cost(&self, output_stat: &Statistics, input_stats: &[&Statistics]) -> usize { + // TODO: do not index directly, that can cause panics. + match self { + Self::Constant => 1, + Self::Linear(linear) => match linear { + LinearComplexity::Column(i) => { + let input_stats = input_stats[0]; + let stats = &input_stats.column_statistics[*i]; + *stats.byte_size.get_value().unwrap_or(&0) + } + LinearComplexity::AllColumns => { + let input_stats = input_stats[0]; + *input_stats.total_byte_size.get_value().unwrap_or(&0) + } + LinearComplexity::ColumnFromLeft(i) => { + let input_stats = input_stats[0]; + let stats = &input_stats.column_statistics[*i]; + + *stats.byte_size.get_value().unwrap_or(&0) + } + LinearComplexity::AllColumnsFromLeft => { + let input_stats = input_stats[0]; + *input_stats.total_byte_size.get_value().unwrap_or(&0) + } + LinearComplexity::ColumnFromRight(i) => { + let input_stats = input_stats[1]; + let stats = &input_stats.column_statistics[*i]; + *stats.byte_size.get_value().unwrap_or(&0) + } + LinearComplexity::AllColumnsFromRight => { + let input_stats = input_stats[1]; + *input_stats.total_byte_size.get_value().unwrap_or(&0) + } + LinearComplexity::OutputColumn(i) => { + let stats = &output_stat.column_statistics[*i]; + *stats.byte_size.get_value().unwrap_or(&0) + } + LinearComplexity::AllOutputColumns => { + *output_stat.total_byte_size.get_value().unwrap_or(&0) + } + }, + Self::Log(n, m) => { + n.cost(output_stat, input_stats) * m.cost(output_stat, input_stats).ilog2() as usize + } + Self::Plus(n, m) => n.cost(output_stat, input_stats) + m.cost(output_stat, input_stats), + Self::Multiply(n, m) => { + n.cost(output_stat, input_stats) * m.cost(output_stat, input_stats) + } + } + } +} + +impl Debug for Complexity { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + fn trim_parenthesis(dbg: &Complexity) -> String { + format!("{dbg:?}") + .trim_start_matches("(") + .trim_end_matches(")") + .to_string() + } + match self { + Self::Constant => write!(f, "1"), + Self::Linear(linear) => match linear { + LinearComplexity::Column(i) => write!(f, "Col{i}"), + LinearComplexity::AllColumns => write!(f, "Cols"), + LinearComplexity::ColumnFromLeft(i) => write!(f, "left_Col{i}"), + LinearComplexity::AllColumnsFromLeft => write!(f, "left_Cols"), + LinearComplexity::ColumnFromRight(i) => write!(f, "right_Col{i}"), + LinearComplexity::AllColumnsFromRight => write!(f, "right_Cols"), + LinearComplexity::OutputColumn(i) => write!(f, "out_Col{i}"), + LinearComplexity::AllOutputColumns => write!(f, "out_Cols"), + }, + Self::Log(n, m) => write!(f, "{n:?}*Log({m:?})"), + Self::Plus(n, m) => { + if matches!(n.as_ref(), &Self::Plus(_, _)) { + write!(f, "({}+{m:?})", trim_parenthesis(m)) + } else { + write!(f, "({n:?}+{m:?})") + } + } + Self::Multiply(n, m) => { + if matches!(n.as_ref(), &Self::Multiply(_, _)) { + write!(f, "({}*{m:?})", trim_parenthesis(m)) + } else { + write!(f, "({n:?}*{m:?})") + } + } + } + } +} + +/// Calculates what's the cost, expressed as a number, per input row for each input children. +/// +/// The Vec return has equal size to `node.children()`, and determines how many each input needs +/// to be processed +pub(crate) fn calculate_compute_complexity(node: &Arc) -> Complexity { + let any = node.as_any(); + + // NestedLoopJoinExec: O(n*m) - evaluates join condition for each pair of rows + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/joins/nested_loop_join.rs + if let Some(node) = any.downcast_ref::() { + // Assume we need to do read all input rows one by one. + let n = Complexity::Linear(LinearComplexity::AllColumnsFromLeft); + let m = Complexity::Linear(LinearComplexity::AllColumnsFromRight); + let c = n.multiply(m); + if let Some(_filter) = node.filter() { + // TODO: how can we take into account he filter here? we don't have column stats for + // that filter because it's an intermediate weird thing. + } + return c; + } + + // CrossJoinExec: O(n*m) - produces Cartesian product of all row pairs + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/joins/cross_join.rs + if let Some(_node) = any.downcast_ref::() { + // Assume we need to do read all input rows one by one. + let n = Complexity::Linear(LinearComplexity::AllColumnsFromLeft); + let m = Complexity::Linear(LinearComplexity::AllColumnsFromRight); + return n.multiply(m); + } + + // SortExec: O(n log n) - uses lexsort_to_indices, may spill to disk + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/sorts/sort.rs + if let Some(node) = any.downcast_ref::() { + // All the rows will need to be copied one by one. + let mut n = Complexity::Linear(LinearComplexity::AllColumns); + // All the sort expressions need to be evaluated on every row. + for expr in node.expr() { + n = n.plus(expression_complexity(&expr.expr)) + } + return n.clone().log(n); + } + + // HashJoinExec: hash table build (O(n)) + probe (O(m)) + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/joins/hash_join/exec.rs + if let Some(join) = any.downcast_ref::() { + // Build side (left): concat_batches copies all data (2x read), plus hash table storage, + // plus hashing left join keys. + let mut c = Complexity::Linear(LinearComplexity::AllColumnsFromLeft) + .plus(Complexity::Linear(LinearComplexity::AllColumnsFromLeft)); + for (left_key, _) in join.on() { + c = c.plus(join_key_complexity(left_key, true)); + } + // Probe side (right): read all columns + hash right join keys + c = c.plus(Complexity::Linear(LinearComplexity::AllColumnsFromRight)); + for (_, right_key) in join.on() { + c = c.plus(join_key_complexity(right_key, false)); + } + // TODO: filter cost (intermediate schema, hard to model with current LinearComplexity) + return c; + } + + // SortMergeJoinExec: merge of sorted streams with comparisons + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/joins/sort_merge_join/exec.rs + // Unlike hash join, sort-merge doesn't buffer all data or build hash tables. It streams + // through both sorted inputs with O(max_group_size) memory, using partial_cmp comparisons + // (no hashing). Per-row cost is just key comparisons + optional filter evaluation. + if let Some(node) = any.downcast_ref::() { + let mut c: Option = None; + // Left side: compare join keys during merge + for (left_key, _) in node.on() { + let key = join_key_complexity(left_key, true); + c = Some(match c { + Some(existing) => existing.plus(key), + None => key, + }); + } + // Right side: compare join keys during merge + for (_, right_key) in node.on() { + let key = join_key_complexity(right_key, false); + c = Some(match c { + Some(existing) => existing.plus(key), + None => key, + }); + } + // TODO: filter cost + return c.unwrap_or(Complexity::Constant); + } + + // SymmetricHashJoinExec: streaming join with hash tables on both sides + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/joins/symmetric_hash_join.rs + // More expensive than HashJoinExec: both sides maintain hash tables, concat_batches + // runs on every incoming batch (not once at end), plus pruning interval computation + // and HashSet tracking for visited rows. + if let Some(node) = any.downcast_ref::() { + // Both sides: concat_batches on every batch (2x read) + hash table + hash keys + let mut c = Complexity::Linear(LinearComplexity::AllColumnsFromLeft) + .plus(Complexity::Linear(LinearComplexity::AllColumnsFromLeft)); + for (left_key, _) in node.on() { + c = c.plus(join_key_complexity(left_key, true)); + } + c = c + .plus(Complexity::Linear(LinearComplexity::AllColumnsFromRight)) + .plus(Complexity::Linear(LinearComplexity::AllColumnsFromRight)); + for (_, right_key) in node.on() { + c = c.plus(join_key_complexity(right_key, false)); + } + // TODO: filter cost + return c; + } + + // Aggregation: hash group-by keys + accumulate aggregate inputs + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/aggregates/mod.rs + if let Some(agg) = any.downcast_ref::() { + // Base: read all input columns for accumulation + let mut c = Complexity::Linear(LinearComplexity::AllColumns); + // Additional: evaluate and hash group-by key expressions + for (expr, _) in agg.group_expr().expr() { + c = c.plus(key_expression_complexity(expr)); + } + // Per-aggregate filter expressions (e.g. COUNT(*) FILTER (WHERE ...)) + for filter in agg.filter_expr().iter().flatten() { + c = c.plus(expression_complexity(filter)); + } + return c; + } + + // Window functions: buffer partitions, compute aggregates over windows + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/windows/window_agg_exec.rs + if let Some(node) = any.downcast_ref::() { + // Read all input data + evaluate/hash partition key expressions + let mut c = Complexity::Linear(LinearComplexity::AllColumns); + for expr in node.partition_keys() { + c = c.plus(key_expression_complexity(&expr)); + } + return c; + } + + if let Some(node) = any.downcast_ref::() { + let mut c = Complexity::Linear(LinearComplexity::AllColumns); + for expr in node.partition_keys() { + c = c.plus(key_expression_complexity(&expr)); + } + return c; + } + + // SortPreservingMergeExec: merges pre-sorted streams with comparisons + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs + // K-way merge: O(N log K) comparisons on sort key expressions + if let Some(node) = any.downcast_ref::() { + // need to copy all rows... + let mut n = Complexity::Linear(LinearComplexity::AllColumns); + // and evaluate the sort expressions on all of them + for expr in node.expr() { + n = n.plus(expression_complexity(&expr.expr)) + } + return n; + } + + // FilterExec: evaluates predicate expression per row + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/filter.rs + // Cost depends on predicate complexity - LIKE/Regex operations are expensive + if let Some(node) = any.downcast_ref::() { + // It needs to perform a copy operation just to the output rows... + let n = Complexity::Linear(LinearComplexity::AllOutputColumns); + // ...and predicate evaluation on all input rows. + return n.plus(expression_complexity(node.predicate())); + } + + // ProjectionExec: cost depends on whether it's simple columns or expressions + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/projection.rs + if let Some(node) = any.downcast_ref::() { + let mut n: Option = None; + for expr in node.expr() { + n = if let Some(n) = n { + Some(n.plus(expression_complexity(&expr.expr))) + } else { + Some(expression_complexity(&expr.expr)) + }; + } + return n.unwrap_or(Complexity::Constant); + } + + // RepartitionExec with Hash: computes hash per row + take_arrays + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/repartition/mod.rs + if let Some(node) = any.downcast_ref::() { + // It needs to copy all the data for chunking it to the different output partitions... + let mut n = Complexity::Linear(LinearComplexity::AllColumns); + // And it might need to compute a hash per row based on the provided expressions. + match node.partitioning() { + Partitioning::Hash(expressions, _) => { + for expr in expressions { + n = n.plus(expression_complexity(expr)) + } + } + Partitioning::RoundRobinBatch(_) => {} + Partitioning::UnknownPartitioning(_) => {} + }; + return n; + } + + // DataSourceExec: Produces data, so assume that it's an O(N) operation over all the columns. + if any.is::() { + return Complexity::Linear(LinearComplexity::AllOutputColumns); + } + + // CoalesceBatchesExec: concatenates batches via concat_batches (memory copy). As it copies + // data, it does an O(N) operation over all the input bytes. + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/coalesce_batches.rs + if any.is::() { + return Complexity::Linear(LinearComplexity::AllColumns); + } + + // Limit: just counts rows and stops early. + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/limit.rs + if any.is::() || any.is::() { + return Complexity::Constant; + } + + // CoalescePartitionsExec: receives batches from partitions, just passes through the record + // batches in a zero copy manner. + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/coalesce_partitions.rs + if any.is::() { + return Complexity::Constant; + } + + // BroadcastExec: This node does not do any computation, does not even read the data. + if any.is::() { + return Complexity::Constant; + } + + // UnionExec: combines multiple input streams, no processing + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/union.rs + if any.is::() || any.is::() { + return Complexity::Constant; + } + + // InterleaveExec: round-robin merging of inputs, no processing + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/union.rs + if any.is::() { + return Complexity::Constant; + } + + // EmptyExec: produces no data + // https://github.com/apache/datafusion/blob/branch-52/datafusion/physical-plan/src/empty.rs + if any.is::() { + return Complexity::Constant; + } + + // For unknown node types, assume we have to do an O(N) operation over all the rows. + Complexity::Linear(LinearComplexity::AllOutputColumns) +} + +struct BytesPerRow { + processed: Option, + cols_read: Vec, +} + +fn expression_complexity(expression: &Arc) -> Complexity { + _expression_complexity(expression) + .processed + .unwrap_or(Complexity::Constant) +} + +/// Computes the complexity of processing a join key expression, including the cost of +/// reading the leaf columns from the appropriate child (left or right). +/// Unlike `expression_complexity`, this accounts for the cost of hashing/comparing +/// simple column references (which have zero evaluation cost but real I/O cost). +fn join_key_complexity(expression: &Arc, from_left: bool) -> Complexity { + let bpr = _expression_complexity(expression); + let mut result: Option = None; + for col_idx in &bpr.cols_read { + let linear = if from_left { + LinearComplexity::ColumnFromLeft(*col_idx) + } else { + LinearComplexity::ColumnFromRight(*col_idx) + }; + result = Some(match result { + Some(r) => r.plus(Complexity::Linear(linear)), + None => Complexity::Linear(linear), + }); + } + result.unwrap_or(Complexity::Constant) +} + +/// Like `expression_complexity`, but includes the cost of reading leaf columns. +/// Use for operations that hash/compare expression results (aggregation group-by keys, +/// window partition keys) where even simple column references have processing cost. +fn key_expression_complexity(expression: &Arc) -> Complexity { + let bpr = _expression_complexity(expression); + let mut result: Option = None; + for col_idx in &bpr.cols_read { + result = Some(match result { + Some(r) => r.plus(Complexity::Linear(LinearComplexity::Column(*col_idx))), + None => Complexity::Linear(LinearComplexity::Column(*col_idx)), + }); + } + result.unwrap_or(Complexity::Constant) +} + +fn _expression_complexity(expression: &Arc) -> BytesPerRow { + let any = expression.as_any(); + + if let Some(col) = any.downcast_ref::() { + BytesPerRow { + processed: None, + cols_read: vec![col.index()], + } + } else if any.is::() { + BytesPerRow { + processed: None, + cols_read: vec![], + } + } else { + // Generic handler for all other expressions: CastExpr, TryCastExpr, CaseExpr, + // InListExpr, IsNullExpr, IsNotNullExpr, NotExpr, NegativeExpr, LikeExpr, + // ScalarFunctionExpr, AsyncFuncExpr, etc. + let mut bytes_per_row = BytesPerRow { + processed: None, + cols_read: vec![], + }; + // TODO: I'm not sure if this is right + for child in expression.children() { + let c = _expression_complexity(child); + bytes_per_row.cols_read.extend(&c.cols_read); + // Linear processing per column: + for col_read in &c.cols_read { + bytes_per_row.processed = if let Some(processed) = bytes_per_row.processed { + Some(processed.plus(Complexity::Linear(LinearComplexity::Column(*col_read)))) + } else { + Some(Complexity::Linear(LinearComplexity::Column(*col_read))) + } + } + } + bytes_per_row + } +} diff --git a/src/distributed_planner/statistics/default_bytes_for_datatype.rs b/src/distributed_planner/statistics/default_bytes_for_datatype.rs new file mode 100644 index 00000000..53f1a865 --- /dev/null +++ b/src/distributed_planner/statistics/default_bytes_for_datatype.rs @@ -0,0 +1,137 @@ +use datafusion::arrow::datatypes::{DataType, IntervalUnit}; + +/// Default data size estimate for variable-width columns when no statistics are available. +/// +/// Reference: Trino's PlanNodeStatsEstimate.java:40 +/// https://github.com/trinodb/trino/blob/458/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsEstimate.java#L40 +const DEFAULT_DATA_SIZE_PER_COLUMN: usize = 50; + +/// This function returns the amount of bytes each row is estimated to occupy. +/// +/// The estimation follows Trino's approach for calculating output size per row: +/// - For fixed-width (primitive) types: uses the type's fixed byte width +/// - For variable-width types: uses a default estimate plus offset overhead +/// - Accounts for validity bitmap overhead (1 bit per value, rounded to 1 byte per row) +/// +/// DataFusion has `Statistics::calculate_total_byte_size()` which uses `DataType::primitive_width()`, +/// but it returns `Precision::Absent` (unknown) when encountering any non-primitive type: +/// https://github.com/apache/datafusion/blob/branch-52/datafusion/common/src/stats.rs#L326-L347 +/// +/// For distributed query planning, we need estimates even for variable-width types to make +/// cost-based decisions about data shuffling and task count assignation. This implementation +/// provides estimates for all types following Trino's cost model. +/// +/// Reference: Trino's PlanNodeStatsEstimate.getOutputSizeForSymbol() +/// https://github.com/trinodb/trino/blob/458/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsEstimate.java#L89-L114 +pub(crate) fn default_bytes_for_datatype(data_type: &DataType) -> usize { + // 1 byte for validity bitmap per row (Arrow uses 1 bit, but we round up for estimation). + // Trino calls this the "is null" boolean array. + // Reference: PlanNodeStatsEstimate.java:98-99 + // https://github.com/trinodb/trino/blob/458/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsEstimate.java#L98-L99 + const VALIDITY_OVERHEAD: usize = 1; + + // Handle non-primitive types. + // NOTE: The cases below are Arrow-specific adaptations. Trino only distinguishes between + // FixedWidthType and variable-width types, using Integer.BYTES (4) for offsets. + // Reference: PlanNodeStatsEstimate.java:108-109 + // https://github.com/trinodb/trino/blob/458/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsEstimate.java#L108-L109 + match data_type { + // Primitive types from data_type.primitive_width() + DataType::Int8 => VALIDITY_OVERHEAD + 1, + DataType::Int16 => VALIDITY_OVERHEAD + 2, + DataType::Int32 => VALIDITY_OVERHEAD + 4, + DataType::Int64 => VALIDITY_OVERHEAD + 8, + DataType::UInt8 => VALIDITY_OVERHEAD + 1, + DataType::UInt16 => VALIDITY_OVERHEAD + 2, + DataType::UInt32 => VALIDITY_OVERHEAD + 4, + DataType::UInt64 => VALIDITY_OVERHEAD + 8, + DataType::Float16 => VALIDITY_OVERHEAD + 2, + DataType::Float32 => VALIDITY_OVERHEAD + 4, + DataType::Float64 => VALIDITY_OVERHEAD + 8, + DataType::Timestamp(_, _) => VALIDITY_OVERHEAD + 8, + DataType::Date32 => VALIDITY_OVERHEAD + 4, + DataType::Date64 => VALIDITY_OVERHEAD + 8, + DataType::Time32(_) => VALIDITY_OVERHEAD + 4, + DataType::Time64(_) => VALIDITY_OVERHEAD + 8, + DataType::Duration(_) => VALIDITY_OVERHEAD + 8, + DataType::Interval(IntervalUnit::YearMonth) => VALIDITY_OVERHEAD + 4, + DataType::Interval(IntervalUnit::DayTime) => VALIDITY_OVERHEAD + 8, + DataType::Interval(IntervalUnit::MonthDayNano) => VALIDITY_OVERHEAD + 16, + DataType::Decimal32(_, _) => VALIDITY_OVERHEAD + 4, + DataType::Decimal64(_, _) => VALIDITY_OVERHEAD + 8, + DataType::Decimal128(_, _) => VALIDITY_OVERHEAD + 16, + DataType::Decimal256(_, _) => VALIDITY_OVERHEAD + 32, + // Null type has no data (Arrow-specific) + DataType::Null => 0, + + // Boolean is stored as bits (1/8 byte per value), but we round up (Arrow-specific) + DataType::Boolean => VALIDITY_OVERHEAD + 1, + + // Fixed-size binary: just the fixed size + validity (Arrow-specific) + DataType::FixedSizeBinary(size) => VALIDITY_OVERHEAD + (*size as usize), + + // Fixed-size list: fixed count * element size (Arrow-specific) + DataType::FixedSizeList(field, size) => { + VALIDITY_OVERHEAD + (*size as usize) * default_bytes_for_datatype(field.data_type()) + } + + // Struct: sum of all child field sizes (Arrow-specific) + // Trino would treat ROW types as variable-width + DataType::Struct(fields) => fields + .iter() + .map(|f| default_bytes_for_datatype(f.data_type())) + .sum(), + + // Dictionary-encoded: just the key indices, values are shared across rows (Arrow-specific) + // Trino doesn't have dictionary encoding at the type level + DataType::Dictionary(key_type, _value_type) => default_bytes_for_datatype(key_type), + + // Union: type_id (1 byte) + max child size (Arrow-specific) + DataType::Union(fields, _) => { + let max_child_size = fields + .iter() + .map(|(_, f)| default_bytes_for_datatype(f.data_type())) + .max() + .unwrap_or(0); + 1 + max_child_size + } + + // Run-end encoded: estimate as if it were the value type (Arrow-specific) + // Actual compression depends on data distribution + DataType::RunEndEncoded(_, values) => default_bytes_for_datatype(values.data_type()), + + // Variable-width string/binary types. + // Offset size follows Trino's Integer.BYTES (4 bytes). + // Reference: PlanNodeStatsEstimate.java:109 + DataType::Utf8 | DataType::Binary => { + VALIDITY_OVERHEAD + size_of::() + DEFAULT_DATA_SIZE_PER_COLUMN + } + // Large variants use i64 offsets (Arrow-specific, Trino doesn't have large variants) + DataType::LargeUtf8 | DataType::LargeBinary => { + VALIDITY_OVERHEAD + size_of::() + DEFAULT_DATA_SIZE_PER_COLUMN + } + // View types use 16-byte inline representation (Arrow-specific) + // Reference: https://arrow.apache.org/docs/format/Columnar.html#variable-size-binary-view-layout + DataType::Utf8View | DataType::BinaryView => VALIDITY_OVERHEAD + 16, + + // List types (Arrow-specific adaptation) + // Spark assumes 1 element average for collections (SPARK-18853). Trino treats them + // as flat variable-width with 50-byte default. We follow Spark's 1-element assumption + // to avoid massive overestimation (e.g. Map was 605 bytes with 10 elements). + DataType::List(field) => { + VALIDITY_OVERHEAD + size_of::() + default_bytes_for_datatype(field.data_type()) + } + DataType::LargeList(field) => { + VALIDITY_OVERHEAD + size_of::() + default_bytes_for_datatype(field.data_type()) + } + DataType::ListView(field) | DataType::LargeListView(field) => { + VALIDITY_OVERHEAD + 8 + default_bytes_for_datatype(field.data_type()) + } + + // Map type: stored as List> (Arrow-specific) + // Uses same 1-element assumption as List types (following Spark). + DataType::Map(field, _) => { + VALIDITY_OVERHEAD + size_of::() + default_bytes_for_datatype(field.data_type()) + } // Fallback for any other types - use Trino's default + } +} diff --git a/src/distributed_planner/statistics/mod.rs b/src/distributed_planner/statistics/mod.rs new file mode 100644 index 00000000..2c4d2e84 --- /dev/null +++ b/src/distributed_planner/statistics/mod.rs @@ -0,0 +1,7 @@ +mod compute_per_node; +mod default_bytes_for_datatype; +mod plan_statistics; + +pub(crate) use compute_per_node::calculate_compute_complexity; +pub(crate) use compute_per_node::{Complexity, LinearComplexity}; +pub use plan_statistics::plan_statistics; diff --git a/src/distributed_planner/statistics/plan_statistics.rs b/src/distributed_planner/statistics/plan_statistics.rs new file mode 100644 index 00000000..124059bc --- /dev/null +++ b/src/distributed_planner/statistics/plan_statistics.rs @@ -0,0 +1,172 @@ +use crate::DistributedConfig; +use crate::distributed_planner::statistics::default_bytes_for_datatype::default_bytes_for_datatype; +use datafusion::common::stats::Precision; +use datafusion::common::{Statistics, not_impl_err, plan_err}; +use datafusion::config::ConfigOptions; +use datafusion::error::Result; +use datafusion::execution::{SendableRecordBatchStream, TaskContext}; +use datafusion::physical_plan::execution_plan::CardinalityEffect; +use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; +use delegate::delegate; +use itertools::Itertools; +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; + +/// Options during the plan statistics calculation +#[derive(Debug, Default, Clone)] +pub struct StatisticsOptions { + /// In case a plan lacks a row count, this default number of rows will be used. + pub default_estimated_row_count: Option, +} + +impl From<&DistributedConfig> for StatisticsOptions { + fn from(value: &DistributedConfig) -> Self { + Self { + default_estimated_row_count: value.default_estimated_row_count, + } + } +} + +/// Uses upstream DataFusion stats system with some small overrides. +pub fn plan_statistics( + node: &Arc, + children_stats: &[&Statistics], + opts: StatisticsOptions, +) -> Result { + let mut stats = partition_statistics_with_children_override(node, None, children_stats)?; + + // If rows are absent, be conservative and assume that all the rows from all the children + // are going to be returned. + if matches!(stats.num_rows, Precision::Absent) { + let num_rows = children_stats + .iter() + .flat_map(|v| v.num_rows.get_value()) + .sum1::(); + let num_rows = if let Some(num_rows) = num_rows { + num_rows + } else if let Some(default) = opts.default_estimated_row_count { + default + } else { + return plan_err!( + "{} does not provide row stats, and none of its children [{}] provides a row count", + node.name(), + node.children() + .iter() + .map(|v| v.name()) + .collect::>() + .join(", ") + ); + }; + stats.num_rows = Precision::Inexact(num_rows) + } + + let schema = node.schema(); + + for (i, col_stats) in &mut stats.column_statistics.iter_mut().enumerate() { + let rows = stats.num_rows.get_value().unwrap_or(&0); + + // If some of the NDVs are not present in one of the column-level stats, assume the + // worst and use the same as the input number of rows. + if matches!(col_stats.distinct_count, Precision::Absent) { + col_stats.distinct_count = Precision::Inexact(*rows); + } + + // If the per-column byte size stats are not present, estimate the byte size based on the + // data type and the row count. + let Some(dt) = schema.fields.get(i).map(|v| v.data_type()) else { + return plan_err!("Field with index {i} not present in schema: {schema:?}"); + }; + // FIXME: byte size calculation upstream is really crappy. Ideally, this override + // should only happen if byte_size is Absent. + col_stats.byte_size = Precision::Inexact(default_bytes_for_datatype(dt) * rows) + } + + // If bytes are absent, let's just infer them based on the schema and the + // number of rows. + if matches!(stats.total_byte_size, Precision::Absent) { + let mut total_byte_size = 0; + for col_stats in &stats.column_statistics { + total_byte_size += col_stats.byte_size.get_value().unwrap_or(&0); + } + stats.total_byte_size = Precision::Inexact(total_byte_size); + } + + Ok(stats) +} + +// FIXME: because of limitations the the statistics API on DataFusion, we need to resource to +// this sketchy way of overriding child statistics, as we cannot just provide our own. +// If we don't do this: +// 1. we cannot tell nodes to compute statistics based on the ones we provide. +// 2. we recompute statistics unnecessarily across the plan +// This is tracked by https://github.com/apache/datafusion/issues/20184 upstream, and until +// that one is solved, we need to resource to this wrapper. +fn partition_statistics_with_children_override( + node: &Arc, + partition: Option, + child_stats: &[&Statistics], +) -> Result { + // DataFusion stats system is not very mature yet. This override layer brings in changes + // that might not have already been released or informed overrides. + let statistics_wrapped_children = child_stats + .iter() + .zip(node.children()) + .map(|(&stats, child)| StatisticsWrapper { + inner: Arc::clone(child), + stats: stats.clone(), + }) + .map(|v| Arc::new(v) as _) + .collect(); + + Arc::clone(node) + .with_new_children(statistics_wrapped_children)? + .partition_statistics(partition) +} + +#[derive(Debug)] +struct StatisticsWrapper { + stats: Statistics, + inner: Arc, +} + +impl DisplayAs for StatisticsWrapper { + delegate! { + to self.inner { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result; + } + } +} + +impl ExecutionPlan for StatisticsWrapper { + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return plan_err!("StatisticsWrapper not prepared for partition-specific stats"); + } + Ok(self.stats.clone()) + } + + delegate! { + to self.inner { + fn name(&self) -> &str; + fn as_any(&self) -> &dyn Any; + fn properties(&self) -> &PlanProperties; + fn maintains_input_order(&self) -> Vec; + fn benefits_from_input_partitioning(&self) -> Vec; + fn children(&self) -> Vec<&Arc>; + fn repartitioned(&self, _target_partitions: usize, _config: &ConfigOptions) -> Result>>; + fn execute(&self, partition: usize, context: Arc) -> Result; + fn supports_limit_pushdown(&self) -> bool; + fn with_fetch(&self, _limit: Option) -> Option>; + fn fetch(&self) -> Option; + fn cardinality_effect(&self) -> CardinalityEffect; + } + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + not_impl_err!("with_new_children not implemented") + } +} diff --git a/src/distributed_planner/task_estimator.rs b/src/distributed_planner/task_estimator.rs deleted file mode 100644 index ae22ae0a..00000000 --- a/src/distributed_planner/task_estimator.rs +++ /dev/null @@ -1,404 +0,0 @@ -use crate::config_extension_ext::set_distributed_option_extension; -use crate::{DistributedConfig, PartitionIsolatorExec}; -use datafusion::catalog::memory::DataSourceExec; -use datafusion::config::ConfigOptions; -use datafusion::datasource::physical_plan::FileScanConfig; -use datafusion::physical_plan::ExecutionPlan; -use datafusion::prelude::SessionConfig; -use delegate::delegate; -use std::fmt::Debug; -use std::sync::Arc; - -/// Annotation attached to a single [ExecutionPlan] that determines how many distributed tasks -/// it should run on. -#[derive(Debug, Clone)] -pub enum TaskCountAnnotation { - /// The desired number of distributed tasks for this node. The final task count for the - /// annotated node might not be exactly this number, it is more like a hint, so depending - /// on the desired task count of adjacent nodes, the final task count might change. - Desired(usize), - /// Sets a maximum number of distributed tasks for this node. Typically used with the inner - /// value of 1, stating that this node cannot be executed in a distributed fashion. - Maximum(usize), -} - -impl From for usize { - fn from(annotation: TaskCountAnnotation) -> Self { - annotation.as_usize() - } -} - -impl TaskCountAnnotation { - pub fn as_usize(&self) -> usize { - match self { - Self::Desired(desired) => *desired, - Self::Maximum(maximum) => *maximum, - } - } - - pub(crate) fn limit(self, limit: usize) -> Self { - match self { - Self::Desired(desired) => Self::Desired(desired.min(limit)), - Self::Maximum(maximum) => Self::Maximum(maximum.min(limit)), - } - } -} - -/// Result of running a [TaskEstimator] on a leaf node. It tells the distributed planner hints -/// about how many tasks should be used in [Stage]s that contain leaf nodes. -pub struct TaskEstimation { - /// The number of tasks that should be used in the [Stage] containing the leaf node. - /// - /// Even if implementations get to decide this number, there are situations where it can - /// get overridden: - /// - If a [Stage] contains multiple leaf nodes, the one that declares the biggest - /// task_count wins. - /// - If there are less available workers than this number, the number of available workers - /// is chosen. - pub task_count: TaskCountAnnotation, -} - -impl TaskEstimation { - /// Tells the distributed planner that the evaluated stage can have **at maximum** the provided - /// number of tasks, setting a hard upper limit. - /// - /// Returning `TaskEstimation::maximum(1)` tells the distributed planner that the evaluated - /// stage cannot be distributed. - /// - /// Even if a `TaskEstimation::maximum(N)` is provided, any other node in the same stage - /// providing a value of `TaskEstimation::maximum(M)` where `M` < `N` will have preference. - pub fn maximum(value: usize) -> Self { - TaskEstimation { - task_count: TaskCountAnnotation::Maximum(value), - } - } - - /// Tells the distributed planner that the evaluated can **optimally** have the provided - /// number of tasks, setting a soft task count hint that can be overridden by others. - /// - /// The provided `TaskEstimation::desired(N)` can be overridden by: - /// - Other nodes providing a `TaskEstimation::desired(M)` where `M` > `N`. - /// - Any other node providing a `TaskEstimation::maximum(M)` where `M` can be anything. - pub fn desired(value: usize) -> Self { - TaskEstimation { - task_count: TaskCountAnnotation::Desired(value), - } - } -} - -/// Given a leaf node, provides an estimation about how many tasks should be used in the -/// stage containing it, and if the leaf node should be replaced by some other. -/// -/// The distributed planner will try many [TaskEstimator]s in order until one provides an -/// estimation for a specific leaf node. Once that's done, upper stages will get their task -/// count calculated based on whether lower stages are reducing the cardinality of the data -/// or increasing it. -pub trait TaskEstimator { - /// Function applied to each node that returns a [TaskEstimation] hinting how many - /// tasks should be used in the [Stage] containing that node. - /// - /// All the [TaskEstimator] registered in the session will be applied to the node - /// until one returns an estimation. - /// - /// - /// If no estimation is returned from any of the registered [TaskEstimator]s, then: - /// - If the node is a leaf node,`Maximum(1)` is assumed, hinting the distributed planner - /// that the leaf node cannot be distributed across tasks. - /// - If the node is a normal node in the plan, then the maximum task count from its children - /// is inherited. - fn task_estimation( - &self, - plan: &Arc, - cfg: &ConfigOptions, - ) -> Option; - - /// After a final task_count is decided, taking into account all the leaf nodes in the [Stage], - /// this allows performing a transformation in the leaf nodes for accounting for the fact that - /// they are going to run in multiple tasks. - fn scale_up_leaf_node( - &self, - plan: &Arc, - task_count: usize, - cfg: &ConfigOptions, - ) -> Option>; -} - -impl TaskEstimator for usize { - fn task_estimation( - &self, - inputs: &Arc, - _: &ConfigOptions, - ) -> Option { - if inputs.children().is_empty() { - Some(TaskEstimation { - task_count: TaskCountAnnotation::Desired(*self), - }) - } else { - None - } - } - - fn scale_up_leaf_node( - &self, - _: &Arc, - _: usize, - _: &ConfigOptions, - ) -> Option> { - None - } -} - -impl TaskEstimator for Arc { - delegate! { - to self.as_ref() { - fn task_estimation(&self, plan: &Arc, cfg: &ConfigOptions) -> Option; - fn scale_up_leaf_node(&self, plan: &Arc, task_count: usize, cfg: &ConfigOptions) -> Option>; - } - } -} - -impl TaskEstimator for Arc { - delegate! { - to self.as_ref() { - fn task_estimation(&self, plan: &Arc, cfg: &ConfigOptions) -> Option; - fn scale_up_leaf_node(&self, plan: &Arc, task_count: usize, cfg: &ConfigOptions) -> Option>; - } - } -} - -pub(crate) fn set_distributed_task_estimator( - cfg: &mut SessionConfig, - estimator: impl TaskEstimator + Send + Sync + 'static, -) { - let opts = cfg.options_mut(); - if let Some(distributed_cfg) = opts.extensions.get_mut::() { - distributed_cfg - .__private_task_estimator - .user_provided - .push(Arc::new(estimator)); - } else { - let mut estimators = CombinedTaskEstimator::default(); - estimators.user_provided.push(Arc::new(estimator)); - set_distributed_option_extension( - cfg, - DistributedConfig { - __private_task_estimator: estimators, - ..Default::default() - }, - ) - } -} - -/// [TaskEstimator] implementation that acts on [DataSourceExec] nodes that contain -/// [FileScanConfig]s data sources (e.g., Parquet or CSV files). it will read the -/// [DistributedConfig].`files_per_task` field and assigns as many tasks as needed so that -/// no task handles more than the configured files. -#[derive(Debug)] -struct FileScanConfigTaskEstimator; - -impl TaskEstimator for FileScanConfigTaskEstimator { - fn task_estimation( - &self, - plan: &Arc, - cfg: &ConfigOptions, - ) -> Option { - let dse: &DataSourceExec = plan.as_any().downcast_ref()?; - let file_scan: &FileScanConfig = dse.data_source().as_any().downcast_ref()?; - - let d_cfg = cfg.extensions.get::()?; - - // Count how many partitioned files we have in the FileScanConfig. - let mut partitioned_files = 0; - for file_group in &file_scan.file_groups { - partitioned_files += file_group.len(); - } - - // Based on the user-provided files_per_task configuration, do the math to calculate - // how many tasks should be used, without surpassing the number of available workers. - let task_count = partitioned_files.div_ceil(d_cfg.files_per_task); - - Some(TaskEstimation { - task_count: TaskCountAnnotation::Desired(task_count), - }) - } - - fn scale_up_leaf_node( - &self, - plan: &Arc, - task_count: usize, - _cfg: &ConfigOptions, - ) -> Option> { - if task_count == 1 { - return Some(Arc::clone(plan)); - } - // Based on the task count, attempt to scale up the partitions in the DataSourceExec by - // repartitioning it. This will result in a DataSourceExec with potentially a lot of - // partitions, but as we are going to wrap it with PartitionIsolatorExec, that's fine. - let dse: &DataSourceExec = plan.as_any().downcast_ref()?; - let file_scan: &FileScanConfig = dse.data_source().as_any().downcast_ref()?; - - let mut new_file_scan = file_scan.clone(); - new_file_scan.file_groups.clear(); - for file_group in file_scan.file_groups.clone() { - new_file_scan - .file_groups - .extend(file_group.split_files(task_count)); - } - let plan = DataSourceExec::from_data_source(new_file_scan); - Some(Arc::new(PartitionIsolatorExec::new(plan, task_count))) - } -} - -/// Tries multiple user-provided [TaskEstimator]s until one returns an estimation. If none -/// returns an estimation, a set of default [TaskEstimation] implementations is tried. Right -/// now the only default [TaskEstimation] is [FileScanConfigTaskEstimator]. -#[derive(Clone, Default)] -pub(crate) struct CombinedTaskEstimator { - pub(crate) user_provided: Vec>, -} - -impl TaskEstimator for CombinedTaskEstimator { - fn task_estimation( - &self, - plan: &Arc, - cfg: &ConfigOptions, - ) -> Option { - for estimator in &self.user_provided { - if let Some(result) = estimator.task_estimation(plan, cfg) { - return Some(result); - } - } - // We want to execute the default estimators last so that the user-provided ones have - // a chance of providing an estimation. - // If none of the user-provided returned an estimation, the default ones are used. - for default_estimator in [&FileScanConfigTaskEstimator as &dyn TaskEstimator] { - if let Some(result) = default_estimator.task_estimation(plan, cfg) { - return Some(result); - } - } - None - } - - fn scale_up_leaf_node( - &self, - plan: &Arc, - task_count: usize, - cfg: &ConfigOptions, - ) -> Option> { - for estimator in &self.user_provided { - if let Some(result) = estimator.scale_up_leaf_node(plan, task_count, cfg) { - return Some(result); - } - } - // We want to execute the default estimators last so that the user-provided ones have - // a chance of providing an estimation. - // If none of the user-provided returned an estimation, the default ones are used. - for default_estimator in [&FileScanConfigTaskEstimator as &dyn TaskEstimator] { - if let Some(result) = default_estimator.scale_up_leaf_node(plan, task_count, cfg) { - return Some(result); - } - } - None - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::networking::WorkerResolverExtension; - use crate::test_utils::in_memory_channel_resolver::InMemoryWorkerResolver; - use crate::test_utils::parquet::register_parquet_tables; - use datafusion::error::DataFusionError; - use datafusion::prelude::SessionContext; - - #[tokio::test] - async fn test_first_user_estimator_wins() -> Result<(), DataFusionError> { - let mut combined = CombinedTaskEstimator::default(); - combined.push(10); - combined.push(20); - - let node = make_data_source_exec().await?; - assert_eq!(combined.task_count(node, |cfg| cfg), 10); - Ok(()) - } - - #[tokio::test] - async fn test_continues_until_some() -> Result<(), DataFusionError> { - let mut combined = CombinedTaskEstimator::default(); - combined.push(|_: &Arc, _: &ConfigOptions| None); - combined.push(30); - - let node = make_data_source_exec().await?; - assert_eq!(combined.task_count(node, |cfg| cfg), 30); - Ok(()) - } - - #[tokio::test] - async fn test_defaults_to_file_scan_config_task_estimator() -> Result<(), DataFusionError> { - let mut combined = CombinedTaskEstimator::default(); - combined.push(|_: &Arc, _: &ConfigOptions| None); - - let node = make_data_source_exec().await?; - assert_eq!(combined.task_count(node, |cfg| cfg), 3); - Ok(()) - } - - impl CombinedTaskEstimator { - fn push(&mut self, value: impl TaskEstimator + Send + Sync + 'static) { - self.user_provided.push(Arc::new(value)); - } - - fn task_count( - &self, - node: Arc, - f: impl FnOnce(DistributedConfig) -> DistributedConfig, - ) -> usize { - let mut cfg = ConfigOptions::default(); - let d_cfg = DistributedConfig { - files_per_task: 1, - __private_worker_resolver: WorkerResolverExtension(Arc::new( - InMemoryWorkerResolver::new(3), - )), - ..Default::default() - }; - cfg.extensions.insert(f(d_cfg)); - self.task_estimation(&node, &cfg) - .unwrap() - .task_count - .as_usize() - } - } - - async fn make_data_source_exec() -> Result, DataFusionError> { - let ctx = SessionContext::new(); - register_parquet_tables(&ctx).await?; - let mut plan = ctx - .sql("SELECT * FROM weather") - .await? - .create_physical_plan() - .await?; - while !plan.children().is_empty() { - plan = Arc::clone(plan.children()[0]) - } - Ok(plan) - } - - impl, &ConfigOptions) -> Option> TaskEstimator for F { - fn task_estimation( - &self, - plan: &Arc, - cfg: &ConfigOptions, - ) -> Option { - self(plan, cfg) - } - - fn scale_up_leaf_node( - &self, - _plan: &Arc, - _task_count: usize, - _cfg: &ConfigOptions, - ) -> Option> { - None - } - } -} diff --git a/src/execution_plans/broadcast.rs b/src/execution_plans/broadcast.rs index 915a5e70..b8f33dd2 100644 --- a/src/execution_plans/broadcast.rs +++ b/src/execution_plans/broadcast.rs @@ -1,6 +1,7 @@ use crate::common::require_one_child; use crossbeam_queue::SegQueue; use datafusion::arrow::datatypes::SchemaRef; +use datafusion::common::Statistics; use datafusion::common::runtime::SpawnedTask; use datafusion::error::{DataFusionError, Result}; use datafusion::execution::memory_pool::MemoryConsumer; @@ -211,6 +212,11 @@ impl ExecutionPlan for BroadcastExec { fn schema(&self) -> SchemaRef { self.input.schema() } + + fn partition_statistics(&self, partition: Option) -> Result { + let real_partition = partition.map(|partition| partition % self.input_partition_count()); + self.input.partition_statistics(real_partition) + } } #[derive(Debug, Clone, Copy)] diff --git a/src/execution_plans/children_isolator_union.rs b/src/execution_plans/children_isolator_union.rs index 6f0dad1b..51c1ae37 100644 --- a/src/execution_plans/children_isolator_union.rs +++ b/src/execution_plans/children_isolator_union.rs @@ -11,7 +11,6 @@ use datafusion::physical_plan::{ Partitioning, PlanProperties, }; use futures::{Stream, StreamExt}; -use itertools::Itertools; use std::any::Any; use std::fmt::Formatter; use std::pin::Pin; @@ -96,23 +95,16 @@ pub struct ChildrenIsolatorUnionExec { impl ChildrenIsolatorUnionExec { pub(crate) fn from_children_and_task_counts( - children: impl IntoIterator>, - children_task_count: impl IntoIterator, - task_count: usize, + children: Vec>, + task_idx_map: Vec< + /* outer distributed task idx */ + Vec<( + /* child index */ usize, + /* inner distributed task ctx for the isolated child*/ + DistributedTaskContext, + )>, + >, ) -> Result { - let children = children.into_iter().collect_vec(); - let task_count_per_children = children_task_count.into_iter().collect_vec(); - - if children.len() != task_count_per_children.len() { - return internal_err!( - "ChildrenIsolatorUnionExec received {} children but a vec of {} positions for those children. This is a bug in the distributed planning logic, please report it", - children.len(), - task_count_per_children.len() - ); - } - - let task_idx_map = split_children(task_count_per_children, task_count)?; - // Because different children might return a different number of partitions, and we might // execute a different number of children in different tasks, the reality is that this node, // depending on which task index is running, it will have a different number of partitions. @@ -328,214 +320,3 @@ impl Stream for ObservedStream { self.baseline_metrics.record_poll(poll) } } - -/// Given a list of children with a different number of tasks assigned each, it redistributes them -/// and re-assign tasks numbers to them so that they fit in the provided `task_count`. -/// -/// For example, given these inputs: -/// `task_count_per_children`: [1, 1, 1] -/// `task_count`: 3 -/// It returns the following output: -/// `[[(0, 0/1)], [(1, 0/1)], [(2, 0/1)]]` -/// That means that: -/// - Task 0 will execute task 0 from child 0 -/// - Task 1 will execute task 0 from child 1 -/// - Task 2 will execute task 0 from child 2 -/// -/// Things can get more complicated if the sum of all the tasks from the children is greater than -/// the `task_count` passed: -/// -/// For example, given these inputs: -/// `task_count_per_children`: [2, 1, 1] -/// `task_count`: 3 -/// It returns the following output: -/// `[[(0, 0/1)], [(1, 0/1)], [(2, 0/1)]]` -/// As we have 3 tasks available for executing 3 children with a total sum of 2+1+1=4 tasks, we -/// need to tell that first child to be executed in 1 task. -/// -/// In this other case: -/// `task_count_per_children`: [2, 3, 1] -/// `task_count`: 5 -/// It returns the following output: -/// `[[(0, 0/2)], [(0, 1/2)], [(1, 0/2)], [(1, 1/2)], [(2, 0/1)]]` -/// Note how in this case there are 2+3+1=6 tasks to be executed, but we only have 5 tasks -/// available. We need to trim a task from somewhere, and the only candidates are child 0 and 1. -/// As child 1 is the one with the greatest task count (3), we prefer to trim the task from there, -/// as that's what will yield the most even distribution of tasks given the 5 tasks budget. -/// -/// Going to the other extreme, given this input: -/// `task_count_per_children`: [1, 1, 1, 1, 1] -/// `task_count`: 2 -/// It returns the following output: -/// `[[(0, 0/1), (1, 0/1), (2, 0/1)] , [(3, 0/1), (4, 0/1)]]` -/// In this case, we have very few `task_count` available, so we cannot afford to execute one -/// child per task. We need to group children so that one task executes several of them, and the -/// other tasks execute the several other remaining. -/// -/// The function looks pretty much like the solution of a LeetCode problem, so it's not super -/// easy to understand just be looking at the code. The best way to get a grasp of what its doing -/// is by looking at the tests. -fn split_children( - mut task_count_per_children: Vec, - task_count_budget: usize, -) -> Result< - // Task idx. This Vec will have `task_count_budget` length. - Vec< - // For this task, the child indexes and DistributedTaskContext that should be executed. - Vec<( - /* Child index */ usize, - /* Distributed task ctx for the child */ DistributedTaskContext, - )>, - >, - DataFusionError, -> { - let total_children_tasks = task_count_per_children.iter().sum::(); - if task_count_budget > total_children_tasks { - return internal_err!( - "ChildrenIsolatorUnionExec had a task count {task_count_budget}, which is greater than the sum of child task counts {total_children_tasks}. This is a bug in the distributed planning logic, please report it" - ); - } else if task_count_budget == 0 { - return internal_err!( - "ChildrenIsolatorUnionExec had a task count {task_count_budget}. This is a bug in the distributed planning logic, please report it" - ); - } - - let mut tasks_to_trim = total_children_tasks - task_count_budget; - while tasks_to_trim > 0 { - let mut max_child_task_count_idx = 0; - let mut max_child_task_count_value = 1; - for (i, child_task_count) in task_count_per_children.iter().enumerate() { - if child_task_count > &max_child_task_count_value { - max_child_task_count_idx = i; - max_child_task_count_value = *child_task_count; - } - } - if max_child_task_count_value == 1 { - break; - } - task_count_per_children[max_child_task_count_idx] -= 1; - tasks_to_trim -= 1; - } - - let total_child_tasks: usize = task_count_per_children.iter().sum(); - let base_per_task = total_child_tasks / task_count_budget; - let mut extra = total_child_tasks % task_count_budget; - - let mut result = vec![vec![]; task_count_budget]; - let mut task_idx = 0; - let mut current_task_count = 0; - let mut current_task_capacity = base_per_task; - if extra > 0 { - extra -= 1; - current_task_capacity += 1 - } - - for (child_idx, &child_task_count) in task_count_per_children.iter().enumerate() { - for task_i in 0..child_task_count { - result[task_idx].push(( - child_idx, - DistributedTaskContext { - task_index: task_i, - task_count: child_task_count, - }, - )); - current_task_count += 1; - - if current_task_count >= current_task_capacity && task_idx < task_count_budget - 1 { - task_idx += 1; - current_task_count = 0; - current_task_capacity = base_per_task; - if extra > 0 { - extra -= 1; - current_task_capacity += 1 - } - } - } - } - - Ok(result) -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn children_split_all_1_task() -> Result<(), Box> { - assert_eq!( - split_children(vec![1, 1, 1], 3)?, - vec![ - vec![(0, ctx(0, 1))], - vec![(1, ctx(0, 1))], - vec![(2, ctx(0, 1))] - ] - ); - assert_eq!( - split_children(vec![1, 1, 1], 2)?, - vec![vec![(0, ctx(0, 1)), (1, ctx(0, 1))], vec![(2, ctx(0, 1))]] - ); - assert_eq!( - split_children(vec![1, 1, 1], 1)?, - vec![vec![(0, ctx(0, 1)), (1, ctx(0, 1)), (2, ctx(0, 1))]] - ); - Ok(()) - } - - #[test] - fn split_children_different_tasks() -> Result<(), Box> { - assert_eq!( - split_children(vec![1, 2, 3], 6)?, - vec![ - vec![(0, ctx(0, 1))], - vec![(1, ctx(0, 2))], - vec![(1, ctx(1, 2))], - vec![(2, ctx(0, 3))], - vec![(2, ctx(1, 3))], - vec![(2, ctx(2, 3))] - ] - ); - assert_eq!( - split_children(vec![1, 2, 3], 5)?, - vec![ - vec![(0, ctx(0, 1))], - vec![(1, ctx(0, 2))], - vec![(1, ctx(1, 2))], - vec![(2, ctx(0, 2))], - vec![(2, ctx(1, 2))], - ] - ); - assert_eq!( - split_children(vec![1, 2, 3], 4)?, - vec![ - vec![(0, ctx(0, 1))], - vec![(1, ctx(0, 1))], - vec![(2, ctx(0, 2))], - vec![(2, ctx(1, 2))], - ] - ); - assert_eq!( - split_children(vec![1, 2, 3], 3)?, - vec![ - vec![(0, ctx(0, 1))], - vec![(1, ctx(0, 1))], - vec![(2, ctx(0, 1))], - ] - ); - assert_eq!( - split_children(vec![1, 2, 3], 2)?, - vec![vec![(0, ctx(0, 1)), (1, ctx(0, 1))], vec![(2, ctx(0, 1))]] - ); - assert_eq!( - split_children(vec![1, 2, 3], 1)?, - vec![vec![(0, ctx(0, 1)), (1, ctx(0, 1)), (2, ctx(0, 1))]] - ); - Ok(()) - } - - fn ctx(task_index: usize, task_count: usize) -> DistributedTaskContext { - DistributedTaskContext { - task_index, - task_count, - } - } -} diff --git a/src/execution_plans/metrics.rs b/src/execution_plans/metrics.rs index c31a80e5..63b91787 100644 --- a/src/execution_plans/metrics.rs +++ b/src/execution_plans/metrics.rs @@ -1,6 +1,7 @@ use datafusion::physical_plan::metrics::MetricsSet; use std::sync::Arc; +use datafusion::common::Statistics; use datafusion::error::Result; use datafusion::execution::{SendableRecordBatchStream, TaskContext}; use datafusion::physical_plan::ExecutionPlan; @@ -43,6 +44,7 @@ impl ExecutionPlan for MetricsWrapperExec { fn name(&self) -> &str; fn properties(&self) -> &PlanProperties; fn as_any(&self) -> &dyn Any; + fn partition_statistics(&self, partition: Option) -> Result; } } diff --git a/src/lib.rs b/src/lib.rs index b15c743f..afb64a22 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -1,4 +1,5 @@ #![deny(clippy::all)] +extern crate core; mod common; mod config_extension_ext; @@ -19,8 +20,8 @@ pub mod test_utils; pub use arrow_ipc::CompressionType; pub use distributed_ext::DistributedExt; pub use distributed_planner::{ - DistributedConfig, DistributedPhysicalOptimizerRule, NetworkBoundary, NetworkBoundaryExt, - TaskCountAnnotation, TaskEstimation, TaskEstimator, + DistributedConfig, DistributedPhysicalOptimizerRule, DistributedPlannerExtension, + NetworkBoundary, NetworkBoundaryExt, plan_statistics, }; pub use execution_plans::{ BroadcastExec, DistributedExec, NetworkBroadcastExec, NetworkCoalesceExec, NetworkShuffleExec, diff --git a/src/metrics/task_metrics_collector.rs b/src/metrics/task_metrics_collector.rs index d1aa9d42..dc814226 100644 --- a/src/metrics/task_metrics_collector.rs +++ b/src/metrics/task_metrics_collector.rs @@ -152,9 +152,10 @@ mod tests { .with_distributed_worker_resolver(InMemoryWorkerResolver::new(10)) .with_distributed_channel_resolver(InMemoryChannelResolver::default()) .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) - .with_distributed_task_estimator(2) .with_distributed_metrics_collection(true) .unwrap() + .with_distributed_bytes_processed_per_partition(10) + .unwrap() .build(); let ctx = SessionContext::from(state); diff --git a/src/metrics/task_metrics_rewriter.rs b/src/metrics/task_metrics_rewriter.rs index 83cf0374..b4eb2a16 100644 --- a/src/metrics/task_metrics_rewriter.rs +++ b/src/metrics/task_metrics_rewriter.rs @@ -324,8 +324,9 @@ mod tests { .with_distributed_channel_resolver(InMemoryChannelResolver::default()) .with_distributed_metrics_collection(true) .unwrap() + .with_distributed_bytes_processed_per_partition(1) + .unwrap() .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) - .with_distributed_task_estimator(2) } let state = builder.build(); diff --git a/src/protobuf/distributed_codec.rs b/src/protobuf/distributed_codec.rs index 47aae6be..0fc9e94c 100644 --- a/src/protobuf/distributed_codec.rs +++ b/src/protobuf/distributed_codec.rs @@ -801,11 +801,20 @@ mod tests { dummy_stage(), )) as Arc; + fn d_ctx(task_index: usize, task_count: usize) -> DistributedTaskContext { + DistributedTaskContext { + task_index, + task_count, + } + } + let plan: Arc = Arc::new(ChildrenIsolatorUnionExec::from_children_and_task_counts( vec![left.clone(), right.clone()], - vec![2, 2], - 4, + vec![ + vec![(0, d_ctx(0, 2)), (0, d_ctx(1, 2))], + vec![(1, d_ctx(0, 2)), (1, d_ctx(1, 2))], + ], )?); let mut buf = Vec::new(); diff --git a/src/test_utils/mod.rs b/src/test_utils/mod.rs index e565d94f..786ea438 100644 --- a/src/test_utils/mod.rs +++ b/src/test_utils/mod.rs @@ -9,5 +9,6 @@ pub mod parquet; pub mod plans; pub mod property_based; pub mod session_context; +pub mod stats_vs_metrics; pub mod tpcds; pub mod tpch; diff --git a/src/test_utils/plans.rs b/src/test_utils/plans.rs index 0932c8af..79babc20 100644 --- a/src/test_utils/plans.rs +++ b/src/test_utils/plans.rs @@ -5,7 +5,7 @@ use crate::protobuf::StageKey; use crate::stage::Stage; use crate::test_utils::in_memory_channel_resolver::InMemoryWorkerResolver; #[cfg(test)] -use crate::{DistributedConfig, TaskEstimation, TaskEstimator}; +use crate::{DistributedConfig, DistributedPlannerExtension}; #[cfg(test)] use datafusion::config::ConfigOptions; use datafusion::{ @@ -107,7 +107,14 @@ pub async fn sql_to_physical_plan( let ctx = SessionContext::new_with_state(state); register_parquet_tables(&ctx).await.unwrap(); - let df = ctx.sql(query).await.unwrap(); + let mut queries = query.split(';').collect::>(); + let last_query = queries.pop().unwrap(); + + for query in queries { + ctx.sql(query).await.unwrap(); + } + + let df = ctx.sql(last_query).await.unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); format!("{}", displayable(physical_plan.as_ref()).indent(true)) @@ -174,15 +181,11 @@ pub(crate) async fn context_with_query( #[cfg(test)] #[derive(Debug)] -pub(crate) struct BuildSideOneTaskEstimator; +pub(crate) struct BuildSideOneDistributedPlannerExtension; #[cfg(test)] -impl TaskEstimator for BuildSideOneTaskEstimator { - fn task_estimation( - &self, - plan: &Arc, - _: &ConfigOptions, - ) -> Option { +impl DistributedPlannerExtension for BuildSideOneDistributedPlannerExtension { + fn max_tasks(&self, plan: &Arc, _: &ConfigOptions) -> Option { if !plan.children().is_empty() { return None; } @@ -190,10 +193,9 @@ impl TaskEstimator for BuildSideOneTaskEstimator { let has_min_temp = schema.fields().iter().any(|f| f.name() == "MinTemp"); let has_max_temp = schema.fields().iter().any(|f| f.name() == "MaxTemp"); if has_min_temp && !has_max_temp { - Some(TaskEstimation::maximum(1)) - } else { - None + return Some(1); } + None } fn scale_up_leaf_node( diff --git a/src/test_utils/stats_vs_metrics.rs b/src/test_utils/stats_vs_metrics.rs new file mode 100644 index 00000000..56d3dc23 --- /dev/null +++ b/src/test_utils/stats_vs_metrics.rs @@ -0,0 +1,107 @@ +use crate::{DistributedMetricsFormat, plan_statistics, rewrite_distributed_plan_with_metrics}; +use datafusion::common::{DataFusionError, Statistics}; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::physical_plan::metrics::MetricValue; +use std::fmt::{Debug, Formatter}; +use std::sync::Arc; + +#[derive(Debug, Default, Copy, Clone)] +pub struct StatsVsMetricsDisplayOptions { + display_output_rows: bool, + display_output_bytes: bool, +} + +impl StatsVsMetricsDisplayOptions { + pub fn with_display_output_rows(mut self) -> Self { + self.display_output_rows = true; + self + } + + pub fn with_display_output_bytes(mut self) -> Self { + self.display_output_bytes = true; + self + } +} + +pub fn stats_vs_metrics_display( + plan: Arc, + opts: StatsVsMetricsDisplayOptions, +) -> Result { + let plan = rewrite_distributed_plan_with_metrics(plan, DistributedMetricsFormat::Aggregated)?; + let node = Node::from_plan(plan, opts)?; + Ok(format!("{node:?}")) +} + +struct Node { + name: String, + stats: Statistics, + output_rows: Option, + output_bytes: Option, + children: Vec, + opts: StatsVsMetricsDisplayOptions, +} + +impl Debug for Node { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + fn fmt(f: &mut Formatter<'_>, node: &Node, depth: usize) -> std::fmt::Result { + for _ in 0..depth { + write!(f, " ")?; + } + write!(f, "{}:", node.name)?; + if let Some(actual) = &node.output_rows + && node.opts.display_output_rows + { + let estimated = *node.stats.num_rows.get_value().unwrap_or(&0); + let err = (100 * estimated.abs_diff(*actual)) / actual.max(&estimated).max(&1); + write!(f, " output_rows={estimated} vs {actual} ({err}%)")?; + } + if let Some(actual) = &node.output_bytes + && node.opts.display_output_bytes + { + let estimated = *node.stats.total_byte_size.get_value().unwrap_or(&0); + let err = (100 * estimated.abs_diff(*actual)) / actual.max(&estimated).max(&1); + write!(f, " output_bytes_err={estimated} vs {actual} ({err}%)")?; + } + writeln!(f)?; + for c in &node.children { + fmt(f, c, depth + 1)?; + } + Ok(()) + } + + fmt(f, self, 0) + } +} + +impl Node { + fn from_plan( + plan: Arc, + opts: StatsVsMetricsDisplayOptions, + ) -> Result { + let mut children = vec![]; + for child in plan.children() { + children.push(Node::from_plan(Arc::clone(child), opts)?); + } + + let mut node = Node { + name: plan.name().to_string(), + stats: plan_statistics( + &plan, + &children.iter().map(|v| &v.stats).collect::>(), + Default::default(), + )?, + output_rows: None, + output_bytes: None, + children, + opts, + }; + if let Some(metrics) = plan.metrics() { + node.output_rows = metrics.output_rows(); + node.output_bytes = metrics + .sum(|v| matches!(v.value(), MetricValue::OutputBytes(_))) + .map(|v| v.as_usize()); + } + + Ok(node) + } +} diff --git a/tests/clickbench_correctness_test.rs b/tests/clickbench_correctness_test.rs index fff1bf2e..fcf66ade 100644 --- a/tests/clickbench_correctness_test.rs +++ b/tests/clickbench_correctness_test.rs @@ -19,8 +19,7 @@ mod tests { use tokio::sync::OnceCell; const NUM_WORKERS: usize = 4; - const FILES_PER_TASK: usize = 2; - const CARDINALITY_TASK_COUNT_FACTOR: f64 = 2.0; + const BYTES_PROCESSED_PER_PARTITION: usize = 8 * 1024 * 1024; const FILE_RANGE: Range = 0..3; #[tokio::test] @@ -30,11 +29,13 @@ mod tests { } #[tokio::test] + #[ignore = "Query 1 did not get distributed"] async fn test_clickbench_1() -> Result<()> { test_clickbench_query("q1").await } #[tokio::test] + #[ignore = "Query 2 did not get distributed"] async fn test_clickbench_2() -> Result<()> { test_clickbench_query("q2").await } @@ -62,6 +63,7 @@ mod tests { } #[tokio::test] + #[ignore = "Query 6 did not get distributed"] async fn test_clickbench_7() -> Result<()> { test_clickbench_query("q7").await } @@ -280,8 +282,7 @@ mod tests { // Make distributed localhost context to run queries let (d_ctx, _guard, _) = start_localhost_context(NUM_WORKERS, DefaultSessionBuilder).await; let d_ctx = d_ctx - .with_distributed_files_per_task(FILES_PER_TASK)? - .with_distributed_cardinality_effect_task_scale_factor(CARDINALITY_TASK_COUNT_FACTOR)? + .with_distributed_bytes_processed_per_partition(BYTES_PROCESSED_PER_PARTITION)? .with_distributed_broadcast_joins(true)?; benchmarks_common::register_tables(&s_ctx, &data_dir).await?; diff --git a/tests/clickbench_plans_test.rs b/tests/clickbench_plans_test.rs index e851827f..f501aa5d 100644 --- a/tests/clickbench_plans_test.rs +++ b/tests/clickbench_plans_test.rs @@ -11,12 +11,10 @@ mod tests { use tokio::sync::OnceCell; const NUM_WORKERS: usize = 4; - const FILES_PER_TASK: usize = 2; - const CARDINALITY_TASK_COUNT_FACTOR: f64 = 1.5; + const BYTES_PROCESSED_PER_PARTITION: usize = 8 * 1024 * 1024; const FILE_RANGE: Range = 0..3; #[tokio::test] - #[ignore = "Query 0 did not get distributed"] async fn test_clickbench_0() -> Result<()> { let display = test_clickbench_query("q0").await?; assert_snapshot!(display, @""); @@ -26,54 +24,27 @@ mod tests { #[tokio::test] async fn test_clickbench_1() -> Result<()> { let display = test_clickbench_query("q1").await?; - assert_snapshot!(display, @r" - ┌───── DistributedExec ── Tasks: t0:[p0] - │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] - │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ FilterExec: AdvEngineID@0 != 0 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@40 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] - └────────────────────────────────────────────────── - "); + assert_snapshot!(display, @""); Ok(()) } #[tokio::test] async fn test_clickbench_2() -> Result<()> { let display = test_clickbench_query("q2").await?; - assert_snapshot!(display, @r" - ┌───── DistributedExec ── Tasks: t0:[p0] - │ ProjectionExec: expr=[sum(hits.AdvEngineID)@0 as sum(hits.AdvEngineID), count(Int64(1))@1 as count(*), avg(hits.ResolutionWidth)@2 as avg(hits.ResolutionWidth)] - │ AggregateExec: mode=Final, gby=[], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth)] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth)] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[ResolutionWidth, AdvEngineID], file_type=parquet - └────────────────────────────────────────────────── - "); + assert_snapshot!(display, @""); Ok(()) } #[tokio::test] - #[ignore = "result sets were not equal: Internal error: Row content differs between result sets\nLeft set size: 1, Right set size: 1\n\nRows only in left (1 total):\n 2533767602294735360.00\n\nRows only in right (1 total):\n 2533767602294735872.00.\nThis issue was likely caused by a bug in DataFusion's code. Please help us to resolve this by filing a bug report in our issue tracker: https://github.com/apache/datafusion/issues"] async fn test_clickbench_3() -> Result<()> { let display = test_clickbench_query("q3").await?; assert_snapshot!(display, @r" - ┌───── DistributedExec ── Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ AggregateExec: mode=Final, gby=[], aggr=[avg(hits.UserID)] │ CoalescePartitionsExec │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(hits.UserID)] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID], file_type=parquet @@ -90,17 +61,17 @@ mod tests { │ ProjectionExec: expr=[count(alias1)@0 as count(DISTINCT hits.UserID)] │ AggregateExec: mode=Final, gby=[], aggr=[count(alias1)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] │ AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([alias1@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([alias1@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[UserID@0 as alias1], aggr=[] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID], file_type=parquet └────────────────────────────────────────────────── "); @@ -115,17 +86,17 @@ mod tests { │ ProjectionExec: expr=[count(alias1)@0 as count(DISTINCT hits.SearchPhrase)] │ AggregateExec: mode=Final, gby=[], aggr=[count(alias1)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] │ AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([alias1@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([alias1@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[SearchPhrase@0 as alias1], aggr=[] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchPhrase], file_type=parquet └────────────────────────────────────────────────── "); @@ -133,7 +104,6 @@ mod tests { } #[tokio::test] - #[ignore = "Query 6 did not get distributed"] async fn test_clickbench_6() -> Result<()> { let display = test_clickbench_query("q6").await?; assert_snapshot!(display, @""); @@ -147,20 +117,19 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[AdvEngineID@0 as AdvEngineID, count(*)@1 as count(*)] │ SortPreservingMergeExec: [count(Int64(1))@2 DESC] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: expr=[count(*)@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[AdvEngineID@0 as AdvEngineID, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([AdvEngineID@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([AdvEngineID@0], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] │ FilterExec: AdvEngineID@0 != 0 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@40 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] + │ DataSourceExec: file_groups={3 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:.., /testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:.., /testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@40 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] └────────────────────────────────────────────────── "); Ok(()) @@ -172,23 +141,26 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [u@1 DESC], fetch=10 - │ SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[RegionID@0 as RegionID, count(alias1)@1 as u] - │ AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[count(alias1)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([RegionID@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[count(alias1)] - │ AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID, alias1@1 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([RegionID@0, alias1@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID, UserID@1 as alias1], aggr=[] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[RegionID, UserID], file_type=parquet + │ [Stage 3] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[RegionID@0 as RegionID, count(alias1)@1 as u] + │ AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[count(alias1)] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([RegionID@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID, alias1@1 as alias1], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([RegionID@0, alias1@1], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID, UserID@1 as alias1], aggr=[] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[RegionID, UserID], file_type=parquet + └────────────────────────────────────────────────── "); Ok(()) } @@ -199,18 +171,18 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[RegionID@0 as RegionID, sum(hits.AdvEngineID)@1 as sum(hits.AdvEngineID), count(Int64(1))@2 as c, avg(hits.ResolutionWidth)@3 as avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)@4 as count(DISTINCT hits.UserID)] │ AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([RegionID@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([RegionID@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[RegionID, UserID, ResolutionWidth, AdvEngineID], file_type=parquet └────────────────────────────────────────────────── "); @@ -223,24 +195,27 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [u@1 DESC], fetch=10 - │ SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[MobilePhoneModel@0 as MobilePhoneModel, count(alias1)@1 as u] - │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([MobilePhoneModel@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] - │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel, alias1@1 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([MobilePhoneModel@0, alias1@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[MobilePhoneModel@1 as MobilePhoneModel, UserID@0 as alias1], aggr=[] - │ FilterExec: MobilePhoneModel@1 != - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] + │ [Stage 3] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[MobilePhoneModel@0 as MobilePhoneModel, count(alias1)@1 as u] + │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([MobilePhoneModel@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel, alias1@1 as alias1], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ RepartitionExec: partitioning=Hash([MobilePhoneModel@0, alias1@1], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[MobilePhoneModel@1 as MobilePhoneModel, UserID@0 as alias1], aggr=[] + │ FilterExec: MobilePhoneModel@1 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -251,24 +226,27 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [u@2 DESC], fetch=10 - │ SortExec: TopK(fetch=10), expr=[u@2 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, count(alias1)@2 as u] - │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] - │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, alias1@2 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1, alias1@2], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[MobilePhone@1 as MobilePhone, MobilePhoneModel@2 as MobilePhoneModel, UserID@0 as alias1], aggr=[] - │ FilterExec: MobilePhoneModel@2 != - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, MobilePhone, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] + │ [Stage 3] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[u@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, count(alias1)@2 as u] + │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, alias1@2 as alias1], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] t3:[p0..p8] + │ RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1, alias1@2], 9), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[MobilePhone@1 as MobilePhone, MobilePhoneModel@2 as MobilePhoneModel, UserID@0 as alias1], aggr=[] + │ FilterExec: MobilePhoneModel@2 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, MobilePhone, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -279,19 +257,19 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@1 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[c@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(Int64(1))@1 as c] │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] │ FilterExec: SearchPhrase@0 != - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] └────────────────────────────────────────────────── "); @@ -304,24 +282,27 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [u@1 DESC], fetch=10 - │ SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(alias1)@1 as u] - │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] - │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase, alias1@1 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([SearchPhrase@0, alias1@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase, UserID@0 as alias1], aggr=[] - │ FilterExec: SearchPhrase@1 != - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + │ [Stage 3] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(alias1)@1 as u] + │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase, alias1@1 as alias1], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] t3:[p0..p8] + │ RepartitionExec: partitioning=Hash([SearchPhrase@0, alias1@1], 9), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase, UserID@0 as alias1], aggr=[] + │ FilterExec: SearchPhrase@1 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -332,19 +313,19 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as c] │ AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([SearchEngineID@0, SearchPhrase@1], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([SearchEngineID@0, SearchPhrase@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] │ FilterExec: SearchPhrase@1 != - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] └────────────────────────────────────────────────── "); @@ -358,18 +339,18 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[UserID@0 as UserID, count(*)@1 as count(*)] │ SortPreservingMergeExec: [count(Int64(1))@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[count(*)@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[UserID@0 as UserID, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([UserID@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([UserID@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID], file_type=parquet └────────────────────────────────────────────────── "); @@ -383,18 +364,18 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase, count(*)@2 as count(*)] │ SortPreservingMergeExec: [count(Int64(1))@3 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[count(*)@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as count(*), count(Int64(1))@2 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, SearchPhrase], file_type=parquet └────────────────────────────────────────────────── "); @@ -402,10 +383,25 @@ mod tests { } #[tokio::test] - #[ignore = "result sets were not equal: Internal error: Row content differs between result sets\nLeft set size: 10, Right set size: 10\n\nRows only in left (10 total):\n 3219866204653196665||4\n 3220056705148678697||11\n 3221898002592879542||1\n 3223026783585713477||23\n 3223839745005575457||116\n 3223839745005575457|d0bcd0bed0b6d0bdd0be20d0bbd0b820d0b220d0bad180d0bed0bad0bed0b4d0b8d180d0bed0b2d0b5d0bbd18cd18820d0b1d180d0bed0b4d18b20d0bdd0b020d181d182d0bed0bbd18b20d0b2d0be20d0b2d0bbd0b0d0b4d0b8d0b2d0bed181d182d0bed0ba20d0b2d0b2d0be|1\n 3223949769615485893||1\n 3226415756450197918||24\n 3226664959488084815||62\n 3227160743723019373||71\n\nRows only in right (10 total):\n 700182585509527889||2\n 724127359630680276|d0b8d0b3d180d18b20d0b820d181d0b5d0b3d0bed0b4d0bdd18f3f|1\n 766120398574852544||1\n 766739966065297239||1\n 783205612738304865||3\n 797289180007803204||2\n 804968013253615745||1\n 830548852254311605||1\n 849024737642146119||1\n 849169469997862534||1.\nThis issue was likely caused by a bug in DataFusion's code. Please help us to resolve this by filing a bug report in our issue tracker: https://github.com/apache/datafusion/issues"] async fn test_clickbench_17() -> Result<()> { let display = test_clickbench_query("q17").await?; - assert_snapshot!(display, @""); + assert_snapshot!(display, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ ProjectionExec: expr=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as count(*)] + │ CoalescePartitionsExec: fetch=10 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, SearchPhrase], file_type=parquet + └────────────────────────────────────────────────── + "); Ok(()) } @@ -416,18 +412,18 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[UserID@0 as UserID, m@1 as m, SearchPhrase@2 as SearchPhrase, count(*)@3 as count(*)] │ SortPreservingMergeExec: [count(Int64(1))@4 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[count(*)@3 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as m, SearchPhrase@2 as SearchPhrase, count(Int64(1))@3 as count(*), count(Int64(1))@3 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([UserID@0, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1, SearchPhrase@2], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([UserID@0, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1, SearchPhrase@2], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[UserID@1 as UserID, date_part(MINUTE, to_timestamp_seconds(EventTime@0)) as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventTime, UserID, SearchPhrase], file_type=parquet └────────────────────────────────────────────────── "#); @@ -440,11 +436,11 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ FilterExec: UserID@0 = 435090932899640449 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID], file_type=parquet, predicate=UserID@9 = 435090932899640449, pruning_predicate=UserID_null_count@2 != row_count@3 AND UserID_min@0 <= 435090932899640449 AND 435090932899640449 <= UserID_max@1, required_guarantees=[UserID in (435090932899640449)] └────────────────────────────────────────────────── "); @@ -459,13 +455,13 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ FilterExec: CAST(URL@0 AS Utf8View) LIKE %google% - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[URL], file_type=parquet, predicate=CAST(URL@13 AS Utf8View) LIKE %google% └────────────────────────────────────────────────── "); @@ -473,18 +469,52 @@ mod tests { } #[tokio::test] - #[ignore = "result sets were not equal: Internal error: Row content differs between result sets\nLeft set size: 10, Right set size: 10\n\nRows only in left (5 total):\n d181d0bbd0b0d0b2d0bbd18fd182d18c20d0bfd0bed180d0bed0b4d0b8d182d181d18f20d0bed182d0b5d0bbd0b8203230313320d181d0bcd0bed182d180d0b5d182d18c|687474703a253246253246766b2e636f6d2e75612f676f6f676c652d6a61726b6f76736b6179612d4c697065636b64|1\n d0b1d0b0d0bdd0bad0bed0bcd0b0d182d0b5d180d0b8d0b0d0bbd18b20d181d0bcd0bed182d180d0b5d182d18c|687474703a2f2f6f72656e627572672e6972722e72752532466b7572746b692532462532467777772e676f6f676c652e72752f6d617a64612d332d6b6f6d6e2d6b762d4b617a616e2e74757475746f72736b2f64657461696c|1\n d0bcd0bed0bdd0b8d182d18c20d0bad0b0d0bad0bed0b520d0bed0b7d0b5d180d0b0|687474703a2f2f6175746f2e7269612e75612f6175746f5f69643d30266f726465723d46616c7365266d696e707269782e72752f6b617465676f726979612f767369652d646c69612d647275676f652f6d61746572696e7374766f2f676f6f676c652d706f6c697331343334343532|1\n d181d0bad0b0d187d0b0d182d18c20d0b4d0b5d0bdd0b5d0b320d181d183d180d0b3d183d182|687474703a2f2f7469656e736b6169612d6d6f64612d627269657469656c6b612d6b6f736b6f76736b2f64657461696c2e676f6f676c65|1\n d0b220d0b0d0b2d0b3d183d181d1822032343720d0b3d180d183d181d182d0b8d0bcd0bed188d0bad0b020d0bdd0b020d0bad180d0b8d181d182d180d0b0d182|687474703a2f2f7469656e736b6169612d6d6f64612d627269756b692f676f6f676c652e72752f7e61706f6b2e72752f635f312d755f313138383839352c39373536|1\n\nRows only in right (5 total):\n d0bcd0bed0b4d0b5d0bad18120d183d0bbd0b8d186d0b5d0bdd0b7d0b8d0bdd0bed0b2d0b020d0b3d0bed0b2d18fd0b4d0b8d0bdd0b0|687474703a2f2f73616d6172612e6972722e72752f636174616c6f675f676f6f676c652d636865726e796a2d393233353636363635372f3f64617465|1\n d0bbd0b0d0b2d0bfd0bbd0b0d0bdd188d0b5d182d0bdd0b8d18520d183d181d0bbd0bed0b2d0b0d0bcd0b820d0b2d181d0b520d181d0b5d180d0b8d0b820d0b4d0b0d182d0b020d186d0b5d0bcd0b5d0bdd0b8|687474703a2f2f73616d6172612e6972722e72752f636174616c6f675f676f6f676c654d425225323661642533443930253236707a|1\n d0bad0b0d0ba20d0bfd180d0bed0b4d0b0d0bcd0b820d0b4d0bbd18f20d0b4d0b5d0b2d183d188d0bad0b8|687474703a253246253246777777772e626f6e707269782e7275253235326625323532663737363925323532663131303931392d6c65766f652d676f6f676c652d7368746f72792e72752f666f72756d2f666f72756d2e6d617465722e72752f6461696c792f63616c63756c61746f72|1\n d0b6d0b0d180d0b5d0bdd18cd18f20d0b32ed181d183d180d0bed0b2d0b0d0bdd0b8d0b520d0b2d0bed180d0bed0bdd0b5d0b6d181d0bad0b0d18f20d0bed0b1d0bbd0b0d181d182d0bed0bfd180d0b8d0bbd0b520d0bfd0bed181d0bbd0b5d0b4d0bdd0b8d0b520d0bad0bed181d18b|687474703a2f2f756b7261696e627572672f65636f2d6d6c656b2f65636f6e646172792f73686f77746f7069632e7068703f69643d3436333837362e68746d6c3f69643d32303634313333363631253246676f6f676c652d4170706c655765624b69742532463533372e333620284b48544d4c2c206c696b65|1\n d180d0b8d0be20d0bdd0b020d0bad0b0d180d182d0bed187d0bdd0b8d186d0b020d181d0bcd0bed182d180d0b5d182d18c20d0bed0bdd0bbd0b0d0b9d0bd|687474703a2f2f73616d6172612e6972722e72752f636174616c6f675f676f6f676c654d425225323661642533443930253236707a|1.\nThis issue was likely caused by a bug in DataFusion's code. Please help us to resolve this by filing a bug report in our issue tracker: https://github.com/apache/datafusion/issues"] async fn test_clickbench_21() -> Result<()> { let display = test_clickbench_query("q21").await?; - assert_snapshot!(display, @""); + assert_snapshot!(display, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [c@2 DESC], fetch=10 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, min(hits.URL)@1 as min(hits.URL), count(Int64(1))@2 as c] + │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))] + │ FilterExec: CAST(URL@0 AS Utf8View) LIKE %google% AND SearchPhrase@1 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[URL, SearchPhrase], file_type=parquet, predicate=CAST(URL@13 AS Utf8View) LIKE %google% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + └────────────────────────────────────────────────── + "); Ok(()) } #[tokio::test] - #[ignore = "result sets were not equal: Internal error: Row content differs between result sets\nLeft set size: 10, Right set size: 10\n\nRows only in left (1 total):\n d0bad0b0d0bad0bed0b920d0bfd0bbd0bed189d0b0d0b4d0bad0b8d0bcd0b820d0b4d0bed181d182d0b0d0b2d0bad0b8|687474703a253246253246766b2e636f6d2f696672616d652d6f77612e68746d6c3f313d31266369643d353737266f6b693d31266f705f63617465676f72795f69645d3d332673656c656374|d092d0b0d0bad0b0d0bdd181d0b8d18f20d091d0a0d090d09ad090d09d20d090d09dd094d0a0d095d0a1202d20d0bfd0bed0bfd0b0d0bbd0b820d0bad183d0bfd0b8d182d18c20d0b4d0bed0bcd0bed0b5d187d0bdd18bd0b520d188d0bad0b0d184d0b020476f6f676c652e636f6d203a3a20d0bad0bed182d182d0b5d0bad181d1822c20d091d183d180d18fd182d0bdd0b8d0bad0b820d0b4d0bbd18f20d0bfd0b5d187d18c20d0bcd0b5d0b1d0b5d0bbd18cd0b520d0b4d0bbd18f20d0b4d0b5d0b2d183d188d0bad0b0|5|1\n\nRows only in right (1 total):\n d0bad0bed0bfd182d0b8d0bcd0b8d0bad0b2d0b8d0b4d18b20d18ed180d0b8d0b920d0bfd0bed181d0bbd0b5d0b4d0bdd18fd18f|68747470733a2f2f70726f64756b747925324670756c6f76652e72752f626f6f6b6c79617474696f6e2d7761722d73696e696a2d393430343139342c3936323435332f666f746f|d09bd0b5d0b3d0bad0be20d0bdd0b020d183d187d0b0d181d182d0bdd18bd0b520d183d187d0b0d181d182d0bdd0b8d0bad0bed0b22e2c20d0a6d0b5d0bdd18b202d20d0a1d182d0b8d0bbd18cd0bdd0b0d18f20d0bfd0b0d180d0bdd0b5d0bc2e20d0a1d0b0d0b3d0b0d0bdd180d0bed0b320d0b4d0bed0b3d0b0d0b4d0b5d0bdd0b8d18f203a20d0a2d183d180d186d0b8d0b82c20d0bad183d0bfd0b8d182d18c20d18320313020d0b4d0bdd0b520d0bad0bed0bbd18cd0bdd18bd0b520d0bcd0b0d188d0b8d0bdd0bad0b820d0bdd0b520d0bfd180d0b5d0b4d181d182d0b0d0b2d0bad0b8202d20d09dd0bed0b2d0b0d18f20d18120d0b8d0b7d0b1d0b8d0b5d0bdd0b8d0b520d181d0bfd180d0bed0b4d0b0d0b6d0b03a20d0bad0bed182d18fd182d0b0203230313420d0b32ed0b22e20d0a6d0b5d0bdd0b03a2034373530302d313045434f30363020e28093202d2d2d2d2d2d2d2d20d0bad183d0bfd0b8d182d18c20d0bad0b2d0b0d180d182d0b8d180d18320d09ed180d0b5d0bdd0b1d183d180d0b32028d0a0d0bed181d181d0b8d0b82047616c616e7472617820466c616d696c6961646120476f6f676c652c204ed0be20313820d184d0bed182d0bed0bad0bed0bdd0b2d0b5d180d0ba20d0a1d183d0bfd0b5d18020d09ad0b0d180d0b4d0b8d0b3d0b0d0bd|5|1.\nThis issue was likely caused by a bug in DataFusion's code. Please help us to resolve this by filing a bug report in our issue tracker: https://github.com/apache/datafusion/issues"] async fn test_clickbench_22() -> Result<()> { let display = test_clickbench_query("q22").await?; - assert_snapshot!(display, @""); + assert_snapshot!(display, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [c@3 DESC], fetch=10 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[c@3 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, min(hits.URL)@1 as min(hits.URL), min(hits.Title)@2 as min(hits.Title), count(Int64(1))@3 as c, count(DISTINCT hits.UserID)@4 as count(DISTINCT hits.UserID)] + │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[SearchPhrase@3 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)] + │ FilterExec: CAST(Title@0 AS Utf8View) LIKE %Google% AND CAST(URL@2 AS Utf8View) NOT LIKE %.google.% AND SearchPhrase@3 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[Title, UserID, URL, SearchPhrase], file_type=parquet, predicate=CAST(Title@2 AS Utf8View) LIKE %Google% AND CAST(URL@13 AS Utf8View) NOT LIKE %.google.% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + └────────────────────────────────────────────────── + "); Ok(()) } @@ -494,12 +524,12 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [EventTime@4 ASC NULLS LAST], fetch=10 - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ SortExec: TopK(fetch=10), expr=[EventTime@4 ASC NULLS LAST], preserve_partitioning=[true] │ FilterExec: CAST(URL@13 AS Utf8View) LIKE %google% - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[WatchID, JavaEnable, Title, GoodEvent, EventTime, EventDate, CounterID, ClientIP, RegionID, UserID, CounterClass, OS, UserAgent, URL, Referer, IsRefresh, RefererCategoryID, RefererRegionID, URLCategoryID, URLRegionID, ResolutionWidth, ResolutionHeight, ResolutionDepth, FlashMajor, FlashMinor, FlashMinor2, NetMajor, NetMinor, UserAgentMajor, UserAgentMinor, CookieEnable, JavascriptEnable, IsMobile, MobilePhone, MobilePhoneModel, Params, IPNetworkID, TraficSourceID, SearchEngineID, SearchPhrase, AdvEngineID, IsArtifical, WindowClientWidth, WindowClientHeight, ClientTimeZone, ClientEventTime, SilverlightVersion1, SilverlightVersion2, SilverlightVersion3, SilverlightVersion4, PageCharset, CodeVersion, IsLink, IsDownload, IsNotBounce, FUniqID, OriginalURL, HID, IsOldCounter, IsEvent, IsParameter, DontCountHits, WithHash, HitColor, LocalEventTime, Age, Sex, Income, Interests, Robotness, RemoteIP, WindowName, OpenerName, HistoryLength, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction, HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, SocialSourcePage, ParamPrice, ParamOrderID, ParamCurrency, ParamCurrencyID, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash, URLHash, CLID], file_type=parquet, predicate=CAST(URL@13 AS Utf8View) LIKE %google% AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -507,10 +537,22 @@ mod tests { } #[tokio::test] - #[ignore = "result sets were not equal: Internal error: Row content differs between result sets\nLeft set size: 10, Right set size: 10\n\nRows only in left (1 total):\n d0b2d181d0bfd0bed0bcd0bdd0b8d182d18c20d181d0bed0bbd0bdd0b5d0bdd0b8d0b520d0b1d0b0d0bdd0bad0b020d0bbd0b0d0b420d184d0b8d0bbd18cd0bc\n\nRows only in right (1 total):\n d0bed182d0b2d0bed0b4d0b020d0b4d0bbd18f20d0bfd0b8d180d0bed0b6d0bad0b820d0bbd0b5d187d0b5d0bdd0bdd18b20d0b2d181d0b520d181d0b5d180d196d197.\nThis issue was likely caused by a bug in DataFusion's code. Please help us to resolve this by filing a bug report in our issue tracker: https://github.com/apache/datafusion/issues"] async fn test_clickbench_24() -> Result<()> { let display = test_clickbench_query("q24").await?; - assert_snapshot!(display, @""); + assert_snapshot!(display, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase] + │ SortPreservingMergeExec: [EventTime@1 ASC NULLS LAST], fetch=10 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ SortExec: TopK(fetch=10), expr=[EventTime@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[SearchPhrase@1 as SearchPhrase, EventTime@0 as EventTime] + │ FilterExec: SearchPhrase@1 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + └────────────────────────────────────────────────── + "); Ok(()) } @@ -520,12 +562,12 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [SearchPhrase@0 ASC NULLS LAST], fetch=10 - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ SortExec: TopK(fetch=10), expr=[SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] │ FilterExec: SearchPhrase@0 != - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] └────────────────────────────────────────────────── "); @@ -539,13 +581,13 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase] │ SortPreservingMergeExec: [EventTime@1 ASC NULLS LAST, SearchPhrase@0 ASC NULLS LAST], fetch=10 - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ SortExec: TopK(fetch=10), expr=[EventTime@1 ASC NULLS LAST, SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[SearchPhrase@1 as SearchPhrase, EventTime@0 as EventTime] │ FilterExec: SearchPhrase@1 != - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] └────────────────────────────────────────────────── "); @@ -558,20 +600,20 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l@1 DESC], fetch=25 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[CounterID@0 as CounterID, avg(length(hits.URL))@1 as l, count(Int64(1))@2 as c] │ FilterExec: count(Int64(1))@2 > 100000 │ AggregateExec: mode=FinalPartitioned, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([CounterID@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CounterID@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] │ FilterExec: URL@1 != - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[CounterID, URL], file_type=parquet, predicate=URL@13 != , pruning_predicate=URL_null_count@2 != row_count@3 AND (URL_min@0 != OR != URL_max@1), required_guarantees=[URL not in ()] └────────────────────────────────────────────────── "); @@ -584,20 +626,20 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l@1 DESC], fetch=25 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as k, avg(length(hits.Referer))@1 as l, count(Int64(1))@2 as c, min(hits.Referer)@3 as min(hits.Referer)] │ FilterExec: count(Int64(1))@2 > 100000 │ AggregateExec: mode=FinalPartitioned, gby=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[regexp_replace(CAST(Referer@0 AS LargeUtf8), ^https?://(?:www\.)?([^/]+)/.*$, \1) as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] │ FilterExec: Referer@0 != - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[Referer], file_type=parquet, predicate=Referer@14 != , pruning_predicate=Referer_null_count@2 != row_count@3 AND (Referer_min@0 != OR != Referer_max@1), required_guarantees=[Referer not in ()] └────────────────────────────────────────────────── "#); @@ -611,11 +653,11 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ AggregateExec: mode=Final, gby=[], aggr=[sum(hits.ResolutionWidth), sum(hits.ResolutionWidth + Int64(1)), sum(hits.ResolutionWidth + Int64(2)), sum(hits.ResolutionWidth + Int64(3)), sum(hits.ResolutionWidth + Int64(4)), sum(hits.ResolutionWidth + Int64(5)), sum(hits.ResolutionWidth + Int64(6)), sum(hits.ResolutionWidth + Int64(7)), sum(hits.ResolutionWidth + Int64(8)), sum(hits.ResolutionWidth + Int64(9)), sum(hits.ResolutionWidth + Int64(10)), sum(hits.ResolutionWidth + Int64(11)), sum(hits.ResolutionWidth + Int64(12)), sum(hits.ResolutionWidth + Int64(13)), sum(hits.ResolutionWidth + Int64(14)), sum(hits.ResolutionWidth + Int64(15)), sum(hits.ResolutionWidth + Int64(16)), sum(hits.ResolutionWidth + Int64(17)), sum(hits.ResolutionWidth + Int64(18)), sum(hits.ResolutionWidth + Int64(19)), sum(hits.ResolutionWidth + Int64(20)), sum(hits.ResolutionWidth + Int64(21)), sum(hits.ResolutionWidth + Int64(22)), sum(hits.ResolutionWidth + Int64(23)), sum(hits.ResolutionWidth + Int64(24)), sum(hits.ResolutionWidth + Int64(25)), sum(hits.ResolutionWidth + Int64(26)), sum(hits.ResolutionWidth + Int64(27)), sum(hits.ResolutionWidth + Int64(28)), sum(hits.ResolutionWidth + Int64(29)), sum(hits.ResolutionWidth + Int64(30)), sum(hits.ResolutionWidth + Int64(31)), sum(hits.ResolutionWidth + Int64(32)), sum(hits.ResolutionWidth + Int64(33)), sum(hits.ResolutionWidth + Int64(34)), sum(hits.ResolutionWidth + Int64(35)), sum(hits.ResolutionWidth + Int64(36)), sum(hits.ResolutionWidth + Int64(37)), sum(hits.ResolutionWidth + Int64(38)), sum(hits.ResolutionWidth + Int64(39)), sum(hits.ResolutionWidth + Int64(40)), sum(hits.ResolutionWidth + Int64(41)), sum(hits.ResolutionWidth + Int64(42)), sum(hits.ResolutionWidth + Int64(43)), sum(hits.ResolutionWidth + Int64(44)), sum(hits.ResolutionWidth + Int64(45)), sum(hits.ResolutionWidth + Int64(46)), sum(hits.ResolutionWidth + Int64(47)), sum(hits.ResolutionWidth + Int64(48)), sum(hits.ResolutionWidth + Int64(49)), sum(hits.ResolutionWidth + Int64(50)), sum(hits.ResolutionWidth + Int64(51)), sum(hits.ResolutionWidth + Int64(52)), sum(hits.ResolutionWidth + Int64(53)), sum(hits.ResolutionWidth + Int64(54)), sum(hits.ResolutionWidth + Int64(55)), sum(hits.ResolutionWidth + Int64(56)), sum(hits.ResolutionWidth + Int64(57)), sum(hits.ResolutionWidth + Int64(58)), sum(hits.ResolutionWidth + Int64(59)), sum(hits.ResolutionWidth + Int64(60)), sum(hits.ResolutionWidth + Int64(61)), sum(hits.ResolutionWidth + Int64(62)), sum(hits.ResolutionWidth + Int64(63)), sum(hits.ResolutionWidth + Int64(64)), sum(hits.ResolutionWidth + Int64(65)), sum(hits.ResolutionWidth + Int64(66)), sum(hits.ResolutionWidth + Int64(67)), sum(hits.ResolutionWidth + Int64(68)), sum(hits.ResolutionWidth + Int64(69)), sum(hits.ResolutionWidth + Int64(70)), sum(hits.ResolutionWidth + Int64(71)), sum(hits.ResolutionWidth + Int64(72)), sum(hits.ResolutionWidth + Int64(73)), sum(hits.ResolutionWidth + Int64(74)), sum(hits.ResolutionWidth + Int64(75)), sum(hits.ResolutionWidth + Int64(76)), sum(hits.ResolutionWidth + Int64(77)), sum(hits.ResolutionWidth + Int64(78)), sum(hits.ResolutionWidth + Int64(79)), sum(hits.ResolutionWidth + Int64(80)), sum(hits.ResolutionWidth + Int64(81)), sum(hits.ResolutionWidth + Int64(82)), sum(hits.ResolutionWidth + Int64(83)), sum(hits.ResolutionWidth + Int64(84)), sum(hits.ResolutionWidth + Int64(85)), sum(hits.ResolutionWidth + Int64(86)), sum(hits.ResolutionWidth + Int64(87)), sum(hits.ResolutionWidth + Int64(88)), sum(hits.ResolutionWidth + Int64(89))] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(hits.ResolutionWidth), sum(hits.ResolutionWidth + Int64(1)), sum(hits.ResolutionWidth + Int64(2)), sum(hits.ResolutionWidth + Int64(3)), sum(hits.ResolutionWidth + Int64(4)), sum(hits.ResolutionWidth + Int64(5)), sum(hits.ResolutionWidth + Int64(6)), sum(hits.ResolutionWidth + Int64(7)), sum(hits.ResolutionWidth + Int64(8)), sum(hits.ResolutionWidth + Int64(9)), sum(hits.ResolutionWidth + Int64(10)), sum(hits.ResolutionWidth + Int64(11)), sum(hits.ResolutionWidth + Int64(12)), sum(hits.ResolutionWidth + Int64(13)), sum(hits.ResolutionWidth + Int64(14)), sum(hits.ResolutionWidth + Int64(15)), sum(hits.ResolutionWidth + Int64(16)), sum(hits.ResolutionWidth + Int64(17)), sum(hits.ResolutionWidth + Int64(18)), sum(hits.ResolutionWidth + Int64(19)), sum(hits.ResolutionWidth + Int64(20)), sum(hits.ResolutionWidth + Int64(21)), sum(hits.ResolutionWidth + Int64(22)), sum(hits.ResolutionWidth + Int64(23)), sum(hits.ResolutionWidth + Int64(24)), sum(hits.ResolutionWidth + Int64(25)), sum(hits.ResolutionWidth + Int64(26)), sum(hits.ResolutionWidth + Int64(27)), sum(hits.ResolutionWidth + Int64(28)), sum(hits.ResolutionWidth + Int64(29)), sum(hits.ResolutionWidth + Int64(30)), sum(hits.ResolutionWidth + Int64(31)), sum(hits.ResolutionWidth + Int64(32)), sum(hits.ResolutionWidth + Int64(33)), sum(hits.ResolutionWidth + Int64(34)), sum(hits.ResolutionWidth + Int64(35)), sum(hits.ResolutionWidth + Int64(36)), sum(hits.ResolutionWidth + Int64(37)), sum(hits.ResolutionWidth + Int64(38)), sum(hits.ResolutionWidth + Int64(39)), sum(hits.ResolutionWidth + Int64(40)), sum(hits.ResolutionWidth + Int64(41)), sum(hits.ResolutionWidth + Int64(42)), sum(hits.ResolutionWidth + Int64(43)), sum(hits.ResolutionWidth + Int64(44)), sum(hits.ResolutionWidth + Int64(45)), sum(hits.ResolutionWidth + Int64(46)), sum(hits.ResolutionWidth + Int64(47)), sum(hits.ResolutionWidth + Int64(48)), sum(hits.ResolutionWidth + Int64(49)), sum(hits.ResolutionWidth + Int64(50)), sum(hits.ResolutionWidth + Int64(51)), sum(hits.ResolutionWidth + Int64(52)), sum(hits.ResolutionWidth + Int64(53)), sum(hits.ResolutionWidth + Int64(54)), sum(hits.ResolutionWidth + Int64(55)), sum(hits.ResolutionWidth + Int64(56)), sum(hits.ResolutionWidth + Int64(57)), sum(hits.ResolutionWidth + Int64(58)), sum(hits.ResolutionWidth + Int64(59)), sum(hits.ResolutionWidth + Int64(60)), sum(hits.ResolutionWidth + Int64(61)), sum(hits.ResolutionWidth + Int64(62)), sum(hits.ResolutionWidth + Int64(63)), sum(hits.ResolutionWidth + Int64(64)), sum(hits.ResolutionWidth + Int64(65)), sum(hits.ResolutionWidth + Int64(66)), sum(hits.ResolutionWidth + Int64(67)), sum(hits.ResolutionWidth + Int64(68)), sum(hits.ResolutionWidth + Int64(69)), sum(hits.ResolutionWidth + Int64(70)), sum(hits.ResolutionWidth + Int64(71)), sum(hits.ResolutionWidth + Int64(72)), sum(hits.ResolutionWidth + Int64(73)), sum(hits.ResolutionWidth + Int64(74)), sum(hits.ResolutionWidth + Int64(75)), sum(hits.ResolutionWidth + Int64(76)), sum(hits.ResolutionWidth + Int64(77)), sum(hits.ResolutionWidth + Int64(78)), sum(hits.ResolutionWidth + Int64(79)), sum(hits.ResolutionWidth + Int64(80)), sum(hits.ResolutionWidth + Int64(81)), sum(hits.ResolutionWidth + Int64(82)), sum(hits.ResolutionWidth + Int64(83)), sum(hits.ResolutionWidth + Int64(84)), sum(hits.ResolutionWidth + Int64(85)), sum(hits.ResolutionWidth + Int64(86)), sum(hits.ResolutionWidth + Int64(87)), sum(hits.ResolutionWidth + Int64(88)), sum(hits.ResolutionWidth + Int64(89))] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[CAST(ResolutionWidth@20 AS Int64) as __common_expr_1], file_type=parquet └────────────────────────────────────────────────── "); @@ -628,19 +670,19 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] │ AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([SearchEngineID@0, ClientIP@1], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([SearchEngineID@0, ClientIP@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[SearchEngineID@3 as SearchEngineID, ClientIP@0 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] │ FilterExec: SearchPhrase@4 != , projection=[ClientIP@0, IsRefresh@1, ResolutionWidth@2, SearchEngineID@3] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[ClientIP, IsRefresh, ResolutionWidth, SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] └────────────────────────────────────────────────── "); @@ -648,18 +690,51 @@ mod tests { } #[tokio::test] - #[ignore = "result sets were not equal: Internal error: Row content differs between result sets\nLeft set size: 10, Right set size: 10\n\nRows only in left (10 total):\n 8673025726158767406|1264438551|1|0|1990.00\n 5320052218057629211|-1703087277|1|0|1996.00\n 6244273852606083750|1554672832|1|0|1638.00\n 8628753750962053665|1215278356|1|0|1087.00\n 7035318163404387241|1326714320|1|0|1638.00\n 8431857775494210873|1237512945|1|0|1996.00\n 5110752526539992124|37611695|1|0|1917.00\n 8986794334343068049|1860752926|1|0|1638.00\n 8044147848299485837|1382122372|1|0|1368.00\n 7936057634954670727|1897481896|1|0|1638.00\n\nRows only in right (10 total):\n 5132615111782210132|-50313020|1|0|1368.00\n 5783789691451717551|-1310327384|1|0|1638.00\n 5756260993772351383|1484317883|1|0|375.00\n 7739310142000732364|991864113|1|0|1368.00\n 7593472904893539271|-151291403|1|0|1087.00\n 6339599967989898410|1543815587|1|0|1638.00\n 7794346560421945218|1645556180|1|0|1368.00\n 6112645108657361792|593586188|1|0|1638.00\n 6675910710751922756|-816379256|1|0|1368.00\n 5802727636196431835|1986422271|1|0|1996.00.\nThis issue was likely caused by a bug in DataFusion's code. Please help us to resolve this by filing a bug report in our issue tracker: https://github.com/apache/datafusion/issues"] async fn test_clickbench_31() -> Result<()> { let display = test_clickbench_query("q31").await?; - assert_snapshot!(display, @""); + assert_snapshot!(display, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [c@2 DESC], fetch=10 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[WatchID@0 as WatchID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] + │ AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([WatchID@0, ClientIP@1], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] + │ FilterExec: SearchPhrase@4 != , projection=[WatchID@0, ClientIP@1, IsRefresh@2, ResolutionWidth@3] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + └────────────────────────────────────────────────── + "); Ok(()) } #[tokio::test] - #[ignore = "result sets were not equal: Internal error: Row content differs between result sets\nLeft set size: 10, Right set size: 10\n\nRows only in left (10 total):\n 7643059318918524417|1767085700|1|0|0.00\n 5437163248266133938|-1465369615|1|0|0.00\n 9142541582422390102|-1465369615|1|0|0.00\n 8438994503411842126|-1465369615|1|0|0.00\n 7362096505818029859|-565678477|1|0|0.00\n 4928022308880516715|1699955284|1|0|0.00\n 5269769817689282522|1699955284|1|0|0.00\n 9081648050908046886|1699955284|1|0|0.00\n 6824181869275536503|1699955284|1|0|0.00\n 6905712404475757487|1552811156|1|0|0.00\n\nRows only in right (10 total):\n 6967277596165459879|-941091661|1|0|1368.00\n 5796237228224217668|-1310327384|1|0|1638.00\n 7218628137278606666|1511490240|1|0|1638.00\n 8314760197723815280|1566105210|1|0|1996.00\n 7053263954762394007|757778490|1|0|339.00\n 6283334114093174531|1216031795|1|0|1368.00\n 8818295356247036741|83042182|1|0|1638.00\n 6620528864937282562|-862894777|1|0|1996.00\n 8466121050002905379|83042182|1|0|1638.00\n 7554844936512227411|-1746904856|1|0|1368.00.\nThis issue was likely caused by a bug in DataFusion's code. Please help us to resolve this by filing a bug report in our issue tracker: https://github.com/apache/datafusion/issues"] async fn test_clickbench_32() -> Result<()> { let display = test_clickbench_query("q32").await?; - assert_snapshot!(display, @""); + assert_snapshot!(display, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [c@2 DESC], fetch=10 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[WatchID@0 as WatchID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] + │ AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([WatchID@0, ClientIP@1], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth], file_type=parquet + └────────────────────────────────────────────────── + "); Ok(()) } @@ -669,18 +744,18 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@1 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[c@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as c] │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([URL@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[URL], file_type=parquet └────────────────────────────────────────────────── "); @@ -693,18 +768,18 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[1 as Int64(1), URL@0 as URL, count(Int64(1))@1 as c] │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([URL@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[URL], file_type=parquet └────────────────────────────────────────────────── "); @@ -717,18 +792,18 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@4 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[c@4 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[ClientIP@0 as ClientIP, hits.ClientIP - Int64(1)@1 as hits.ClientIP - Int64(1), hits.ClientIP - Int64(2)@2 as hits.ClientIP - Int64(2), hits.ClientIP - Int64(3)@3 as hits.ClientIP - Int64(3), count(Int64(1))@4 as c] │ AggregateExec: mode=FinalPartitioned, gby=[ClientIP@0 as ClientIP, hits.ClientIP - Int64(1)@1 as hits.ClientIP - Int64(1), hits.ClientIP - Int64(2)@2 as hits.ClientIP - Int64(2), hits.ClientIP - Int64(3)@3 as hits.ClientIP - Int64(3)], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ClientIP@0, hits.ClientIP - Int64(1)@1, hits.ClientIP - Int64(2)@2, hits.ClientIP - Int64(3)@3], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ClientIP@0, hits.ClientIP - Int64(1)@1, hits.ClientIP - Int64(2)@2, hits.ClientIP - Int64(3)@3], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ClientIP@1 as ClientIP, __common_expr_1@0 - 1 as hits.ClientIP - Int64(1), __common_expr_1@0 - 2 as hits.ClientIP - Int64(2), __common_expr_1@0 - 3 as hits.ClientIP - Int64(3)], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[CAST(ClientIP@7 AS Int64) as __common_expr_1, ClientIP], file_type=parquet └────────────────────────────────────────────────── "); @@ -741,19 +816,19 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([URL@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND URL@2 != , projection=[URL@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, URL, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(EventDate@5 AS Utf8) >= 2013-07-01 AND CAST(EventDate@5 AS Utf8) <= 2013-07-31 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND URL@13 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND DontCountHits_null_count@6 != row_count@3 AND DontCountHits_min@4 <= 0 AND 0 <= DontCountHits_max@5 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND URL_null_count@12 != row_count@3 AND (URL_min@10 != OR != URL_max@11), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URL not in ()] └────────────────────────────────────────────────── "); @@ -766,19 +841,19 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[Title@0 as Title, count(Int64(1))@1 as pageviews] │ AggregateExec: mode=FinalPartitioned, gby=[Title@0 as Title], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([Title@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([Title@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[Title@0 as Title], aggr=[count(Int64(1))] │ FilterExec: CounterID@2 = 62 AND CAST(EventDate@1 AS Utf8) >= 2013-07-01 AND CAST(EventDate@1 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND Title@0 != , projection=[Title@0] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[Title, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(EventDate@5 AS Utf8) >= 2013-07-01 AND CAST(EventDate@5 AS Utf8) <= 2013-07-31 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND Title@2 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND DontCountHits_null_count@6 != row_count@3 AND DontCountHits_min@4 <= 0 AND 0 <= DontCountHits_max@5 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND Title_null_count@12 != row_count@3 AND (Title_min@10 != OR != Title_max@11), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), Title not in ()] └────────────────────────────────────────────────── "); @@ -792,19 +867,19 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=1000, fetch=10 │ SortPreservingMergeExec: [pageviews@1 DESC], fetch=1010 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=1010), expr=[pageviews@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([URL@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@3 = 0 AND IsLink@4 != 0 AND IsDownload@5 = 0, projection=[URL@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, URL, IsRefresh, IsLink, IsDownload], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(EventDate@5 AS Utf8) >= 2013-07-01 AND CAST(EventDate@5 AS Utf8) <= 2013-07-31 AND IsRefresh@15 = 0 AND IsLink@52 != 0 AND IsDownload@53 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND IsLink_null_count@9 != row_count@3 AND (IsLink_min@7 != 0 OR 0 != IsLink_max@8) AND IsDownload_null_count@12 != row_count@3 AND IsDownload_min@10 <= 0 AND 0 <= IsDownload_max@11, required_guarantees=[CounterID in (62), IsDownload in (0), IsLink not in (0), IsRefresh in (0)] └────────────────────────────────────────────────── "); @@ -818,19 +893,19 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=1000, fetch=10 │ SortPreservingMergeExec: [pageviews@5 DESC], fetch=1010 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=1010), expr=[pageviews@5 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as src, URL@4 as dst, count(Int64(1))@5 as pageviews] │ AggregateExec: mode=FinalPartitioned, gby=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@4 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([TraficSourceID@0, SearchEngineID@1, AdvEngineID@2, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3, URL@4], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([TraficSourceID@0, SearchEngineID@1, AdvEngineID@2, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3, URL@4], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[TraficSourceID@2 as TraficSourceID, SearchEngineID@3 as SearchEngineID, AdvEngineID@4 as AdvEngineID, CASE WHEN SearchEngineID@3 = 0 AND AdvEngineID@4 = 0 THEN Referer@1 ELSE END as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@0 as URL], aggr=[count(Int64(1))] │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@4 = 0, projection=[URL@2, Referer@3, TraficSourceID@5, SearchEngineID@6, AdvEngineID@7] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, URL, Referer, IsRefresh, TraficSourceID, SearchEngineID, AdvEngineID], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(EventDate@5 AS Utf8) >= 2013-07-01 AND CAST(EventDate@5 AS Utf8) <= 2013-07-31 AND IsRefresh@15 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5, required_guarantees=[CounterID in (62), IsRefresh in (0)] └────────────────────────────────────────────────── "#); @@ -844,19 +919,19 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=100, fetch=10 │ SortPreservingMergeExec: [pageviews@2 DESC], fetch=110 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=110), expr=[pageviews@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[URLHash@0 as URLHash, EventDate@1 as EventDate, count(Int64(1))@2 as pageviews] │ AggregateExec: mode=FinalPartitioned, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([URLHash@0, EventDate@1], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([URLHash@0, EventDate@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[URLHash@1 as URLHash, EventDate@0 as EventDate], aggr=[count(Int64(1))] │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, projection=[EventDate@0, URLHash@5] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, IsRefresh, TraficSourceID, RefererHash, URLHash], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(EventDate@5 AS Utf8) >= 2013-07-01 AND CAST(EventDate@5 AS Utf8) <= 2013-07-31 AND IsRefresh@15 = 0 AND (TraficSourceID@37 = -1 OR TraficSourceID@37 = 6) AND RefererHash@102 = 3594120000172545465, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND (TraficSourceID_null_count@9 != row_count@3 AND TraficSourceID_min@7 <= -1 AND -1 <= TraficSourceID_max@8 OR TraficSourceID_null_count@9 != row_count@3 AND TraficSourceID_min@7 <= 6 AND 6 <= TraficSourceID_max@8) AND RefererHash_null_count@12 != row_count@3 AND RefererHash_min@10 <= 3594120000172545465 AND 3594120000172545465 <= RefererHash_max@11, required_guarantees=[CounterID in (62), IsRefresh in (0), RefererHash in (3594120000172545465), TraficSourceID in (-1, 6)] └────────────────────────────────────────────────── "); @@ -870,19 +945,19 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=10000, fetch=10 │ SortPreservingMergeExec: [pageviews@2 DESC], fetch=10010 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=10010), expr=[pageviews@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight, count(Int64(1))@2 as pageviews] │ AggregateExec: mode=FinalPartitioned, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([WindowClientWidth@0, WindowClientHeight@1], 6), input_partitions=2 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([WindowClientWidth@0, WindowClientHeight@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND DontCountHits@5 = 0 AND URLHash@6 = 2868770270353813622, projection=[WindowClientWidth@3, WindowClientHeight@4] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, IsRefresh, WindowClientWidth, WindowClientHeight, DontCountHits, URLHash], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(EventDate@5 AS Utf8) >= 2013-07-01 AND CAST(EventDate@5 AS Utf8) <= 2013-07-31 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0 AND URLHash@103 = 2868770270353813622, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND URLHash_null_count@12 != row_count@3 AND URLHash_min@10 <= 2868770270353813622 AND 2868770270353813622 <= URLHash_max@11, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URLHash in (2868770270353813622)] └────────────────────────────────────────────────── "); @@ -890,10 +965,28 @@ mod tests { } #[tokio::test] - #[ignore = "ordering mismatch: expected ordering: Some(LexOrdering { exprs: [PhysicalSortExpr { expr: ScalarFunctionExpr { fun: \"\", name: \"date_trunc\", args: [Literal { value: Utf8(\"minute\"), field: Field { name: \"lit\", data_type: Utf8 } }, Column { name: \"m\", index: 0 }], return_field: Field { name: \"date_trunc\", data_type: Timestamp(Second, None), nullable: true } }, options: SortOptions { descending: false, nulls_first: false } }], set: {ScalarFunctionExpr { fun: \"\", name: \"date_trunc\", args: [Literal { value: Utf8(\"minute\"), field: Field { name: \"lit\", data_type: Utf8 } }, Column { name: \"m\", index: 0 }], return_field: Field { name: \"date_trunc\", data_type: Timestamp(Second, None), nullable: true } }} }), actual ordering: Some(LexOrdering { exprs: [PhysicalSortExpr { expr: ScalarFunctionExpr { fun: \"\", name: \"date_trunc\", args: [Literal { value: Utf8(\"minute\"), field: Field { name: \"lit\", data_type: Utf8 } }, Column { name: \"m\", index: 0 }], return_field: Field { name: \"date_trunc\", data_type: Timestamp(Second, None), nullable: true } }, options: SortOptions { descending: false, nulls_first: false } }], set: {ScalarFunctionExpr { fun: \"\", name: \"date_trunc\", args: [Literal { value: Utf8(\"minute\"), field: Field { name: \"lit\", data_type: Utf8 } }, Column { name: \"m\", index: 0 }], return_field: Field { name: \"date_trunc\", data_type: Timestamp(Second, None), nullable: true } }} })"] async fn test_clickbench_42() -> Result<()> { let display = test_clickbench_query("q42").await?; - assert_snapshot!(display, @""); + assert_snapshot!(display, @r#" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ GlobalLimitExec: skip=1000, fetch=10 + │ SortPreservingMergeExec: [date_trunc(minute, m@0) ASC NULLS LAST], fetch=1010 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=1010), expr=[date_trunc(minute, m@0) ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0 as m, count(Int64(1))@1 as pageviews] + │ AggregateExec: mode=FinalPartitioned, gby=[date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0 as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[date_trunc(minute, to_timestamp_seconds(EventTime@0)) as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))] + │ FilterExec: CounterID@2 = 62 AND CAST(EventDate@1 AS Utf8) >= 2013-07-14 AND CAST(EventDate@1 AS Utf8) <= 2013-07-15 AND IsRefresh@3 = 0 AND DontCountHits@4 = 0, projection=[EventTime@0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,__,__] t2:[__,__,__,p0,__] t3:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventTime, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(EventDate@5 AS Utf8) >= 2013-07-14 AND CAST(EventDate@5 AS Utf8) <= 2013-07-15 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0)] + └────────────────────────────────────────────────── + "#); Ok(()) } @@ -916,8 +1009,7 @@ mod tests { let (d_ctx, _guard, _) = start_localhost_context(NUM_WORKERS, DefaultSessionBuilder).await; let d_ctx = d_ctx - .with_distributed_files_per_task(FILES_PER_TASK)? - .with_distributed_cardinality_effect_task_scale_factor(CARDINALITY_TASK_COUNT_FACTOR)? + .with_distributed_bytes_processed_per_partition(BYTES_PROCESSED_PER_PARTITION)? .with_distributed_broadcast_joins(true)?; benchmarks_common::register_tables(&d_ctx, &data_dir).await?; diff --git a/tests/custom_config_extension.rs b/tests/custom_config_extension.rs index cf493692..14c324d2 100644 --- a/tests/custom_config_extension.rs +++ b/tests/custom_config_extension.rs @@ -32,6 +32,7 @@ mod tests { #[tokio::test] async fn custom_config_extension() -> Result<(), Box> { let (mut ctx, _guard, _) = start_localhost_context(3, build_state).await; + ctx.set_distributed_bytes_processed_per_partition(1000)?; ctx.set_distributed_user_codec(CustomConfigExtensionRequiredExecCodec); ctx.set_distributed_option_extension(CustomExtension { foo: "foo".to_string(), diff --git a/tests/custom_extension_codec.rs b/tests/custom_extension_codec.rs index baaacc4f..d1bdf4c7 100644 --- a/tests/custom_extension_codec.rs +++ b/tests/custom_extension_codec.rs @@ -31,6 +31,7 @@ mod tests { } let (mut ctx, _guard, _) = start_localhost_context(3, build_state).await; + ctx.set_distributed_bytes_processed_per_partition(1000)?; ctx.set_distributed_user_codec(CustomPassThroughExecCodec); let query = r#"SELECT "MinTemp", "RainToday" FROM weather WHERE "MinTemp" > 20.0 ORDER BY "MinTemp" DESC"#; diff --git a/tests/distributed_aggregation.rs b/tests/distributed_aggregation.rs index 8ce80cda..e7ba1210 100644 --- a/tests/distributed_aggregation.rs +++ b/tests/distributed_aggregation.rs @@ -9,7 +9,9 @@ mod tests { use datafusion_distributed::test_utils::localhost::start_localhost_context; use datafusion_distributed::test_utils::parquet::register_parquet_tables; use datafusion_distributed::test_utils::session_context::register_temp_parquet_table; - use datafusion_distributed::{DefaultSessionBuilder, assert_snapshot, display_plan_ascii}; + use datafusion_distributed::{ + DefaultSessionBuilder, DistributedExt, assert_snapshot, display_plan_ascii, + }; use futures::TryStreamExt; use std::error::Error; use std::sync::Arc; @@ -17,7 +19,9 @@ mod tests { #[tokio::test] async fn distributed_aggregation() -> Result<(), Box> { - let (ctx_distributed, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + let (mut ctx_distributed, _guard, _) = + start_localhost_context(3, DefaultSessionBuilder).await; + ctx_distributed.set_distributed_bytes_processed_per_partition(1000)?; let query = r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*)"#; @@ -52,16 +56,16 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ 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))] │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1 + ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] + │ RepartitionExec: partitioning=Hash([RainToday@0], 9), input_partitions=1 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet @@ -103,7 +107,9 @@ mod tests { #[tokio::test] async fn distributed_aggregation_head_node_partitioned() -> Result<(), Box> { - let (ctx_distributed, _guard, _) = start_localhost_context(6, DefaultSessionBuilder).await; + let (mut ctx_distributed, _guard, _) = + start_localhost_context(6, DefaultSessionBuilder).await; + ctx_distributed.set_distributed_bytes_processed_per_partition(1000)?; let query = r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday""#; @@ -133,17 +139,17 @@ mod tests { @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=18, input_tasks=6 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..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))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=6 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1 + ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] t4:[p0..p17] t5:[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] + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] t3:[__,__,__] t4:[__,__,__] t5:[__,__,__] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── ", @@ -206,6 +212,9 @@ mod tests { datafusion::prelude::ParquetReadOptions::default(), ) .await?; + let (mut ctx_distributed, _guard, _) = + start_localhost_context(6, DefaultSessionBuilder).await; + ctx_distributed.set_distributed_bytes_processed_per_partition(1000)?; let query = r#"SELECT group_id, first_value(trace_id) AS fv1, first_value(value) AS fv2 FROM records_partitioned diff --git a/tests/distributed_unions.rs b/tests/distributed_unions.rs index db90999b..da689312 100644 --- a/tests/distributed_unions.rs +++ b/tests/distributed_unions.rs @@ -6,14 +6,18 @@ mod tests { use datafusion::prelude::SessionContext; use datafusion_distributed::test_utils::localhost::start_localhost_context; use datafusion_distributed::test_utils::parquet::register_parquet_tables; - use datafusion_distributed::{DefaultSessionBuilder, assert_snapshot, display_plan_ascii}; + use datafusion_distributed::{ + DefaultSessionBuilder, DistributedExt, assert_snapshot, display_plan_ascii, + }; use futures::TryStreamExt; use std::error::Error; use std::sync::Arc; #[tokio::test] async fn more_tasks_than_children() -> Result<(), Box> { - let (ctx_distributed, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + let (mut ctx_distributed, _guard, _) = + start_localhost_context(3, DefaultSessionBuilder).await; + ctx_distributed.set_distributed_bytes_processed_per_partition(1000)?; let query = r#" SELECT "MinTemp", "RainToday" FROM weather WHERE "MinTemp" > 10.0 @@ -61,7 +65,9 @@ mod tests { #[tokio::test] async fn same_children_than_tasks() -> Result<(), Box> { - let (ctx_distributed, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + let (mut ctx_distributed, _guard, _) = + start_localhost_context(3, DefaultSessionBuilder).await; + ctx_distributed.set_distributed_bytes_processed_per_partition(1000)?; let query = r#" SELECT "MinTemp", "RainToday" FROM weather WHERE "MinTemp" > 20.0 @@ -114,7 +120,9 @@ mod tests { #[tokio::test] async fn more_children_than_tasks() -> Result<(), Box> { - let (ctx_distributed, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + let (mut ctx_distributed, _guard, _) = + start_localhost_context(3, DefaultSessionBuilder).await; + ctx_distributed.set_distributed_bytes_processed_per_partition(1000)?; let query = r#" SELECT "MinTemp", "RainToday" FROM weather WHERE "MinTemp" > 10.0 diff --git a/tests/error_propagation.rs b/tests/error_propagation.rs index 52dc5c43..b850c35e 100644 --- a/tests/error_propagation.rs +++ b/tests/error_propagation.rs @@ -32,6 +32,7 @@ mod tests { } let (mut ctx, _guard, _) = start_localhost_context(3, build_state).await; + ctx.set_distributed_bytes_processed_per_partition(1000)?; ctx.set_distributed_user_codec(ErrorThrowingExecCodec); let query = r#"SELECT "MinTemp" FROM weather WHERE "MinTemp" > 20.0"#; diff --git a/tests/join.rs b/tests/join.rs index a72c185f..511fe56e 100644 --- a/tests/join.rs +++ b/tests/join.rs @@ -11,7 +11,7 @@ mod tests { prelude::{ParquetReadOptions, SessionContext, col}, }; use datafusion_distributed::{ - DefaultSessionBuilder, assert_snapshot, display_plan_ascii, + DefaultSessionBuilder, DistributedExt, assert_snapshot, display_plan_ascii, test_utils::localhost::start_localhost_context, }; @@ -43,6 +43,8 @@ mod tests { .options_mut() .optimizer .hash_join_single_partition_threshold_rows = 0; + ctx.set_distributed_bytes_processed_per_partition(10) + .unwrap(); } async fn register_tables(ctx: &SessionContext) -> Result<()> { @@ -264,24 +266,27 @@ mod tests { assert_snapshot!(&distributed_plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST] - │ SortExec: expr=[env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[env@0 as env, time_bin@1 as time_bin, avg(a.max_bin_value)@2 as avg_max_value] - │ AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p0..p3] - │ RepartitionExec: partitioning=Hash([env@0, time_bin@1], 4), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] - │ ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] - │ AggregateExec: mode=SinglePartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) - │ ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] - │ FilterExec: service@1 = log, projection=[env@0, d_dkey@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/dim/d_dkey=A/data0.parquet], [/testdata/join/parquet/dim/d_dkey=B/data0.parquet], [/testdata/join/parquet/dim/d_dkey=C/data0.parquet], [/testdata/join/parquet/dim/d_dkey=D/data0.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/fact/f_dkey=A/data0.parquet], [/testdata/join/parquet/fact/f_dkey=B/data0.parquet], [/testdata/join/parquet/fact/f_dkey=C/data0.parquet], [/testdata/join/parquet/fact/f_dkey=D/data0.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] + │ SortExec: expr=[env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[env@0 as env, time_bin@1 as time_bin, avg(a.max_bin_value)@2 as avg_max_value] + │ AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] + │ RepartitionExec: partitioning=Hash([env@0, time_bin@1], 8), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] + │ ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] + │ AggregateExec: mode=SinglePartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) + │ ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] + │ FilterExec: service@1 = log, projection=[env@0, d_dkey@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/dim/d_dkey=A/data0.parquet], [/testdata/join/parquet/dim/d_dkey=B/data0.parquet], [/testdata/join/parquet/dim/d_dkey=C/data0.parquet], [/testdata/join/parquet/dim/d_dkey=D/data0.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/fact/f_dkey=A/data0.parquet], [/testdata/join/parquet/fact/f_dkey=B/data0.parquet], [/testdata/join/parquet/fact/f_dkey=C/data0.parquet], [/testdata/join/parquet/fact/f_dkey=D/data0.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── "#); // Ensure distributed results are correct. diff --git a/tests/metrics_collection.rs b/tests/metrics_collection.rs index d7694539..3667c3f6 100644 --- a/tests/metrics_collection.rs +++ b/tests/metrics_collection.rs @@ -9,8 +9,8 @@ mod tests { use datafusion_distributed::test_utils::localhost::start_localhost_context; use datafusion_distributed::test_utils::parquet::register_parquet_tables; use datafusion_distributed::{ - DefaultSessionBuilder, DistributedMetricsFormat, NetworkCoalesceExec, NetworkShuffleExec, - display_plan_ascii, rewrite_distributed_plan_with_metrics, + DefaultSessionBuilder, DistributedExt, DistributedMetricsFormat, NetworkCoalesceExec, + NetworkShuffleExec, display_plan_ascii, rewrite_distributed_plan_with_metrics, }; use futures::TryStreamExt; use std::sync::Arc; @@ -22,7 +22,8 @@ mod tests { async fn test_metrics_collection_in_aggregation( format: DistributedMetricsFormat, ) -> Result<(), Box> { - let (d_ctx, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + let (mut d_ctx, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + d_ctx.set_distributed_bytes_processed_per_partition(1000)?; let query = r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*)"#; @@ -49,7 +50,8 @@ mod tests { async fn test_metrics_collection_in_join( format: DistributedMetricsFormat, ) -> Result<(), Box> { - let (d_ctx, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + let (mut d_ctx, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + d_ctx.set_distributed_bytes_processed_per_partition(1000)?; let query = r#" WITH a AS ( @@ -99,7 +101,8 @@ mod tests { async fn test_metrics_collection_in_union( format: DistributedMetricsFormat, ) -> Result<(), Box> { - let (d_ctx, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + let (mut d_ctx, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + d_ctx.set_distributed_bytes_processed_per_partition(1000)?; let query = r#" SELECT "MinTemp", "RainToday" FROM weather WHERE "MinTemp" > 10.0 @@ -137,7 +140,8 @@ mod tests { async fn test_metric_collection_network_boundaries( format: DistributedMetricsFormat, ) -> Result<(), Box> { - let (d_ctx, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + let (mut d_ctx, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await; + d_ctx.set_distributed_bytes_processed_per_partition(1000)?; let query = r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*)"#; diff --git a/tests/stateful_data_cleanup.rs b/tests/stateful_data_cleanup.rs index e294e4c3..5eee21c4 100644 --- a/tests/stateful_data_cleanup.rs +++ b/tests/stateful_data_cleanup.rs @@ -4,9 +4,9 @@ mod tests { use datafusion::error::Result; use datafusion::physical_plan::execute_stream; use datafusion::prelude::SessionContext; + use datafusion_distributed::DefaultSessionBuilder; use datafusion_distributed::test_utils::localhost::start_localhost_context; use datafusion_distributed::test_utils::{benchmarks_common, tpch}; - use datafusion_distributed::{DefaultSessionBuilder, DistributedExt}; use futures::TryStreamExt; use std::fs; use std::path::Path; @@ -17,7 +17,6 @@ mod tests { const NUM_WORKERS: usize = 4; const TPCH_SCALE_FACTOR: f64 = 1.0; const TPCH_DATA_PARTS: i32 = 16; - const CARDINALITY_TASK_COUNT_FACTOR: f64 = 1.0; #[tokio::test] async fn no_pending_tasks_if_query_completes() -> Result<()> { @@ -69,9 +68,6 @@ mod tests { let query_sql = tpch::get_query(query_id)?; - let d_ctx = d_ctx - .with_distributed_cardinality_effect_task_scale_factor(CARDINALITY_TASK_COUNT_FACTOR)?; - benchmarks_common::register_tables(&d_ctx, &data_dir).await?; let df = d_ctx.sql(&query_sql).await?; diff --git a/tests/stateful_execution_plan.rs b/tests/stateful_execution_plan.rs index e93763a0..8bd9b023 100644 --- a/tests/stateful_execution_plan.rs +++ b/tests/stateful_execution_plan.rs @@ -43,6 +43,7 @@ mod tests { } let (mut ctx_distributed, _guard, _) = start_localhost_context(3, build_state).await; + ctx_distributed.set_distributed_bytes_processed_per_partition(1000)?; ctx_distributed.set_distributed_user_codec(PassThroughExecCodec); register_parquet_tables(&ctx_distributed).await?; diff --git a/tests/tpcds_correctness_test.rs b/tests/tpcds_correctness_test.rs index 2bf8ecc5..510ea91c 100644 --- a/tests/tpcds_correctness_test.rs +++ b/tests/tpcds_correctness_test.rs @@ -19,8 +19,7 @@ mod tests { use tokio::sync::OnceCell; const NUM_WORKERS: usize = 4; - const FILES_PER_TASK: usize = 2; - const CARDINALITY_TASK_COUNT_FACTOR: f64 = 2.0; + const BYTES_PROCESSED_PER_PARTITION: usize = 1024 * 1024; const SF: f64 = 1.0; const PARQUET_PARTITIONS: usize = 4; @@ -229,6 +228,7 @@ mod tests { } #[tokio::test(flavor = "multi_thread")] + #[ignore = "Query q41 did not get distributed"] async fn test_tpcds_41() -> Result<()> { test_tpcds_query("q41").await } @@ -264,7 +264,6 @@ mod tests { } #[tokio::test(flavor = "multi_thread")] - #[ignore = "Query q48 did not get distributed"] async fn test_tpcds_48() -> Result<()> { test_tpcds_query("q48").await } @@ -337,7 +336,6 @@ mod tests { } #[tokio::test(flavor = "multi_thread")] - #[ignore = "Query q62 did not get distributed"] async fn test_tpcds_62() -> Result<()> { test_tpcds_query("q62").await } @@ -487,7 +485,6 @@ mod tests { } #[tokio::test(flavor = "multi_thread")] - #[ignore = "Query q91 did not get distributed"] async fn test_tpcds_91() -> Result<()> { test_tpcds_query("q91").await } @@ -569,8 +566,7 @@ mod tests { // Make distributed localhost context to run queries let (d_ctx, _guard, _) = start_localhost_context(NUM_WORKERS, DefaultSessionBuilder).await; let d_ctx = d_ctx - .with_distributed_files_per_task(FILES_PER_TASK)? - .with_distributed_cardinality_effect_task_scale_factor(CARDINALITY_TASK_COUNT_FACTOR)? + .with_distributed_bytes_processed_per_partition(BYTES_PROCESSED_PER_PARTITION)? .with_distributed_broadcast_joins(true)?; benchmarks_common::register_tables(&s_ctx, &data_dir).await?; diff --git a/tests/tpcds_plans_test.rs b/tests/tpcds_plans_test.rs index a59892bb..5268aa6a 100644 --- a/tests/tpcds_plans_test.rs +++ b/tests/tpcds_plans_test.rs @@ -12,8 +12,7 @@ mod tests { use tokio::sync::OnceCell; const NUM_WORKERS: usize = 4; - const FILES_PER_TASK: usize = 2; - const CARDINALITY_TASK_COUNT_FACTOR: f64 = 2.0; + const BYTES_PROCESSED_PER_PARTITION: usize = 1024 * 1024; const SF: f64 = 1.0; const PARQUET_PARTITIONS: usize = 4; @@ -23,84 +22,81 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_customer_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[c_customer_id@0 ASC NULLS LAST], preserve_partitioning=[true] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_store_sk@0, ctr_store_sk@1)], filter=CAST(ctr_total_return@0 AS Decimal128(30, 15)) > avg(ctr2.ctr_total_return) * Float64(1.2)@1, projection=[c_customer_id@2] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[CAST(CAST(avg(ctr2.ctr_total_return)@1 AS Float64) * 1.2 AS Decimal128(30, 15)) as avg(ctr2.ctr_total_return) * Float64(1.2), ctr_store_sk@0 as ctr_store_sk] │ AggregateExec: mode=FinalPartitioned, gby=[ctr_store_sk@0 as ctr_store_sk], aggr=[avg(ctr2.ctr_total_return)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ctr_store_sk@1, ctr_total_return@2, c_customer_id@4] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ctr_store_sk@1)], projection=[ctr_customer_sk@2, ctr_store_sk@3, ctr_total_return@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[sr_customer_sk@0 as ctr_customer_sk, sr_store_sk@1 as ctr_store_sk, sum(store_returns.sr_return_amt)@2 as ctr_total_return] │ AggregateExec: mode=FinalPartitioned, gby=[sr_customer_sk@0 as sr_customer_sk, sr_store_sk@1 as sr_store_sk], aggr=[sum(store_returns.sr_return_amt)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([sr_customer_sk@0, sr_store_sk@1], 6), input_partitions=2 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@0, sr_store_sk@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[sr_customer_sk@0 as sr_customer_sk, sr_store_sk@1 as sr_store_sk], aggr=[sum(store_returns.sr_return_amt)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sr_returned_date_sk@0)], projection=[sr_customer_sk@3, sr_store_sk@4, sr_return_amt@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_customer_sk, sr_store_sk, sr_return_amt], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ RepartitionExec: partitioning=Hash([ctr_store_sk@0], 6), input_partitions=3 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ctr_store_sk@0], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ctr_store_sk@0 as ctr_store_sk], aggr=[avg(ctr2.ctr_total_return)] │ ProjectionExec: expr=[sr_store_sk@1 as ctr_store_sk, sum(store_returns.sr_return_amt)@2 as ctr_total_return] │ AggregateExec: mode=FinalPartitioned, gby=[sr_customer_sk@0 as sr_customer_sk, sr_store_sk@1 as sr_store_sk], aggr=[sum(store_returns.sr_return_amt)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([sr_customer_sk@0, sr_store_sk@1], 6), input_partitions=2 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@0, sr_store_sk@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[sr_customer_sk@0 as sr_customer_sk, sr_store_sk@1 as sr_store_sk], aggr=[sum(store_returns.sr_return_amt)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sr_returned_date_sk@0)], projection=[sr_customer_sk@3, sr_store_sk@4, sr_return_amt@5] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_customer_sk, sr_store_sk, sr_return_amt], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── "); Ok(()) @@ -112,85 +108,81 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [d_week_seq1@0 ASC] - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: expr=[d_week_seq1@0 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[d_week_seq1@0 as d_week_seq1, round(sun_sales1@1 / sun_sales2@8, 2) as r1, round(mon_sales1@2 / mon_sales2@9, 2) as r2, round(tue_sales1@3 / tue_sales2@10, 2) as r3, round(wed_sales1@4 / wed_sales2@11, 2) as r4, round(thu_sales1@5 / thu_sales2@12, 2) as r5, round(fri_sales1@6 / fri_sales2@13, 2) as r6, round(sat_sales1@7 / sat_sales2@14, 2) as round(y.sat_sales1 / z.sat_sales2,Int64(2))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(y.d_week_seq1 AS Int64)@8, z.d_week_seq2 - Int64(53)@8)], projection=[d_week_seq1@0, sun_sales1@1, mon_sales1@2, tue_sales1@3, wed_sales1@4, thu_sales1@5, fri_sales1@6, sat_sales1@7, sun_sales2@10, mon_sales2@11, tue_sales2@12, wed_sales2@13, thu_sales2@14, fri_sales2@15, sat_sales2@16] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq2, sun_sales@1 as sun_sales2, mon_sales@2 as mon_sales2, tue_sales@3 as tue_sales2, wed_sales@4 as wed_sales2, thu_sales@5 as thu_sales2, fri_sales@6 as fri_sales2, sat_sales@7 as sat_sales2, CAST(d_week_seq@0 AS Int64) - 53 as z.d_week_seq2 - Int64(53)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@0)], projection=[d_week_seq@1, sun_sales@2, mon_sales@3, tue_sales@4, wed_sales@5, thu_sales@6, fri_sales@7, sat_sales@8] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END)@1 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END)@2 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END)@3 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END)@4 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END)@5 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END)@6 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)@7 as sat_sales] │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq1, sun_sales@1 as sun_sales1, mon_sales@2 as mon_sales1, tue_sales@3 as tue_sales1, wed_sales@4 as wed_sales1, thu_sales@5 as thu_sales1, fri_sales@6 as fri_sales1, sat_sales@7 as sat_sales1, CAST(d_week_seq@0 AS Int64) as CAST(y.d_week_seq1 AS Int64)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@0)], projection=[d_week_seq@1, sun_sales@2, mon_sales@3, tue_sales@4, wed_sales@5, thu_sales@6, fri_sales@7, sat_sales@8] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END)@1 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END)@2 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END)@3 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END)@4 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END)@5 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END)@6 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)@7 as sat_sales] │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)] │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: d_year@1 = 2001, projection=[d_week_seq@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_week_seq, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_week_seq, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_week_seq@0], 6), input_partitions=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([d_week_seq@0], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_week_seq@1 as d_week_seq], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)] │ ProjectionExec: expr=[sales_price@2 as sales_price, d_week_seq@0 as d_week_seq, d_day_name@1 as d_day_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, sales_price@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk@0 as sold_date_sk, ws_ext_sales_price@23 as sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk@0 as sold_date_sk, cs_ext_sales_price@23 as sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + ┌───── Stage 2 ── Tasks: t0:[p0..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: d_year@1 = 2002, projection=[d_week_seq@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_week_seq, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_week_seq, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_week_seq@0], 6), input_partitions=3 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([d_week_seq@0], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_week_seq@1 as d_week_seq], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)] │ ProjectionExec: expr=[sales_price@2 as sales_price, d_week_seq@0 as d_week_seq, d_day_name@1 as d_day_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, sales_price@5] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk@0 as sold_date_sk, ws_ext_sales_price@23 as sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk@0 as sold_date_sk, cs_ext_sales_price@23 as sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] + ┌───── Stage 6 ── Tasks: t0:[p0..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -201,40 +193,39 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [d_year@0 ASC NULLS LAST, sum_agg@3 DESC, brand_id@1 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[d_year@0 ASC NULLS LAST, sum_agg@3 DESC, brand_id@1 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[d_year@0 as d_year, i_brand_id@2 as brand_id, i_brand@1 as brand, sum(store_sales.ss_ext_sales_price)@3 as sum_agg] │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand@1 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_year@0, i_brand@1, i_brand_id@2], 6), input_partitions=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([d_year@0, i_brand@1, i_brand_id@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand@3 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@1, i_item_sk@0)], projection=[d_year@0, ss_ext_sales_price@2, i_brand_id@4, i_brand@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ FilterExec: i_manufact_id@3 = 128, projection=[i_item_sk@0, i_brand_id@1, i_brand@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_brand, i_manufact_id], file_type=parquet, predicate=i_manufact_id@13 = 128 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 128 AND 128 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (128)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(dt.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ext_sales_price@5] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(dt.d_date_sk AS Float64)] │ FilterExec: d_moy@2 = 11, projection=[d_date_sk@0, d_year@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1, required_guarantees=[d_moy in (11)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1, required_guarantees=[d_moy in (11)] └────────────────────────────────────────────────── "); Ok(()) @@ -245,233 +236,227 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], fetch=100 - │ [Stage 24] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 24] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 24 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], preserve_partitioning=[true] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@0 > Some(0),24,6 THEN year_total@1 / year_total@0 END > CASE WHEN year_total@2 > Some(0),24,6 THEN year_total@3 / year_total@2 END, projection=[customer_id@1, customer_first_name@2, customer_last_name@3, customer_preferred_cust_flag@4] │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 20 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[customer_id@1 as customer_id, customer_id@2 as customer_id, customer_first_name@3 as customer_first_name, customer_last_name@4 as customer_last_name, customer_preferred_cust_flag@5 as customer_preferred_cust_flag, year_total@6 as year_total, year_total@7 as year_total, year_total@0 as year_total] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, customer_id@3, customer_first_name@4, customer_last_name@5, customer_preferred_cust_flag@6, year_total@7, year_total@8] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),24,6 THEN year_total@3 / year_total@2 END > CASE WHEN year_total@0 > Some(0),24,6 THEN year_total@1 / year_total@0 END, projection=[customer_id@0, customer_id@2, customer_first_name@3, customer_last_name@4, customer_preferred_cust_flag@5, year_total@7, year_total@9] │ CoalescePartitionsExec - │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 19] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 19] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))@1 as year_total] │ FilterExec: sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))@1 > Some(0),24,6 │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))@8 as sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@11 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ws_ext_discount_amt@8 as ws_ext_discount_amt, ws_ext_sales_price@9 as ws_ext_sales_price, ws_ext_wholesale_cost@10 as ws_ext_wholesale_cost, ws_ext_list_price@11 as ws_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_ext_discount_amt, ws_ext_sales_price, ws_ext_wholesale_cost, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 16 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[customer_id@1 as customer_id, year_total@2 as year_total, customer_id@3 as customer_id, customer_first_name@4 as customer_first_name, customer_last_name@5 as customer_last_name, customer_preferred_cust_flag@6 as customer_preferred_cust_flag, year_total@7 as year_total, year_total@0 as year_total] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, year_total@3, customer_id@4, customer_first_name@5, customer_last_name@6, customer_preferred_cust_flag@7, year_total@8] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, c_preferred_cust_flag@3 as customer_preferred_cust_flag, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@1 as year_total] │ FilterExec: sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@1 > Some(0),24,6 │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@8 as sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@11 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, cs_ext_discount_amt@8 as cs_ext_discount_amt, cs_ext_sales_price@9 as cs_ext_sales_price, cs_ext_wholesale_cost@10 as cs_ext_wholesale_cost, cs_ext_list_price@11 as cs_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, cs_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, cs_sold_date_sk@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_ext_discount_amt, cs_ext_sales_price, cs_ext_wholesale_cost, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 12 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@1 as year_total] │ FilterExec: sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@1 > Some(0),24,6 │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@8 as sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 11 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@11 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ss_ext_discount_amt@8 as ss_ext_discount_amt, ss_ext_sales_price@9 as ss_ext_sales_price, ss_ext_wholesale_cost@10 as ss_ext_wholesale_cost, ss_ext_list_price@11 as ss_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_discount_amt, ss_ext_sales_price, ss_ext_wholesale_cost, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 15 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@11 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ss_ext_discount_amt@8 as ss_ext_discount_amt, ss_ext_sales_price@9 as ss_ext_sales_price, ss_ext_wholesale_cost@10 as ss_ext_wholesale_cost, ss_ext_list_price@11 as ss_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_discount_amt, ss_ext_sales_price, ss_ext_wholesale_cost, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 13 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2002 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 14 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 19 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@11 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, cs_ext_discount_amt@8 as cs_ext_discount_amt, cs_ext_sales_price@9 as cs_ext_sales_price, cs_ext_wholesale_cost@10 as cs_ext_wholesale_cost, cs_ext_list_price@11 as cs_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, cs_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, cs_sold_date_sk@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_ext_discount_amt, cs_ext_sales_price, cs_ext_wholesale_cost, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 17 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2002 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 18 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 23 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 23 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@11 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ws_ext_discount_amt@8 as ws_ext_discount_amt, ws_ext_sales_price@9 as ws_ext_sales_price, ws_ext_wholesale_cost@10 as ws_ext_wholesale_cost, ws_ext_list_price@11 as ws_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] │ CoalescePartitionsExec - │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_ext_discount_amt, ws_ext_sales_price, ws_ext_wholesale_cost, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 21 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2002 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 22 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── "); @@ -483,126 +468,121 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [channel@0 ASC, id@1 ASC], fetch=100 - │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, id@1 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[channel@0 as channel, id@1 as id, sum(x.sales)@3 as sales, sum(x.returns_)@4 as returns_, sum(x.profit)@5 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, id@1 as id, __grouping_id@2 as __grouping_id], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 13] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] - │ RepartitionExec: partitioning=Hash([channel@0, id@1, __grouping_id@2], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[(NULL as channel, NULL as id), (channel@0 as channel, NULL as id), (channel@0 as channel, id@1 as id)], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] - │ ProjectionExec: expr=[store channel as channel, concat(store, s_store_id@0) as id, sum(salesreturns.sales_price)@1 as sales, sum(salesreturns.return_amt)@3 as returns_, sum(salesreturns.profit)@2 - sum(salesreturns.net_loss)@4 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[s_store_id@0 as s_store_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 - │ ProjectionExec: expr=[catalog channel as channel, concat(catalog_page, cp_catalog_page_id@0) as id, sum(salesreturns.sales_price)@1 as sales, sum(salesreturns.return_amt)@3 as returns_, sum(salesreturns.profit)@2 - sum(salesreturns.net_loss)@4 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[cp_catalog_page_id@0 as cp_catalog_page_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 - │ ProjectionExec: expr=[web channel as channel, concat(web_site, web_site_id@0) as id, sum(salesreturns.sales_price)@1 as sales, sum(salesreturns.return_amt)@3 as returns_, sum(salesreturns.profit)@2 - sum(salesreturns.net_loss)@4 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[web_site_id@0 as web_site_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] - │ RepartitionExec: partitioning=Hash([s_store_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[s_store_id@4 as s_store_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] - │ ProjectionExec: expr=[sales_price@1 as sales_price, profit@2 as profit, return_amt@3 as return_amt, net_loss@4 as net_loss, s_store_id@0 as s_store_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, store_sk@0)], projection=[s_store_id@1, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, date_sk@1)], projection=[store_sk@2, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 - │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_store_sk@7 as store_sk, ss_sold_date_sk@0 as date_sk, ss_ext_sales_price@15 as sales_price, CAST(ss_net_profit@22 AS Decimal128(7, 2)) as profit, Some(0),7,2 as return_amt, Some(0),7,2 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_store_sk@7 as store_sk, sr_returned_date_sk@0 as date_sk, Some(0),7,2 as sales_price, Some(0),7,2 as profit, sr_return_amt@11 as return_amt, CAST(sr_net_loss@19 AS Decimal128(7, 2)) as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p8..p15] - │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-06, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] - │ RepartitionExec: partitioning=Hash([cp_catalog_page_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[cp_catalog_page_id@4 as cp_catalog_page_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] - │ ProjectionExec: expr=[sales_price@1 as sales_price, profit@2 as profit, return_amt@3 as return_amt, net_loss@4 as net_loss, cp_catalog_page_id@0 as cp_catalog_page_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(catalog_page.cp_catalog_page_sk AS Float64)@2, page_sk@0)], projection=[cp_catalog_page_id@1, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, date_sk@1)], projection=[page_sk@2, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 - │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_catalog_page_sk@12 as page_sk, cs_sold_date_sk@0 as date_sk, cs_ext_sales_price@23 as sales_price, cs_net_profit@33 as profit, Some(0),7,2 as return_amt, Some(0),7,2 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_catalog_page_sk@12 as page_sk, CAST(cr_returned_date_sk@0 AS Float64) as date_sk, Some(0),7,2 as sales_price, Some(0),7,2 as profit, cr_return_amount@18 as return_amt, cr_net_loss@26 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ ProjectionExec: expr=[cp_catalog_page_sk@0 as cp_catalog_page_sk, cp_catalog_page_id@1 as cp_catalog_page_id, CAST(cp_catalog_page_sk@0 AS Float64) as CAST(catalog_page.cp_catalog_page_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-3.parquet]]}, projection=[cp_catalog_page_sk, cp_catalog_page_id], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-06, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([web_site_id@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[web_site_id@4 as web_site_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] - │ ProjectionExec: expr=[sales_price@1 as sales_price, profit@2 as profit, return_amt@3 as return_amt, net_loss@4 as net_loss, web_site_id@0 as web_site_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@2, wsr_web_site_sk@0)], projection=[web_site_id@1, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, date_sk@1)], projection=[wsr_web_site_sk@2, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 - │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_web_site_sk@13 as wsr_web_site_sk, ws_sold_date_sk@0 as date_sk, ws_ext_sales_price@23 as sales_price, ws_net_profit@33 as profit, Some(0),7,2 as return_amt, Some(0),7,2 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ ProjectionExec: expr=[ws_web_site_sk@3 as wsr_web_site_sk, wr_returned_date_sk@0 as date_sk, Some(0),7,2 as sales_price, Some(0),7,2 as profit, wr_return_amt@1 as return_amt, wr_net_loss@2 as net_loss] - │ HashJoinExec: mode=Partitioned, join_type=Left, on=[(wr_item_sk@1, ws_item_sk@0), (wr_order_number@2, ws_order_number@2)], projection=[wr_returned_date_sk@0, wr_return_amt@3, wr_net_loss@4, ws_web_site_sk@6] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p7] t1:[p8..p15] - │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_site_id, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-06, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ RepartitionExec: partitioning=Hash([wr_item_sk@1, wr_order_number@2], 6), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_returned_date_sk, wr_item_sk, wr_order_number, wr_return_amt, wr_net_loss], file_type=parquet + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, id@1 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[channel@0 as channel, id@1 as id, sum(x.sales)@3 as sales, sum(x.returns_)@4 as returns_, sum(x.profit)@5 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, id@1 as id, __grouping_id@2 as __grouping_id], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([channel@0, id@1, __grouping_id@2], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[(NULL as channel, NULL as id), (channel@0 as channel, NULL as id), (channel@0 as channel, id@1 as id)], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] + │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] t3:[c2] + │ ProjectionExec: expr=[store channel as channel, concat(store, s_store_id@0) as id, sum(salesreturns.sales_price)@1 as sales, sum(salesreturns.return_amt)@3 as returns_, sum(salesreturns.profit)@2 - sum(salesreturns.net_loss)@4 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[s_store_id@0 as s_store_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[catalog channel as channel, concat(catalog_page, cp_catalog_page_id@0) as id, sum(salesreturns.sales_price)@1 as sales, sum(salesreturns.return_amt)@3 as returns_, sum(salesreturns.profit)@2 - sum(salesreturns.net_loss)@4 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[cp_catalog_page_id@0 as cp_catalog_page_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[web channel as channel, concat(web_site, web_site_id@0) as id, sum(salesreturns.sales_price)@1 as sales, sum(salesreturns.return_amt)@3 as returns_, sum(salesreturns.profit)@2 - sum(salesreturns.net_loss)@4 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[web_site_id@0 as web_site_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([s_store_id@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[s_store_id@4 as s_store_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] + │ ProjectionExec: expr=[sales_price@1 as sales_price, profit@2 as profit, return_amt@3 as return_amt, net_loss@4 as net_loss, s_store_id@0 as s_store_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, store_sk@0)], projection=[s_store_id@1, sales_price@4, profit@5, return_amt@6, net_loss@7] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, date_sk@1)], projection=[store_sk@2, sales_price@4, profit@5, return_amt@6, net_loss@7] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ DistributedUnionExec: t0:[c0(0/3)] t1:[c0(1/3)] t2:[c0(2/3)] t3:[c1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_store_sk@7 as store_sk, ss_sold_date_sk@0 as date_sk, ss_ext_sales_price@15 as sales_price, CAST(ss_net_profit@22 AS Decimal128(7, 2)) as profit, Some(0),7,2 as return_amt, Some(0),7,2 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-3.parquet:..]]}, projection=[sr_store_sk@7 as store_sk, sr_returned_date_sk@0 as date_sk, Some(0),7,2 as sales_price, Some(0),7,2 as profit, sr_return_amt@11 as return_amt, CAST(sr_net_loss@19 AS Decimal128(7, 2)) as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ws_item_sk@0, ws_order_number@2], 6), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_item_sk, ws_web_site_sk, ws_order_number], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-06, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([cp_catalog_page_id@0], 6), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[cp_catalog_page_id@4 as cp_catalog_page_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] + │ ProjectionExec: expr=[sales_price@1 as sales_price, profit@2 as profit, return_amt@3 as return_amt, net_loss@4 as net_loss, cp_catalog_page_id@0 as cp_catalog_page_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(catalog_page.cp_catalog_page_sk AS Float64)@2, page_sk@0)], projection=[cp_catalog_page_id@1, sales_price@4, profit@5, return_amt@6, net_loss@7] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, date_sk@1)], projection=[page_sk@2, sales_price@4, profit@5, return_amt@6, net_loss@7] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ DistributedUnionExec: t0:[c0(0/3)] t1:[c0(1/3)] t2:[c0(2/3)] t3:[c1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_catalog_page_sk@12 as page_sk, cs_sold_date_sk@0 as date_sk, cs_ext_sales_price@23 as sales_price, cs_net_profit@33 as profit, Some(0),7,2 as return_amt, Some(0),7,2 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_catalog_page_sk@12 as page_sk, CAST(cr_returned_date_sk@0 AS Float64) as date_sk, Some(0),7,2 as sales_price, Some(0),7,2 as profit, cr_return_amount@18 as return_amt, cr_net_loss@26 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[cp_catalog_page_sk@0 as cp_catalog_page_sk, cp_catalog_page_id@1 as cp_catalog_page_id, CAST(cp_catalog_page_sk@0 AS Float64) as CAST(catalog_page.cp_catalog_page_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/catalog_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/catalog_page/part-3.parquet]]}, projection=[cp_catalog_page_sk, cp_catalog_page_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-06, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([web_site_id@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[web_site_id@4 as web_site_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] + │ ProjectionExec: expr=[sales_price@1 as sales_price, profit@2 as profit, return_amt@3 as return_amt, net_loss@4 as net_loss, web_site_id@0 as web_site_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@2, wsr_web_site_sk@0)], projection=[web_site_id@1, sales_price@4, profit@5, return_amt@6, net_loss@7] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, date_sk@1)], projection=[wsr_web_site_sk@2, sales_price@4, profit@5, return_amt@6, net_loss@7] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_web_site_sk@13 as wsr_web_site_sk, ws_sold_date_sk@0 as date_sk, ws_ext_sales_price@23 as sales_price, ws_net_profit@33 as profit, Some(0),7,2 as return_amt, Some(0),7,2 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ ProjectionExec: expr=[ws_web_site_sk@3 as wsr_web_site_sk, wr_returned_date_sk@0 as date_sk, Some(0),7,2 as sales_price, Some(0),7,2 as profit, wr_return_amt@1 as return_amt, wr_net_loss@2 as net_loss] + │ HashJoinExec: mode=Partitioned, join_type=Left, on=[(wr_item_sk@1, ws_item_sk@0), (wr_order_number@2, ws_order_number@2)], projection=[wr_returned_date_sk@0, wr_return_amt@3, wr_net_loss@4, ws_web_site_sk@6] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_site_id, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-06, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ RepartitionExec: partitioning=Hash([wr_item_sk@1, wr_order_number@2], 6), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_returned_date_sk, wr_item_sk, wr_order_number, wr_return_amt, wr_net_loss], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([ws_item_sk@0, ws_order_number@2], 6), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_item_sk, ws_web_site_sk, ws_order_number], file_type=parquet + └────────────────────────────────────────────────── "); Ok(()) } @@ -621,29 +601,32 @@ mod tests { │ FilterExec: CAST(i_current_price@1 AS Decimal128(30, 15)) > CAST(1.2 * avg(j.i_current_price)@2 AS Decimal128(30, 15)), projection=[ca_state@0] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(i_category@2, i_category@1)], projection=[ca_state@0, i_current_price@1, avg(j.i_current_price)@3] │ CoalescePartitionsExec - │ [Stage 7] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 │ ProjectionExec: expr=[CAST(avg(j.i_current_price)@1 AS Float64) as avg(j.i_current_price), i_category@0 as i_category] │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category], aggr=[avg(j.i_current_price)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_category@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category], aggr=[avg(j.i_current_price)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_current_price, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_month_seq@0, d_month_seq@1)], projection=[ca_state@1, i_current_price@3, i_category@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=1 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[ca_state@2 as ca_state, d_month_seq@3 as d_month_seq, i_current_price@0 as i_current_price, i_category@1 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_current_price@1, i_category@2, ca_state@3, d_month_seq@5] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@1, CAST(d.d_date_sk AS Float64)@2)], projection=[ca_state@0, ss_item_sk@2, d_month_seq@4] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_month_seq@1 as d_month_seq, CAST(d_date_sk@0 AS Float64) as CAST(d.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ AggregateExec: mode=FinalPartitioned, gby=[d_month_seq@0 as d_month_seq], aggr=[] │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── @@ -655,41 +638,32 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_month_seq, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_moy in (1), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_category], file_type=parquet + ┌───── Stage 3 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 6 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(c.c_customer_sk AS Float64)@2, ss_customer_sk@2)], projection=[ca_state@0, ss_sold_date_sk@3, ss_item_sk@4] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ca_state@0 as ca_state, c_customer_sk@1 as c_customer_sk, CAST(c_customer_sk@1 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@1)], projection=[ca_state@1, c_customer_sk@2] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] + ┌───── Stage 4 ── Tasks: t0:[p0..p3] │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category], aggr=[avg(j.i_current_price)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_current_price, i_category], file_type=parquet - └────────────────────────────────────────────────── "); Ok(()) } @@ -699,64 +673,59 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, avg(store_sales.ss_quantity)@1 as agg1, avg(store_sales.ss_list_price)@2 as agg2, avg(store_sales.ss_coupon_amt)@3 as agg3, avg(store_sales.ss_sales_price)@4 as agg4] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_list_price), avg(store_sales.ss_coupon_amt), avg(store_sales.ss_sales_price)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, avg(store_sales.ss_quantity)@1 as agg1, avg(store_sales.ss_list_price)@2 as agg2, avg(store_sales.ss_coupon_amt)@3 as agg3, avg(store_sales.ss_sales_price)@4 as agg4] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_list_price), avg(store_sales.ss_coupon_amt), avg(store_sales.ss_sales_price)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@4 as i_item_id], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_list_price), avg(store_sales.ss_coupon_amt), avg(store_sales.ss_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@0)], projection=[ss_quantity@3, ss_list_price@4, ss_sales_price@5, ss_coupon_amt@6, i_item_id@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_promo_sk@1, ss_quantity@2, ss_list_price@3, ss_sales_price@4, ss_coupon_amt@5, i_item_id@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@4 as i_item_id], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_list_price), avg(store_sales.ss_coupon_amt), avg(store_sales.ss_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@0)], projection=[ss_quantity@3, ss_list_price@4, ss_sales_price@5, ss_coupon_amt@6, i_item_id@7] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_promo_sk@1, ss_quantity@2, ss_list_price@3, ss_sales_price@4, ss_coupon_amt@5, i_item_id@7] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_email@1 = N OR p_channel_event@2 = N, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_email, p_channel_event], file_type=parquet, predicate=p_channel_email@9 = N OR p_channel_event@14 = N, pruning_predicate=p_channel_email_null_count@2 != row_count@3 AND p_channel_email_min@0 <= N AND N <= p_channel_email_max@1 OR p_channel_event_null_count@6 != row_count@3 AND p_channel_event_min@4 <= N AND N <= p_channel_event_max@5, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_email@1 = N OR p_channel_event@2 = N, projection=[p_promo_sk@0] + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_promo_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_promo_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_email, p_channel_event], file_type=parquet, predicate=p_channel_email@9 = N OR p_channel_event@14 = N, pruning_predicate=p_channel_email_null_count@2 != row_count@3 AND p_channel_email_min@0 <= N AND N <= p_channel_email_max@1 OR p_channel_event_null_count@6 != row_count@3 AND p_channel_event_min@4 <= N AND N <= p_channel_event_max@5, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_promo_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_promo_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 12), input_partitions=2 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_promo_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_promo_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── "); Ok(()) } @@ -772,63 +741,53 @@ mod tests { │ AggregateExec: mode=Partial, gby=[s_store_name@1 as s_store_name], aggr=[sum(store_sales.ss_net_profit)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(substr(store.s_zip,Int64(1),Int64(2))@3, substr(v1.ca_zip,Int64(1),Int64(2))@1)], projection=[ss_net_profit@0, s_store_name@1] │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ ProjectionExec: expr=[ss_net_profit@2 as ss_net_profit, s_store_name@0 as s_store_name, s_zip@1 as s_zip, substr(s_zip@1, 1, 2) as substr(store.s_zip,Int64(1),Int64(2))] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@0)], projection=[s_store_name@1, s_zip@2, ss_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_store_sk@3, ss_net_profit@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 - │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_sales/part-3.parquet:..]]}, projection=[ss_sold_date_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=4 │ ProjectionExec: expr=[ca_zip@0 as ca_zip, substr(ca_zip@0, 1, 2) as substr(v1.ca_zip,Int64(1),Int64(2))] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ca_zip@0, ca_zip@0)], NullsEqual: true │ CoalescePartitionsExec │ AggregateExec: mode=FinalPartitioned, gby=[ca_zip@0 as ca_zip], aggr=[] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([ca_zip@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[ca_zip@0 as ca_zip], aggr=[] + │ ProjectionExec: expr=[substr(ca_zip@0, 1, 5) as ca_zip] + │ FilterExec: substr(ca_zip@0, 1, 5) IN (SET) ([24128, 76232, 65084, 87816, 83926, 77556, 20548, 26231, 43848, 15126, 91137, 61265, 98294, 25782, 17920, 18426, 98235, 40081, 84093, 28577, 55565, 17183, 54601, 67897, 22752, 86284, 18376, 38607, 45200, 21756, 29741, 96765, 23932, 89360, 29839, 25989, 28898, 91068, 72550, 10390, 18845, 47770, 82636, 41367, 76638, 86198, 81312, 37126, 39192, 88424, 72175, 81426, 53672, 10445, 42666, 66864, 66708, 41248, 48583, 82276, 18842, 78890, 49448, 14089, 38122, 34425, 79077, 19849, 43285, 39861, 66162, 77610, 13695, 99543, 83444, 83041, 12305, 57665, 68341, 25003, 57834, 62878, 49130, 81096, 18840, 27700, 23470, 50412, 21195, 16021, 76107, 71954, 68309, 18119, 98359, 64544, 10336, 86379, 27068, 39736, 98569, 28915, 24206, 56529, 57647, 54917, 42961, 91110, 63981, 14922, 36420, 23006, 67467, 32754, 30903, 20260, 31671, 51798, 72325, 85816, 68621, 13955, 36446, 41766, 68806, 16725, 15146, 22744, 35850, 88086, 51649, 18270, 52867, 39972, 96976, 63792, 11376, 94898, 13595, 10516, 90225, 58943, 39371, 94945, 28587, 96576, 57855, 28488, 26105, 83933, 25858, 34322, 44438, 73171, 30122, 34102, 22685, 71256, 78451, 54364, 13354, 45375, 40558, 56458, 28286, 45266, 47305, 69399, 83921, 26233, 11101, 15371, 69913, 35942, 15882, 25631, 24610, 44165, 99076, 33786, 70738, 26653, 14328, 72305, 62496, 22152, 10144, 64147, 48425, 14663, 21076, 18799, 30450, 63089, 81019, 68893, 24996, 51200, 51211, 45692, 92712, 70466, 79994, 22437, 25280, 38935, 71791, 73134, 56571, 14060, 19505, 72425, 56575, 74351, 68786, 51650, 20004, 18383, 76614, 11634, 18906, 15765, 41368, 73241, 76698, 78567, 97189, 28545, 76231, 75691, 22246, 51061, 90578, 56691, 68014, 51103, 94167, 57047, 14867, 73520, 15734, 63435, 25733, 35474, 24676, 94627, 53535, 17879, 15559, 53268, 59166, 11928, 59402, 33282, 45721, 43933, 68101, 33515, 36634, 71286, 19736, 58058, 55253, 67473, 41918, 19515, 36495, 19430, 22351, 77191, 91393, 49156, 50298, 87501, 18652, 53179, 18767, 63193, 23968, 65164, 68880, 21286, 72823, 58470, 67301, 13394, 31016, 70372, 67030, 40604, 24317, 45748, 39127, 26065, 77721, 31029, 31880, 60576, 24671, 45549, 13376, 50016, 33123, 19769, 22927, 97789, 46081, 72151, 15723, 46136, 51949, 68100, 96888, 64528, 14171, 79777, 28709, 11489, 25103, 32213, 78668, 22245, 15798, 27156, 37930, 62971, 21337, 51622, 67853, 10567, 38415, 15455, 58263, 42029, 60279, 37125, 56240, 88190, 50308, 26859, 64457, 89091, 82136, 62377, 36233, 63837, 58078, 17043, 30010, 60099, 28810, 98025, 29178, 87343, 73273, 30469, 64034, 39516, 86057, 21309, 90257, 67875, 40162, 11356, 73650, 61810, 72013, 30431, 22461, 19512, 13375, 55307, 30625, 83849, 68908, 26689, 96451, 38193, 46820, 88885, 84935, 69035, 83144, 47537, 56616, 94983, 48033, 69952, 25486, 61547, 27385, 61860, 58048, 56910, 16807, 17871, 35258, 31387, 35458, 35576]) + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_zip], file_type=parquet, predicate=substr(ca_zip@9, 1, 5) IN (SET) ([24128, 76232, 65084, 87816, 83926, 77556, 20548, 26231, 43848, 15126, 91137, 61265, 98294, 25782, 17920, 18426, 98235, 40081, 84093, 28577, 55565, 17183, 54601, 67897, 22752, 86284, 18376, 38607, 45200, 21756, 29741, 96765, 23932, 89360, 29839, 25989, 28898, 91068, 72550, 10390, 18845, 47770, 82636, 41367, 76638, 86198, 81312, 37126, 39192, 88424, 72175, 81426, 53672, 10445, 42666, 66864, 66708, 41248, 48583, 82276, 18842, 78890, 49448, 14089, 38122, 34425, 79077, 19849, 43285, 39861, 66162, 77610, 13695, 99543, 83444, 83041, 12305, 57665, 68341, 25003, 57834, 62878, 49130, 81096, 18840, 27700, 23470, 50412, 21195, 16021, 76107, 71954, 68309, 18119, 98359, 64544, 10336, 86379, 27068, 39736, 98569, 28915, 24206, 56529, 57647, 54917, 42961, 91110, 63981, 14922, 36420, 23006, 67467, 32754, 30903, 20260, 31671, 51798, 72325, 85816, 68621, 13955, 36446, 41766, 68806, 16725, 15146, 22744, 35850, 88086, 51649, 18270, 52867, 39972, 96976, 63792, 11376, 94898, 13595, 10516, 90225, 58943, 39371, 94945, 28587, 96576, 57855, 28488, 26105, 83933, 25858, 34322, 44438, 73171, 30122, 34102, 22685, 71256, 78451, 54364, 13354, 45375, 40558, 56458, 28286, 45266, 47305, 69399, 83921, 26233, 11101, 15371, 69913, 35942, 15882, 25631, 24610, 44165, 99076, 33786, 70738, 26653, 14328, 72305, 62496, 22152, 10144, 64147, 48425, 14663, 21076, 18799, 30450, 63089, 81019, 68893, 24996, 51200, 51211, 45692, 92712, 70466, 79994, 22437, 25280, 38935, 71791, 73134, 56571, 14060, 19505, 72425, 56575, 74351, 68786, 51650, 20004, 18383, 76614, 11634, 18906, 15765, 41368, 73241, 76698, 78567, 97189, 28545, 76231, 75691, 22246, 51061, 90578, 56691, 68014, 51103, 94167, 57047, 14867, 73520, 15734, 63435, 25733, 35474, 24676, 94627, 53535, 17879, 15559, 53268, 59166, 11928, 59402, 33282, 45721, 43933, 68101, 33515, 36634, 71286, 19736, 58058, 55253, 67473, 41918, 19515, 36495, 19430, 22351, 77191, 91393, 49156, 50298, 87501, 18652, 53179, 18767, 63193, 23968, 65164, 68880, 21286, 72823, 58470, 67301, 13394, 31016, 70372, 67030, 40604, 24317, 45748, 39127, 26065, 77721, 31029, 31880, 60576, 24671, 45549, 13376, 50016, 33123, 19769, 22927, 97789, 46081, 72151, 15723, 46136, 51949, 68100, 96888, 64528, 14171, 79777, 28709, 11489, 25103, 32213, 78668, 22245, 15798, 27156, 37930, 62971, 21337, 51622, 67853, 10567, 38415, 15455, 58263, 42029, 60279, 37125, 56240, 88190, 50308, 26859, 64457, 89091, 82136, 62377, 36233, 63837, 58078, 17043, 30010, 60099, 28810, 98025, 29178, 87343, 73273, 30469, 64034, 39516, 86057, 21309, 90257, 67875, 40162, 11356, 73650, 61810, 72013, 30431, 22461, 19512, 13375, 55307, 30625, 83849, 68908, 26689, 96451, 38193, 46820, 88885, 84935, 69035, 83144, 47537, 56616, 94983, 48033, 69952, 25486, 61547, 27385, 61860, 58048, 56910, 16807, 17871, 35258, 31387, 35458, 35576]) │ ProjectionExec: expr=[substr(ca_zip@0, 1, 5) as ca_zip] │ FilterExec: count(Int64(1))@1 > 10, projection=[ca_zip@0] │ AggregateExec: mode=FinalPartitioned, gby=[ca_zip@0 as ca_zip], aggr=[count(Int64(1))] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([ca_zip@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[ca_zip@0 as ca_zip], aggr=[count(Int64(1))] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@0, ca_address_sk@0)], projection=[ca_zip@2] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ FilterExec: c_preferred_cust_flag@1 = Y, projection=[c_current_addr_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_current_addr_sk, c_preferred_cust_flag], file_type=parquet, predicate=c_preferred_cust_flag@10 = Y, pruning_predicate=c_preferred_cust_flag_null_count@2 != row_count@3 AND c_preferred_cust_flag_min@0 <= Y AND Y <= c_preferred_cust_flag_max@1, required_guarantees=[c_preferred_cust_flag in (Y)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 1998, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 1998, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([ca_zip@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[ca_zip@0 as ca_zip], aggr=[] - │ ProjectionExec: expr=[substr(ca_zip@0, 1, 5) as ca_zip] - │ FilterExec: substr(ca_zip@0, 1, 5) IN (SET) ([24128, 76232, 65084, 87816, 83926, 77556, 20548, 26231, 43848, 15126, 91137, 61265, 98294, 25782, 17920, 18426, 98235, 40081, 84093, 28577, 55565, 17183, 54601, 67897, 22752, 86284, 18376, 38607, 45200, 21756, 29741, 96765, 23932, 89360, 29839, 25989, 28898, 91068, 72550, 10390, 18845, 47770, 82636, 41367, 76638, 86198, 81312, 37126, 39192, 88424, 72175, 81426, 53672, 10445, 42666, 66864, 66708, 41248, 48583, 82276, 18842, 78890, 49448, 14089, 38122, 34425, 79077, 19849, 43285, 39861, 66162, 77610, 13695, 99543, 83444, 83041, 12305, 57665, 68341, 25003, 57834, 62878, 49130, 81096, 18840, 27700, 23470, 50412, 21195, 16021, 76107, 71954, 68309, 18119, 98359, 64544, 10336, 86379, 27068, 39736, 98569, 28915, 24206, 56529, 57647, 54917, 42961, 91110, 63981, 14922, 36420, 23006, 67467, 32754, 30903, 20260, 31671, 51798, 72325, 85816, 68621, 13955, 36446, 41766, 68806, 16725, 15146, 22744, 35850, 88086, 51649, 18270, 52867, 39972, 96976, 63792, 11376, 94898, 13595, 10516, 90225, 58943, 39371, 94945, 28587, 96576, 57855, 28488, 26105, 83933, 25858, 34322, 44438, 73171, 30122, 34102, 22685, 71256, 78451, 54364, 13354, 45375, 40558, 56458, 28286, 45266, 47305, 69399, 83921, 26233, 11101, 15371, 69913, 35942, 15882, 25631, 24610, 44165, 99076, 33786, 70738, 26653, 14328, 72305, 62496, 22152, 10144, 64147, 48425, 14663, 21076, 18799, 30450, 63089, 81019, 68893, 24996, 51200, 51211, 45692, 92712, 70466, 79994, 22437, 25280, 38935, 71791, 73134, 56571, 14060, 19505, 72425, 56575, 74351, 68786, 51650, 20004, 18383, 76614, 11634, 18906, 15765, 41368, 73241, 76698, 78567, 97189, 28545, 76231, 75691, 22246, 51061, 90578, 56691, 68014, 51103, 94167, 57047, 14867, 73520, 15734, 63435, 25733, 35474, 24676, 94627, 53535, 17879, 15559, 53268, 59166, 11928, 59402, 33282, 45721, 43933, 68101, 33515, 36634, 71286, 19736, 58058, 55253, 67473, 41918, 19515, 36495, 19430, 22351, 77191, 91393, 49156, 50298, 87501, 18652, 53179, 18767, 63193, 23968, 65164, 68880, 21286, 72823, 58470, 67301, 13394, 31016, 70372, 67030, 40604, 24317, 45748, 39127, 26065, 77721, 31029, 31880, 60576, 24671, 45549, 13376, 50016, 33123, 19769, 22927, 97789, 46081, 72151, 15723, 46136, 51949, 68100, 96888, 64528, 14171, 79777, 28709, 11489, 25103, 32213, 78668, 22245, 15798, 27156, 37930, 62971, 21337, 51622, 67853, 10567, 38415, 15455, 58263, 42029, 60279, 37125, 56240, 88190, 50308, 26859, 64457, 89091, 82136, 62377, 36233, 63837, 58078, 17043, 30010, 60099, 28810, 98025, 29178, 87343, 73273, 30469, 64034, 39516, 86057, 21309, 90257, 67875, 40162, 11356, 73650, 61810, 72013, 30431, 22461, 19512, 13375, 55307, 30625, 83849, 68908, 26689, 96451, 38193, 46820, 88885, 84935, 69035, 83144, 47537, 56616, 94983, 48033, 69952, 25486, 61547, 27385, 61860, 58048, 56910, 16807, 17871, 35258, 31387, 35458, 35576]) - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_zip], file_type=parquet, predicate=substr(ca_zip@9, 1, 5) IN (SET) ([24128, 76232, 65084, 87816, 83926, 77556, 20548, 26231, 43848, 15126, 91137, 61265, 98294, 25782, 17920, 18426, 98235, 40081, 84093, 28577, 55565, 17183, 54601, 67897, 22752, 86284, 18376, 38607, 45200, 21756, 29741, 96765, 23932, 89360, 29839, 25989, 28898, 91068, 72550, 10390, 18845, 47770, 82636, 41367, 76638, 86198, 81312, 37126, 39192, 88424, 72175, 81426, 53672, 10445, 42666, 66864, 66708, 41248, 48583, 82276, 18842, 78890, 49448, 14089, 38122, 34425, 79077, 19849, 43285, 39861, 66162, 77610, 13695, 99543, 83444, 83041, 12305, 57665, 68341, 25003, 57834, 62878, 49130, 81096, 18840, 27700, 23470, 50412, 21195, 16021, 76107, 71954, 68309, 18119, 98359, 64544, 10336, 86379, 27068, 39736, 98569, 28915, 24206, 56529, 57647, 54917, 42961, 91110, 63981, 14922, 36420, 23006, 67467, 32754, 30903, 20260, 31671, 51798, 72325, 85816, 68621, 13955, 36446, 41766, 68806, 16725, 15146, 22744, 35850, 88086, 51649, 18270, 52867, 39972, 96976, 63792, 11376, 94898, 13595, 10516, 90225, 58943, 39371, 94945, 28587, 96576, 57855, 28488, 26105, 83933, 25858, 34322, 44438, 73171, 30122, 34102, 22685, 71256, 78451, 54364, 13354, 45375, 40558, 56458, 28286, 45266, 47305, 69399, 83921, 26233, 11101, 15371, 69913, 35942, 15882, 25631, 24610, 44165, 99076, 33786, 70738, 26653, 14328, 72305, 62496, 22152, 10144, 64147, 48425, 14663, 21076, 18799, 30450, 63089, 81019, 68893, 24996, 51200, 51211, 45692, 92712, 70466, 79994, 22437, 25280, 38935, 71791, 73134, 56571, 14060, 19505, 72425, 56575, 74351, 68786, 51650, 20004, 18383, 76614, 11634, 18906, 15765, 41368, 73241, 76698, 78567, 97189, 28545, 76231, 75691, 22246, 51061, 90578, 56691, 68014, 51103, 94167, 57047, 14867, 73520, 15734, 63435, 25733, 35474, 24676, 94627, 53535, 17879, 15559, 53268, 59166, 11928, 59402, 33282, 45721, 43933, 68101, 33515, 36634, 71286, 19736, 58058, 55253, 67473, 41918, 19515, 36495, 19430, 22351, 77191, 91393, 49156, 50298, 87501, 18652, 53179, 18767, 63193, 23968, 65164, 68880, 21286, 72823, 58470, 67301, 13394, 31016, 70372, 67030, 40604, 24317, 45748, 39127, 26065, 77721, 31029, 31880, 60576, 24671, 45549, 13376, 50016, 33123, 19769, 22927, 97789, 46081, 72151, 15723, 46136, 51949, 68100, 96888, 64528, 14171, 79777, 28709, 11489, 25103, 32213, 78668, 22245, 15798, 27156, 37930, 62971, 21337, 51622, 67853, 10567, 38415, 15455, 58263, 42029, 60279, 37125, 56240, 88190, 50308, 26859, 64457, 89091, 82136, 62377, 36233, 63837, 58078, 17043, 30010, 60099, 28810, 98025, 29178, 87343, 73273, 30469, 64034, 39516, 86057, 21309, 90257, 67875, 40162, 11356, 73650, 61810, 72013, 30431, 22461, 19512, 13375, 55307, 30625, 83849, 68908, 26689, 96451, 38193, 46820, 88885, 84935, 69035, 83144, 47537, 56616, 94983, 48033, 69952, 25486, 61547, 27385, 61860, 58048, 56910, 16807, 17871, 35258, 31387, 35458, 35576]) - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([ca_zip@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[ca_zip@0 as ca_zip], aggr=[count(Int64(1))] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@0, ca_address_sk@0)], projection=[ca_zip@2] + │ ProjectionExec: expr=[ss_net_profit@2 as ss_net_profit, s_store_name@0 as s_store_name, s_zip@1 as s_zip, substr(s_zip@1, 1, 2) as substr(store.s_zip,Int64(1),Int64(2))] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@0)], projection=[s_store_name@1, s_zip@2, ss_net_profit@5] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_store_sk@3, ss_net_profit@4] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: c_preferred_cust_flag@1 = Y, projection=[c_current_addr_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_current_addr_sk, c_preferred_cust_flag], file_type=parquet, predicate=c_preferred_cust_flag@10 = Y, pruning_predicate=c_preferred_cust_flag_null_count@2 != row_count@3 AND c_preferred_cust_flag_min@0 <= Y AND Y <= c_preferred_cust_flag_max@1, required_guarantees=[c_preferred_cust_flag in (Y)] + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 1998, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 1998, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (1998)] └────────────────────────────────────────────────── "); Ok(()) @@ -857,158 +816,154 @@ mod tests { │ NestedLoopJoinExec: join_type=Left │ NestedLoopJoinExec: join_type=Left │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[] + │ FilterExec: r_reason_sk@0 = 1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk], file_type=parquet, predicate=r_reason_sk@0 = 1, pruning_predicate=r_reason_sk_null_count@2 != row_count@3 AND r_reason_sk_min@0 <= 1 AND 1 <= r_reason_sk_max@1, required_guarantees=[r_reason_sk in (1)] │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 13] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 14] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 13] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 15] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 14] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 15] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ ProjectionExec: expr=[] - │ FilterExec: r_reason_sk@0 = 1 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk], file_type=parquet, predicate=r_reason_sk@0 = 1, pruning_predicate=r_reason_sk_null_count@2 != row_count@3 AND r_reason_sk_min@0 <= 1 AND 1 <= r_reason_sk_max@1, required_guarantees=[r_reason_sk in (1)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@10 >= 1 AND ss_quantity@10 <= 20, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 1 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 20, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@10 >= 1 AND ss_quantity@10 <= 20, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 1 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 20, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@10 >= 1 AND ss_quantity@10 <= 20, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 1 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 20, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@10 >= 21 AND ss_quantity@10 <= 40, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 21 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 40, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@10 >= 21 AND ss_quantity@10 <= 40, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 21 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 40, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@10 >= 21 AND ss_quantity@10 <= 40, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 21 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 40, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@10 >= 41 AND ss_quantity@10 <= 60, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 41 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 60, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@10 >= 41 AND ss_quantity@10 <= 60, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 41 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 60, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@10 >= 41 AND ss_quantity@10 <= 60, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 41 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 60, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 10 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@10 >= 61 AND ss_quantity@10 <= 80, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 61 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 80, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 11 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@10 >= 61 AND ss_quantity@10 <= 80, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 61 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 80, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@10 >= 61 AND ss_quantity@10 <= 80, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 61 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 80, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 13 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@10 >= 81 AND ss_quantity@10 <= 100, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 81 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 100, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 14 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@10 >= 81 AND ss_quantity@10 <= 100, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 81 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 100, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 15 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@10 >= 81 AND ss_quantity@10 <= 100, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 81 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 100, required_guarantees=[] └────────────────────────────────────────────────── "); @@ -1020,97 +975,96 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [cd_gender@0 ASC NULLS LAST, cd_marital_status@1 ASC NULLS LAST, cd_education_status@2 ASC NULLS LAST, cd_purchase_estimate@4 ASC NULLS LAST, cd_credit_rating@6 ASC NULLS LAST, cd_dep_count@8 ASC NULLS LAST, cd_dep_employed_count@10 ASC NULLS LAST, cd_dep_college_count@12 ASC NULLS LAST], fetch=100 - │ SortExec: TopK(fetch=100), expr=[cd_gender@0 ASC NULLS LAST, cd_marital_status@1 ASC NULLS LAST, cd_education_status@2 ASC NULLS LAST, cd_purchase_estimate@4 ASC NULLS LAST, cd_credit_rating@6 ASC NULLS LAST, cd_dep_count@8 ASC NULLS LAST, cd_dep_employed_count@10 ASC NULLS LAST, cd_dep_college_count@12 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, count(Int64(1))@8 as cnt1, cd_purchase_estimate@3 as cd_purchase_estimate, count(Int64(1))@8 as cnt2, cd_credit_rating@4 as cd_credit_rating, count(Int64(1))@8 as cnt3, cd_dep_count@5 as cd_dep_count, count(Int64(1))@8 as cnt4, cd_dep_employed_count@6 as cd_dep_employed_count, count(Int64(1))@8 as cnt5, cd_dep_college_count@7 as cd_dep_college_count, count(Int64(1))@8 as cnt6] - │ AggregateExec: mode=FinalPartitioned, gby=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, cd_purchase_estimate@3 as cd_purchase_estimate, cd_credit_rating@4 as cd_credit_rating, cd_dep_count@5 as cd_dep_count, cd_dep_employed_count@6 as cd_dep_employed_count, cd_dep_college_count@7 as cd_dep_college_count], aggr=[count(Int64(1))] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([cd_gender@0, cd_marital_status@1, cd_education_status@2, cd_purchase_estimate@3, cd_credit_rating@4, cd_dep_count@5, cd_dep_employed_count@6, cd_dep_college_count@7], 3), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, cd_purchase_estimate@3 as cd_purchase_estimate, cd_credit_rating@4 as cd_credit_rating, cd_dep_count@5 as cd_dep_count, cd_dep_employed_count@6 as cd_dep_employed_count, cd_dep_college_count@7 as cd_dep_college_count], aggr=[count(Int64(1))] - │ FilterExec: mark@8 OR mark@9, projection=[cd_gender@0, cd_marital_status@1, cd_education_status@2, cd_purchase_estimate@3, cd_credit_rating@4, cd_dep_count@5, cd_dep_employed_count@6, cd_dep_college_count@7] - │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(cs_ship_customer_sk@0, CAST(c.c_customer_sk AS Float64)@10)], projection=[cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8, mark@9, mark@11] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, cd_dep_count@6 as cd_dep_count, cd_dep_employed_count@7 as cd_dep_employed_count, cd_dep_college_count@8 as cd_dep_college_count, mark@9 as mark, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] - │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(ws_bill_customer_sk@0, CAST(c.c_customer_sk AS Float64)@9)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8, mark@10] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, cd_dep_count@6 as cd_dep_count, cd_dep_employed_count@7 as cd_dep_employed_count, cd_dep_college_count@8 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(ss_customer_sk@0, CAST(c.c_customer_sk AS Float64)@9)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, cd_dep_count@6 as cd_dep_count, cd_dep_employed_count@7 as cd_dep_employed_count, cd_dep_college_count@8 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@9)], projection=[c_customer_sk@0, cd_gender@3, cd_marital_status@4, cd_education_status@5, cd_purchase_estimate@6, cd_credit_rating@7, cd_dep_count@8, cd_dep_employed_count@9, cd_dep_college_count@10] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status, cd_purchase_estimate, cd_credit_rating, cd_dep_count, cd_dep_employed_count, cd_dep_college_count, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[cd_gender@0 ASC NULLS LAST, cd_marital_status@1 ASC NULLS LAST, cd_education_status@2 ASC NULLS LAST, cd_purchase_estimate@4 ASC NULLS LAST, cd_credit_rating@6 ASC NULLS LAST, cd_dep_count@8 ASC NULLS LAST, cd_dep_employed_count@10 ASC NULLS LAST, cd_dep_college_count@12 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, count(Int64(1))@8 as cnt1, cd_purchase_estimate@3 as cd_purchase_estimate, count(Int64(1))@8 as cnt2, cd_credit_rating@4 as cd_credit_rating, count(Int64(1))@8 as cnt3, cd_dep_count@5 as cd_dep_count, count(Int64(1))@8 as cnt4, cd_dep_employed_count@6 as cd_dep_employed_count, count(Int64(1))@8 as cnt5, cd_dep_college_count@7 as cd_dep_college_count, count(Int64(1))@8 as cnt6] + │ AggregateExec: mode=FinalPartitioned, gby=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, cd_purchase_estimate@3 as cd_purchase_estimate, cd_credit_rating@4 as cd_credit_rating, cd_dep_count@5 as cd_dep_count, cd_dep_employed_count@6 as cd_dep_employed_count, cd_dep_college_count@7 as cd_dep_college_count], aggr=[count(Int64(1))] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cd_gender@0, cd_marital_status@1, cd_education_status@2, cd_purchase_estimate@3, cd_credit_rating@4, cd_dep_count@5, cd_dep_employed_count@6, cd_dep_college_count@7], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, cd_purchase_estimate@3 as cd_purchase_estimate, cd_credit_rating@4 as cd_credit_rating, cd_dep_count@5 as cd_dep_count, cd_dep_employed_count@6 as cd_dep_employed_count, cd_dep_college_count@7 as cd_dep_college_count], aggr=[count(Int64(1))] + │ FilterExec: mark@8 OR mark@9, projection=[cd_gender@0, cd_marital_status@1, cd_education_status@2, cd_purchase_estimate@3, cd_credit_rating@4, cd_dep_count@5, cd_dep_employed_count@6, cd_dep_college_count@7] + │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(cs_ship_customer_sk@0, CAST(c.c_customer_sk AS Float64)@10)], projection=[cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8, mark@9, mark@11] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, cd_dep_count@6 as cd_dep_count, cd_dep_employed_count@7 as cd_dep_employed_count, cd_dep_college_count@8 as cd_dep_college_count, mark@9 as mark, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] + │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(ws_bill_customer_sk@0, CAST(c.c_customer_sk AS Float64)@9)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8, mark@10] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, cd_dep_count@6 as cd_dep_count, cd_dep_employed_count@7 as cd_dep_employed_count, cd_dep_college_count@8 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(ss_customer_sk@0, CAST(c.c_customer_sk AS Float64)@9)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, cd_dep_count@6 as cd_dep_count, cd_dep_employed_count@7 as cd_dep_employed_count, cd_dep_college_count@8 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@9)], projection=[c_customer_sk@0, cd_gender@3, cd_marital_status@4, cd_education_status@5, cd_purchase_estimate@6, cd_credit_rating@7, cd_dep_count@8, cd_dep_employed_count@9, cd_dep_college_count@10] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status, cd_purchase_estimate, cd_credit_rating, cd_dep_count, cd_dep_employed_count, cd_dep_college_count, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_sk@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[c_customer_sk@1, c_current_cdemo_sk@2] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: ca_county@1 IN (SET) ([Rush County, Toole County, Jefferson County, Dona Ana County, La Porte County]), projection=[ca_address_sk@0] + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[c_customer_sk@1, c_current_cdemo_sk@2] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county], file_type=parquet, predicate=ca_county@7 IN (SET) ([Rush County, Toole County, Jefferson County, Dona Ana County, La Porte County]), pruning_predicate=ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Rush County AND Rush County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Toole County AND Toole County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Jefferson County AND Jefferson County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Dona Ana County AND Dona Ana County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= La Porte County AND La Porte County <= ca_county_max@1, required_guarantees=[ca_county in (Dona Ana County, Jefferson County, La Porte County, Rush County, Toole County)] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ FilterExec: ca_county@1 IN (SET) ([Rush County, Toole County, Jefferson County, Dona Ana County, La Porte County]), projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county], file_type=parquet, predicate=ca_county@7 IN (SET) ([Rush County, Toole County, Jefferson County, Dona Ana County, La Porte County]), pruning_predicate=ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Rush County AND Rush County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Toole County AND Toole County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Jefferson County AND Jefferson County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Dona Ana County AND Dona Ana County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= La Porte County AND La Porte County <= ca_county_max@1, required_guarantees=[ca_county in (Dona Ana County, Jefferson County, La Porte County, Rush County, Toole County)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -1120,156 +1074,152 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], fetch=100 - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], preserve_partitioning=[true] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),18,2 THEN CAST(year_total@3 AS Float64) / CAST(year_total@2 AS Float64) ELSE 0 END > CASE WHEN year_total@0 > Some(0),18,2 THEN CAST(year_total@1 AS Float64) / CAST(year_total@0 AS Float64) ELSE 0 END, projection=[customer_id@2, customer_first_name@3, customer_last_name@4, customer_preferred_cust_flag@5] │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)], ordering_mode=PartiallySorted([7]) - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 12 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[customer_id@1 as customer_id, year_total@2 as year_total, customer_id@3 as customer_id, customer_first_name@4 as customer_first_name, customer_last_name@5 as customer_last_name, customer_preferred_cust_flag@6 as customer_preferred_cust_flag, year_total@7 as year_total, year_total@0 as year_total] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, year_total@3, customer_id@4, customer_first_name@5, customer_last_name@6, customer_preferred_cust_flag@7, year_total@8] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, c_preferred_cust_flag@3 as customer_preferred_cust_flag, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@1 as year_total] │ FilterExec: sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@1 > Some(0),18,2 │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@8 as sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)], ordering_mode=PartiallySorted([7]) - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@9 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ws_ext_discount_amt@8 as ws_ext_discount_amt, ws_ext_list_price@9 as ws_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ws_ext_discount_amt@11, ws_ext_list_price@12] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_list_price@12] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_ext_discount_amt, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@1 as year_total] │ FilterExec: sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@1 > Some(0),18,2 │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@8 as sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@9 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ss_ext_discount_amt@8 as ss_ext_discount_amt, ss_ext_list_price@9 as ss_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ss_ext_discount_amt@11, ss_ext_list_price@12] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_list_price@12] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_discount_amt, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 11 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@9 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ss_ext_discount_amt@8 as ss_ext_discount_amt, ss_ext_list_price@9 as ss_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ss_ext_discount_amt@11, ss_ext_list_price@12] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_list_price@12] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_discount_amt, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2002 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 + ┌───── Stage 15 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@9 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ws_ext_discount_amt@8 as ws_ext_discount_amt, ws_ext_list_price@9 as ws_ext_list_price, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ws_ext_discount_amt@11, ws_ext_list_price@12] │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_list_price@12] │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_ext_discount_amt, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 13 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2002 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 14 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── "); @@ -1281,45 +1231,47 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_category@2 ASC NULLS LAST, i_class@3 ASC NULLS LAST, i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, revenueratio@6 ASC NULLS LAST], fetch=100 - │ SortExec: TopK(fetch=100), expr=[i_category@2 ASC NULLS LAST, i_class@3 ASC NULLS LAST, i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, revenueratio@6 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price, sum(web_sales.ws_ext_sales_price)@5 as itemrevenue, CAST(sum(web_sales.ws_ext_sales_price)@5 AS Float64) * 100 / CAST(sum(sum(web_sales.ws_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@6 AS Float64) as revenueratio] - │ WindowAggExec: wdw=[sum(sum(web_sales.ws_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(sum(web_sales.ws_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(27, 2), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] - │ SortExec: expr=[i_class@3 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_class@3], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price], aggr=[sum(web_sales.ws_ext_sales_price)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_category@2, i_class@3, i_current_price@4], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id, i_item_desc@2 as i_item_desc, i_category@5 as i_category, i_class@4 as i_class, i_current_price@3 as i_current_price], aggr=[sum(web_sales.ws_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_ext_sales_price@3, i_item_id@4, i_item_desc@5, i_current_price@6, i_class@7, i_category@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@5 as ws_sold_date_sk, ws_ext_sales_price@6 as ws_ext_sales_price, i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price, i_class@3 as i_class, i_category@4 as i_category] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3, i_class@4, i_category@5, ws_sold_date_sk@6, ws_ext_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[i_category@2 ASC NULLS LAST, i_class@3 ASC NULLS LAST, i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, revenueratio@6 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price, sum(web_sales.ws_ext_sales_price)@5 as itemrevenue, CAST(sum(web_sales.ws_ext_sales_price)@5 AS Float64) * 100 / CAST(sum(sum(web_sales.ws_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@6 AS Float64) as revenueratio] + │ WindowAggExec: wdw=[sum(sum(web_sales.ws_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(sum(web_sales.ws_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(27, 2), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] + │ SortExec: expr=[i_class@3 ASC NULLS LAST], preserve_partitioning=[true] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_class@3], 12), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price], aggr=[sum(web_sales.ws_ext_sales_price)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_category@2, i_class@3, i_current_price@4], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id, i_item_desc@2 as i_item_desc, i_category@5 as i_category, i_class@4 as i_class, i_current_price@3 as i_current_price], aggr=[sum(web_sales.ws_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_ext_sales_price@3, i_item_id@4, i_item_desc@5, i_current_price@6, i_class@7, i_category@8] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@5 as ws_sold_date_sk, ws_ext_sales_price@6 as ws_ext_sales_price, i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price, i_class@3 as i_class, i_category@4 as i_category] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3, i_class@4, i_category@5, ws_sold_date_sk@6, ws_ext_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] + └────────────────────────────────────────────────── "#); Ok(()) } @@ -1331,67 +1283,58 @@ mod tests { │ ProjectionExec: expr=[avg(store_sales.ss_quantity)@0 as avg1, avg(store_sales.ss_ext_sales_price)@1 as avg2, avg(store_sales.ss_ext_wholesale_cost)@2 as avg3, sum(store_sales.ss_ext_wholesale_cost)@3 as sum(store_sales.ss_ext_wholesale_cost)] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_ext_sales_price), avg(store_sales.ss_ext_wholesale_cost), sum(store_sales.ss_ext_wholesale_cost)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_ext_sales_price), avg(store_sales.ss_ext_wholesale_cost), sum(store_sales.ss_ext_wholesale_cost)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_quantity@1, ss_ext_sales_price@2, ss_ext_wholesale_cost@3] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = TX OR ca_state@1 = OH) AND ss_net_profit@0 >= Some(10000),6,2 AND ss_net_profit@0 <= Some(20000),6,2 OR (ca_state@1 = OR OR ca_state@1 = NM OR ca_state@1 = KY) AND ss_net_profit@0 >= Some(15000),6,2 AND ss_net_profit@0 <= Some(30000),6,2 OR (ca_state@1 = VA OR ca_state@1 = TX OR ca_state@1 = MS) AND ss_net_profit@0 >= Some(5000),6,2 AND ss_net_profit@0 <= Some(25000),6,2, projection=[ss_sold_date_sk@0, ss_quantity@2, ss_ext_sales_price@3, ss_ext_wholesale_cost@4] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: __common_expr_4@0 AND __common_expr_4@0 AND ca_country@3 = United States AND ca_state@2 IN (SET) ([TX, OH, OR, NM, KY, VA, MS]), projection=[ca_address_sk@1, ca_state@2] + │ ProjectionExec: expr=[__common_expr_5@0 OR ca_state@2 = OH OR ca_state@2 = OR OR ca_state@2 = NM OR ca_state@2 = KY OR ca_state@2 = VA OR __common_expr_5@0 OR ca_state@2 = MS as __common_expr_4, ca_address_sk@1 as ca_address_sk, ca_state@2 as ca_state, ca_country@3 as ca_country] + │ ProjectionExec: expr=[ca_state@1 = TX as __common_expr_5, ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, ca_country@2 as ca_country] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=ca_country@10 = United States AND ca_state@8 IN (SET) ([TX, OH, OR, NM, KY, VA, MS]), pruning_predicate=ca_country_null_count@2 != row_count@3 AND ca_country_min@0 <= United States AND United States <= ca_country_max@1 AND (ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= TX AND TX <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= OH AND OH <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= OR AND OR <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= NM AND NM <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= KY AND KY <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= VA AND VA <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= MS AND MS <= ca_state_max@5), required_guarantees=[ca_country in (United States), ca_state in (KY, MS, NM, OH, OR, TX, VA)] + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_ext_sales_price), avg(store_sales.ss_ext_wholesale_cost), sum(store_sales.ss_ext_wholesale_cost)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_quantity@1, ss_ext_sales_price@2, ss_ext_wholesale_cost@3] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(household_demographics.hd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 AND hd_dep_count@3 = 3 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 AND hd_dep_count@3 = 1 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2 AND hd_dep_count@3 = 1, projection=[ss_sold_date_sk@0, ss_addr_sk@2, ss_quantity@3, ss_ext_sales_price@5, ss_ext_wholesale_cost@6, ss_net_profit@7] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = TX OR ca_state@1 = OH) AND ss_net_profit@0 >= Some(10000),6,2 AND ss_net_profit@0 <= Some(20000),6,2 OR (ca_state@1 = OR OR ca_state@1 = NM OR ca_state@1 = KY) AND ss_net_profit@0 >= Some(15000),6,2 AND ss_net_profit@0 <= Some(30000),6,2 OR (ca_state@1 = VA OR ca_state@1 = TX OR ca_state@1 = MS) AND ss_net_profit@0 >= Some(5000),6,2 AND ss_net_profit@0 <= Some(25000),6,2, projection=[ss_sold_date_sk@0, ss_quantity@2, ss_ext_sales_price@3, ss_ext_wholesale_cost@4] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, hd_dep_count@1 as hd_dep_count, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ FilterExec: (__common_expr_3@0 OR hd_dep_count@2 = 1) AND __common_expr_3@0, projection=[hd_demo_sk@1, hd_dep_count@2] + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_dep_count@3 = 3 OR hd_dep_count@3 = 1 as __common_expr_3, hd_demo_sk, hd_dep_count], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2, projection=[ss_sold_date_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_quantity@4, ss_sales_price@5, ss_ext_sales_price@6, ss_ext_wholesale_cost@7, ss_net_profit@8, cd_marital_status@10, cd_education_status@11] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: __common_expr_4@0 AND __common_expr_4@0 AND ca_country@3 = United States AND ca_state@2 IN (SET) ([TX, OH, OR, NM, KY, VA, MS]), projection=[ca_address_sk@1, ca_state@2] - │ ProjectionExec: expr=[__common_expr_5@0 OR ca_state@2 = OH OR ca_state@2 = OR OR ca_state@2 = NM OR ca_state@2 = KY OR ca_state@2 = VA OR __common_expr_5@0 OR ca_state@2 = MS as __common_expr_4, ca_address_sk@1 as ca_address_sk, ca_state@2 as ca_state, ca_country@3 as ca_country] - │ ProjectionExec: expr=[ca_state@1 = TX as __common_expr_5, ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, ca_country@2 as ca_country] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=ca_country@10 = United States AND ca_state@8 IN (SET) ([TX, OH, OR, NM, KY, VA, MS]), pruning_predicate=ca_country_null_count@2 != row_count@3 AND ca_country_min@0 <= United States AND United States <= ca_country_max@1 AND (ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= TX AND TX <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= OH AND OH <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= OR AND OR <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= NM AND NM <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= KY AND KY <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= VA AND VA <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= MS AND MS <= ca_state_max@5), required_guarantees=[ca_country in (United States), ca_state in (KY, MS, NM, OH, OR, TX, VA)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(household_demographics.hd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 AND hd_dep_count@3 = 3 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 AND hd_dep_count@3 = 1 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2 AND hd_dep_count@3 = 1, projection=[ss_sold_date_sk@0, ss_addr_sk@2, ss_quantity@3, ss_ext_sales_price@5, ss_ext_wholesale_cost@6, ss_net_profit@7] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Advanced Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College OR cd_marital_status@2 = W AND cd_education_status@3 = 2 yr Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, Advanced Degree, College), cd_marital_status in (M, S, W)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@4)], projection=[ss_sold_date_sk@2, ss_cdemo_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_quantity@7, ss_sales_price@8, ss_ext_sales_price@9, ss_ext_wholesale_cost@10, ss_net_profit@11] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, hd_dep_count@1 as hd_dep_count, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ FilterExec: (__common_expr_3@0 OR hd_dep_count@2 = 1) AND __common_expr_3@0, projection=[hd_demo_sk@1, hd_dep_count@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_dep_count@3 = 3 OR hd_dep_count@3 = 1 as __common_expr_3, hd_demo_sk, hd_dep_count], file_type=parquet + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ FilterExec: (ss_net_profit@9 >= Some(10000),6,2 AND ss_net_profit@9 <= Some(20000),6,2 OR ss_net_profit@9 >= Some(15000),6,2 AND ss_net_profit@9 <= Some(30000),6,2 OR ss_net_profit@9 >= Some(5000),6,2 AND ss_net_profit@9 <= Some(25000),6,2) AND (ss_sales_price@6 >= Some(10000),5,2 AND ss_sales_price@6 <= Some(15000),5,2 OR ss_sales_price@6 >= Some(5000),5,2 AND ss_sales_price@6 <= Some(10000),5,2 OR ss_sales_price@6 >= Some(15000),5,2 AND ss_sales_price@6 <= Some(20000),5,2) + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_quantity, ss_sales_price, ss_ext_sales_price, ss_ext_wholesale_cost, ss_net_profit], file_type=parquet, predicate=(ss_net_profit@22 >= Some(10000),6,2 AND ss_net_profit@22 <= Some(20000),6,2 OR ss_net_profit@22 >= Some(15000),6,2 AND ss_net_profit@22 <= Some(30000),6,2 OR ss_net_profit@22 >= Some(5000),6,2 AND ss_net_profit@22 <= Some(25000),6,2) AND (ss_sales_price@13 >= Some(10000),5,2 AND ss_sales_price@13 <= Some(15000),5,2 OR ss_sales_price@13 >= Some(5000),5,2 AND ss_sales_price@13 <= Some(10000),5,2 OR ss_sales_price@13 >= Some(15000),5,2 AND ss_sales_price@13 <= Some(20000),5,2) AND DynamicFilter [ empty ], pruning_predicate=(ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(10000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(20000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(15000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(30000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(5000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(25000),6,2) AND (ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(10000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(15000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(5000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(10000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(15000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2, projection=[ss_sold_date_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_quantity@4, ss_sales_price@5, ss_ext_sales_price@6, ss_ext_wholesale_cost@7, ss_net_profit@8, cd_marital_status@10, cd_education_status@11] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Advanced Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College OR cd_marital_status@2 = W AND cd_education_status@3 = 2 yr Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, Advanced Degree, College), cd_marital_status in (M, S, W)] + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@4)], projection=[ss_sold_date_sk@2, ss_cdemo_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_quantity@7, ss_sales_price@8, ss_ext_sales_price@9, ss_ext_wholesale_cost@10, ss_net_profit@11] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ FilterExec: (ss_net_profit@9 >= Some(10000),6,2 AND ss_net_profit@9 <= Some(20000),6,2 OR ss_net_profit@9 >= Some(15000),6,2 AND ss_net_profit@9 <= Some(30000),6,2 OR ss_net_profit@9 >= Some(5000),6,2 AND ss_net_profit@9 <= Some(25000),6,2) AND (ss_sales_price@6 >= Some(10000),5,2 AND ss_sales_price@6 <= Some(15000),5,2 OR ss_sales_price@6 >= Some(5000),5,2 AND ss_sales_price@6 <= Some(10000),5,2 OR ss_sales_price@6 >= Some(15000),5,2 AND ss_sales_price@6 <= Some(20000),5,2) - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_quantity, ss_sales_price, ss_ext_sales_price, ss_ext_wholesale_cost, ss_net_profit], file_type=parquet, predicate=(ss_net_profit@22 >= Some(10000),6,2 AND ss_net_profit@22 <= Some(20000),6,2 OR ss_net_profit@22 >= Some(15000),6,2 AND ss_net_profit@22 <= Some(30000),6,2 OR ss_net_profit@22 >= Some(5000),6,2 AND ss_net_profit@22 <= Some(25000),6,2) AND (ss_sales_price@13 >= Some(10000),5,2 AND ss_sales_price@13 <= Some(15000),5,2 OR ss_sales_price@13 >= Some(5000),5,2 AND ss_sales_price@13 <= Some(10000),5,2 OR ss_sales_price@13 >= Some(15000),5,2 AND ss_sales_price@13 <= Some(20000),5,2) AND DynamicFilter [ empty ], pruning_predicate=(ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(10000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(20000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(15000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(30000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(5000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(25000),6,2) AND (ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(10000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(15000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(5000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(10000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(15000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── "); Ok(()) } @@ -1418,20 +1361,11 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id], aggr=[sum(store_sales.ss_quantity * store_sales.ss_list_price), count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ss_item_sk@0, ss_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, i_brand_id@3, i_class_id@4, i_category_id@5] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[i_item_sk@0 as ss_item_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(brand_id@0, i_brand_id@1), (class_id@1, i_class_id@2), (category_id@2, i_category_id@3)], projection=[i_item_sk@3] │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 - │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 - │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[catalog as channel, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)@3 as sales, count(Int64(1))@4 as number_sales] @@ -1439,26 +1373,17 @@ mod tests { │ ProjectionExec: expr=[avg(sq2.quantity * sq2.list_price)@0 as average_sales] │ AggregateExec: mode=Final, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ [Stage 17] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 │ AggregateExec: mode=FinalPartitioned, gby=[i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id], aggr=[sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price), count(Int64(1))] │ RepartitionExec: partitioning=Hash([i_brand_id@0, i_class_id@1, i_category_id@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id], aggr=[sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price), count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(cs_item_sk@0, ss_item_sk@0)], projection=[cs_quantity@1, cs_list_price@2, i_brand_id@3, i_class_id@4, i_category_id@5] │ CoalescePartitionsExec - │ [Stage 19] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 20] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[i_item_sk@0 as ss_item_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(brand_id@0, i_brand_id@1), (class_id@1, i_class_id@2), (category_id@2, i_category_id@3)], projection=[i_item_sk@3] │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 - │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 - │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] - │ [Stage 28] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 30] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[web as channel, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, sum(web_sales.ws_quantity * web_sales.ws_list_price)@3 as sales, count(Int64(1))@4 as number_sales] @@ -1466,32 +1391,23 @@ mod tests { │ ProjectionExec: expr=[avg(sq2.quantity * sq2.list_price)@0 as average_sales] │ AggregateExec: mode=Final, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] │ CoalescePartitionsExec - │ [Stage 30] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ [Stage 32] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 │ AggregateExec: mode=FinalPartitioned, gby=[i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id], aggr=[sum(web_sales.ws_quantity * web_sales.ws_list_price), count(Int64(1))] │ RepartitionExec: partitioning=Hash([i_brand_id@0, i_class_id@1, i_category_id@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id], aggr=[sum(web_sales.ws_quantity * web_sales.ws_list_price), count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ws_item_sk@0, ss_item_sk@0)], projection=[ws_quantity@1, ws_list_price@2, i_brand_id@3, i_class_id@4, i_category_id@5] │ CoalescePartitionsExec - │ [Stage 33] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 35] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[i_item_sk@0 as ss_item_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(brand_id@0, i_brand_id@1), (class_id@1, i_class_id@2), (category_id@2, i_category_id@3)], projection=[i_item_sk@3] │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 36] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 - │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 39] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 - │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] - │ [Stage 42] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 45] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] + │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] t3:[c2] │ ProjectionExec: expr=[ss_quantity@0 as quantity, ss_list_price@1 as list_price] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_list_price@4] │ CoalescePartitionsExec @@ -1504,131 +1420,134 @@ mod tests { │ ProjectionExec: expr=[cs_quantity@0 as quantity, cs_list_price@1 as list_price] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_quantity@3, cs_list_price@4] │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] + │ CoalescePartitionsExec │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] - │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-3.parquet:..]]}, projection=[cs_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-3.parquet:..]]}, projection=[ws_sold_date_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_quantity@4, ss_list_price@5, i_brand_id@6, i_class_id@7, i_category_id@8] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, ss_item_sk@4 as ss_item_sk, ss_quantity@5 as ss_quantity, ss_list_price@6 as ss_list_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4, ss_item_sk@5, ss_quantity@6, ss_list_price@7] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_quantity, ss_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 11, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 4 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=4 + │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=4 + │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] + │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 8 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 3), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] - │ ProjectionExec: expr=[i_brand_id@0 as brand_id, i_class_id@1 as class_id, i_category_id@2 as category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4] + │ CoalescePartitionsExec + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] + │ ProjectionExec: expr=[i_brand_id@0 as brand_id, i_class_id@1 as class_id, i_category_id@2 as category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4] + │ CoalescePartitionsExec + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + ┌───── Stage 12 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] + │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] t3:[c2] │ ProjectionExec: expr=[ss_quantity@0 as quantity, ss_list_price@1 as list_price] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_list_price@4] │ CoalescePartitionsExec @@ -1641,131 +1560,134 @@ mod tests { │ ProjectionExec: expr=[cs_quantity@0 as quantity, cs_list_price@1 as list_price] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_quantity@3, cs_list_price@4] │ CoalescePartitionsExec + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] + │ CoalescePartitionsExec │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] - │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-3.parquet:..]]}, projection=[cs_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] - │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-3.parquet:..]]}, projection=[ws_sold_date_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p6..p11] + ┌───── Stage 16 ── Tasks: t0:[p0..p5] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 20 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_quantity@4, cs_list_price@5, i_brand_id@6, i_class_id@7, i_category_id@8] │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4, cs_item_sk@5, cs_quantity@6, cs_list_price@7] │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 18 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 11, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 19 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 25 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + ┌───── Stage 30 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4] + │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=4 + │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=4 + │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] + │ [Stage 29] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 23 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 23 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ CoalescePartitionsExec + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4] + │ CoalescePartitionsExec + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 28 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 3), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] - │ ProjectionExec: expr=[i_brand_id@0 as brand_id, i_class_id@1 as class_id, i_category_id@2 as category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 26 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + ┌───── Stage 21 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 22 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 26 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ CoalescePartitionsExec + │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4] + │ CoalescePartitionsExec + │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 27 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 24 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 25 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 29 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] + │ ProjectionExec: expr=[i_brand_id@0 as brand_id, i_class_id@1 as class_id, i_category_id@2 as category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ CoalescePartitionsExec + │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4] + │ CoalescePartitionsExec + │ [Stage 28] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 30 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + ┌───── Stage 27 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 28 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 32 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] + │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] t3:[c2] │ ProjectionExec: expr=[ss_quantity@0 as quantity, ss_list_price@1 as list_price] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_list_price@4] │ CoalescePartitionsExec @@ -1778,128 +1700,131 @@ mod tests { │ ProjectionExec: expr=[cs_quantity@0 as quantity, cs_list_price@1 as list_price] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_quantity@3, cs_list_price@4] │ CoalescePartitionsExec + │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] + │ CoalescePartitionsExec │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] - │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-3.parquet:..]]}, projection=[cs_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] - │ CoalescePartitionsExec - │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-3.parquet:..]]}, projection=[ws_sold_date_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 29 ── Tasks: t0:[p0..p5] t1:[p6..p11] + ┌───── Stage 31 ── Tasks: t0:[p0..p5] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 33 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 35 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_quantity@4, ws_list_price@5, i_brand_id@6, i_class_id@7, i_category_id@8] │ CoalescePartitionsExec - │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 33] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, ws_item_sk@4 as ws_item_sk, ws_quantity@5 as ws_quantity, ws_list_price@6 as ws_list_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4, ws_item_sk@5, ws_quantity@6, ws_list_price@7] │ CoalescePartitionsExec - │ [Stage 32] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 34] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 31 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 33 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 11, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 32 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 36 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 34] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 35] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 34 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 35 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 34 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 39 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + ┌───── Stage 45 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 37] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4] + │ [Stage 38] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=4 + │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 38] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 41] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=4 + │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] + │ [Stage 44] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 37 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + ┌───── Stage 38 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ CoalescePartitionsExec + │ [Stage 36] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4] + │ CoalescePartitionsExec + │ [Stage 37] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 38 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 36 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 37 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 41 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ CoalescePartitionsExec + │ [Stage 39] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4] + │ CoalescePartitionsExec + │ [Stage 40] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 42 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 3), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] - │ ProjectionExec: expr=[i_brand_id@0 as brand_id, i_class_id@1 as class_id, i_category_id@2 as category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 40] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 41] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 40 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 41 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 39 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 40 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 44 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] + │ ProjectionExec: expr=[i_brand_id@0 as brand_id, i_class_id@1 as class_id, i_category_id@2 as category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ CoalescePartitionsExec + │ [Stage 42] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4] + │ CoalescePartitionsExec + │ [Stage 43] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 42 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 43 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── "); Ok(()) } @@ -1909,51 +1834,49 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ca_zip@0 ASC], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[ca_zip@0 ASC], preserve_partitioning=[true] │ AggregateExec: mode=FinalPartitioned, gby=[ca_zip@0 as ca_zip], aggr=[sum(catalog_sales.cs_sales_price)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ca_zip@0], 6), input_partitions=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ca_zip@0], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_zip@1 as ca_zip], aggr=[sum(catalog_sales.cs_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_sales_price@3, ca_zip@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_sales_price@2 as cs_sales_price, ca_zip@0 as ca_zip] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], filter=substr(ca_zip@2, 1, 5) IN (SET) ([85669, 86197, 88274, 83405, 86475, 85392, 85460, 80348, 81792]) OR ca_state@1 = CA OR ca_state@1 = WA OR ca_state@1 = GA OR cs_sales_price@0 > Some(50000),5,2, projection=[ca_zip@2, cs_sold_date_sk@3, cs_sales_price@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_sales_price@2 as cs_sales_price, c_current_addr_sk@0 as c_current_addr_sk] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@2, cs_bill_customer_sk@1)], projection=[c_current_addr_sk@1, cs_sold_date_sk@3, cs_sales_price@5] │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2001, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2001, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 9), input_partitions=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 12), input_partitions=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -1976,46 +1899,43 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(cs_order_number@1, cs_order_number@1)], filter=cs_warehouse_sk@0 != cs_warehouse_sk@1, projection=[cs_order_number@1, cs_ext_ship_cost@2, cs_net_profit@3] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-3.parquet:..]]}, projection=[cs_warehouse_sk, cs_order_number], file_type=parquet │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@1, cs_call_center_sk@0)], projection=[cs_warehouse_sk@3, cs_order_number@4, cs_ext_ship_cost@5, cs_net_profit@6] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_address.ca_address_sk AS Float64)@1, cs_ship_addr_sk@0)], projection=[cs_call_center_sk@3, cs_warehouse_sk@4, cs_order_number@5, cs_ext_ship_cost@6, cs_net_profit@7] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_ship_date_sk@0)], projection=[cs_ship_addr_sk@3, cs_call_center_sk@4, cs_warehouse_sk@5, cs_order_number@6, cs_ext_ship_cost@7, cs_net_profit@8] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_ship_date_sk, cs_ship_addr_sk, cs_call_center_sk, cs_warehouse_sk, cs_order_number, cs_ext_ship_cost, cs_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[cc_call_center_sk@0 as cc_call_center_sk, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)] │ FilterExec: cc_county@1 = Williamson County, projection=[cc_call_center_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_county], file_type=parquet, predicate=cc_county@25 = Williamson County, pruning_predicate=cc_county_null_count@2 != row_count@3 AND cc_county_min@0 <= Williamson County AND Williamson County <= cc_county_max@1, required_guarantees=[cc_county in (Williamson County)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_county], file_type=parquet, predicate=cc_county@25 = Williamson County, pruning_predicate=cc_county_null_count@2 != row_count@3 AND cc_county_min@0 <= Williamson County AND Williamson County <= cc_county_max@1, required_guarantees=[cc_county in (Williamson County)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] │ FilterExec: ca_state@1 = GA, projection=[ca_address_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet, predicate=ca_state@8 = GA, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1, required_guarantees=[ca_state in (GA)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet, predicate=ca_state@8 = GA, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1, required_guarantees=[ca_state in (GA)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2002-02-01 AND d_date@1 <= 2002-04-02, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2002-02-01 AND d_date@2 <= 2002-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2002-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2002-04-02, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2002-02-01 AND d_date@2 <= 2002-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2002-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2002-04-02, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -2026,95 +1946,88 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC, i_item_desc@1 ASC, s_state@2 ASC], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC, i_item_desc@1 ASC, s_state@2 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_state@2 as s_state, count(store_sales.ss_quantity)@3 as store_sales_quantitycount, avg(store_sales.ss_quantity)@4 as store_sales_quantityave, stddev(store_sales.ss_quantity)@5 as store_sales_quantitystdev, stddev(store_sales.ss_quantity)@5 / avg(store_sales.ss_quantity)@4 as store_sales_quantitycov, count(store_returns.sr_return_quantity)@6 as store_returns_quantitycount, avg(store_returns.sr_return_quantity)@7 as store_returns_quantityave, stddev(store_returns.sr_return_quantity)@8 as store_returns_quantitystdev, stddev(store_returns.sr_return_quantity)@8 / avg(store_returns.sr_return_quantity)@7 as store_returns_quantitycov, count(catalog_sales.cs_quantity)@9 as catalog_sales_quantitycount, avg(catalog_sales.cs_quantity)@10 as catalog_sales_quantityave, stddev(catalog_sales.cs_quantity)@11 as catalog_sales_quantitystdev, stddev(catalog_sales.cs_quantity)@11 / avg(catalog_sales.cs_quantity)@10 as catalog_sales_quantitycov] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_state@2 as s_state], aggr=[count(store_sales.ss_quantity), avg(store_sales.ss_quantity), stddev(store_sales.ss_quantity), count(store_returns.sr_return_quantity), avg(store_returns.sr_return_quantity), stddev(store_returns.sr_return_quantity), count(catalog_sales.cs_quantity), avg(catalog_sales.cs_quantity), stddev(catalog_sales.cs_quantity)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC, i_item_desc@1 ASC, s_state@2 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_state@2 as s_state, count(store_sales.ss_quantity)@3 as store_sales_quantitycount, avg(store_sales.ss_quantity)@4 as store_sales_quantityave, stddev(store_sales.ss_quantity)@5 as store_sales_quantitystdev, stddev(store_sales.ss_quantity)@5 / avg(store_sales.ss_quantity)@4 as store_sales_quantitycov, count(store_returns.sr_return_quantity)@6 as store_returns_quantitycount, avg(store_returns.sr_return_quantity)@7 as store_returns_quantityave, stddev(store_returns.sr_return_quantity)@8 as store_returns_quantitystdev, stddev(store_returns.sr_return_quantity)@8 / avg(store_returns.sr_return_quantity)@7 as store_returns_quantitycov, count(catalog_sales.cs_quantity)@9 as catalog_sales_quantitycount, avg(catalog_sales.cs_quantity)@10 as catalog_sales_quantityave, stddev(catalog_sales.cs_quantity)@11 as catalog_sales_quantitystdev, stddev(catalog_sales.cs_quantity)@11 / avg(catalog_sales.cs_quantity)@10 as catalog_sales_quantitycov] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_state@2 as s_state], aggr=[count(store_sales.ss_quantity), avg(store_sales.ss_quantity), stddev(store_sales.ss_quantity), count(store_returns.sr_return_quantity), avg(store_returns.sr_return_quantity), stddev(store_returns.sr_return_quantity), count(catalog_sales.cs_quantity), avg(catalog_sales.cs_quantity), stddev(catalog_sales.cs_quantity)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_state@2], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@4 as i_item_id, i_item_desc@5 as i_item_desc, s_state@3 as s_state], aggr=[count(store_sales.ss_quantity), avg(store_sales.ss_quantity), stddev(store_sales.ss_quantity), count(store_returns.sr_return_quantity), avg(store_returns.sr_return_quantity), stddev(store_returns.sr_return_quantity), count(catalog_sales.cs_quantity), avg(catalog_sales.cs_quantity), stddev(catalog_sales.cs_quantity)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, sr_return_quantity@2, cs_quantity@3, s_state@4, i_item_id@6, i_item_desc@7] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_state@2], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@4 as i_item_id, i_item_desc@5 as i_item_desc, s_state@3 as s_state], aggr=[count(store_sales.ss_quantity), avg(store_sales.ss_quantity), stddev(store_sales.ss_quantity), count(store_returns.sr_return_quantity), avg(store_returns.sr_return_quantity), stddev(store_returns.sr_return_quantity), count(catalog_sales.cs_quantity), avg(catalog_sales.cs_quantity), stddev(catalog_sales.cs_quantity)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, sr_return_quantity@2, cs_quantity@3, s_state@4, i_item_id@6, i_item_desc@7] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, sr_return_quantity@3 as sr_return_quantity, cs_quantity@4 as cs_quantity, s_state@0 as s_state] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_state@1, ss_item_sk@3, ss_quantity@5, sr_return_quantity@6, cs_quantity@7] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, cs_sold_date_sk@4)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@5, cs_quantity@7] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, sr_returned_date_sk@3)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@6, cs_sold_date_sk@7, cs_quantity@8] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, sr_returned_date_sk@6, sr_return_quantity@7, cs_sold_date_sk@8, cs_quantity@9] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, sr_return_quantity@3 as sr_return_quantity, cs_quantity@4 as cs_quantity, s_state@0 as s_state] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_state@1, ss_item_sk@3, ss_quantity@5, sr_return_quantity@6, cs_quantity@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, cs_sold_date_sk@4)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@5, cs_quantity@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, sr_returned_date_sk@3)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@6, cs_sold_date_sk@7, cs_quantity@8] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, sr_returned_date_sk@6, sr_return_quantity@7, cs_sold_date_sk@8, cs_quantity@9] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_quarter_name@1 = 2001Q1 OR d_quarter_name@1 = 2001Q2 OR d_quarter_name@1 = 2001Q3, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1 OR d_quarter_name@15 = 2001Q2 OR d_quarter_name@15 = 2001Q3, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q2 AND 2001Q2 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q3 AND 2001Q3 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1, 2001Q2, 2001Q3)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_quarter_name@1 = 2001Q1 OR d_quarter_name@1 = 2001Q2 OR d_quarter_name@1 = 2001Q3, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1 OR d_quarter_name@15 = 2001Q2 OR d_quarter_name@15 = 2001Q3, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q2 AND 2001Q2 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q3 AND 2001Q3 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1, 2001Q2, 2001Q3)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_quarter_name@1 = 2001Q1, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1)] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_customer_sk@6, sr_item_sk@5], 9), input_partitions=3 - │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_store_sk@6 as ss_store_sk, ss_quantity@7 as ss_quantity, sr_returned_date_sk@0 as sr_returned_date_sk, sr_item_sk@1 as sr_item_sk, sr_customer_sk@2 as sr_customer_sk, sr_return_quantity@3 as sr_return_quantity] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_customer_sk@2, ss_customer_sk@2), (sr_item_sk@1, ss_item_sk@1), (sr_ticket_number@3, ss_ticket_number@4)], projection=[sr_returned_date_sk@0, sr_item_sk@1, sr_customer_sk@2, sr_return_quantity@4, ss_sold_date_sk@5, ss_item_sk@6, ss_store_sk@8, ss_quantity@10] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_quarter_name@1 = 2001Q1 OR d_quarter_name@1 = 2001Q2 OR d_quarter_name@1 = 2001Q3, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1 OR d_quarter_name@15 = 2001Q2 OR d_quarter_name@15 = 2001Q3, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q2 AND 2001Q2 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q3 AND 2001Q3 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1, 2001Q2, 2001Q3)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_quarter_name@1 = 2001Q1 OR d_quarter_name@1 = 2001Q2 OR d_quarter_name@1 = 2001Q3, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1 OR d_quarter_name@15 = 2001Q2 OR d_quarter_name@15 = 2001Q3, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q2 AND 2001Q2 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q3 AND 2001Q3 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1, 2001Q2, 2001Q3)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_quarter_name@1 = 2001Q1, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1)] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@6, sr_item_sk@5], 12), input_partitions=3 + │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_store_sk@6 as ss_store_sk, ss_quantity@7 as ss_quantity, sr_returned_date_sk@0 as sr_returned_date_sk, sr_item_sk@1 as sr_item_sk, sr_customer_sk@2 as sr_customer_sk, sr_return_quantity@3 as sr_return_quantity] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_customer_sk@2, ss_customer_sk@2), (sr_item_sk@1, ss_item_sk@1), (sr_ticket_number@3, ss_ticket_number@4)], projection=[sr_returned_date_sk@0, sr_item_sk@1, sr_customer_sk@2, sr_return_quantity@4, ss_sold_date_sk@5, ss_item_sk@6, ss_store_sk@8, ss_quantity@10] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number, sr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number, sr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── "); Ok(()) } @@ -2124,88 +2037,83 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ca_country@1 ASC, ca_state@2 ASC, ca_county@3 ASC, i_item_id@0 ASC], fetch=100 - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[ca_country@1 ASC, ca_state@2 ASC, ca_county@3 ASC, i_item_id@0 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, ca_country@1 as ca_country, ca_state@2 as ca_state, ca_county@3 as ca_county, avg(catalog_sales.cs_quantity)@5 as agg1, avg(catalog_sales.cs_list_price)@6 as agg2, avg(catalog_sales.cs_coupon_amt)@7 as agg3, avg(catalog_sales.cs_sales_price)@8 as agg4, avg(catalog_sales.cs_net_profit)@9 as agg5, avg(customer.c_birth_year)@10 as agg6, avg(cd1.cd_dep_count)@11 as agg7] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, ca_country@1 as ca_country, ca_state@2 as ca_state, ca_county@3 as ca_county, __grouping_id@4 as __grouping_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price), avg(catalog_sales.cs_net_profit), avg(customer.c_birth_year), avg(cd1.cd_dep_count)] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[ca_country@1 ASC, ca_state@2 ASC, ca_county@3 ASC, i_item_id@0 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, ca_country@1 as ca_country, ca_state@2 as ca_state, ca_county@3 as ca_county, avg(catalog_sales.cs_quantity)@5 as agg1, avg(catalog_sales.cs_list_price)@6 as agg2, avg(catalog_sales.cs_coupon_amt)@7 as agg3, avg(catalog_sales.cs_sales_price)@8 as agg4, avg(catalog_sales.cs_net_profit)@9 as agg5, avg(customer.c_birth_year)@10 as agg6, avg(cd1.cd_dep_count)@11 as agg7] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, ca_country@1 as ca_country, ca_state@2 as ca_state, ca_county@3 as ca_county, __grouping_id@4 as __grouping_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price), avg(catalog_sales.cs_net_profit), avg(customer.c_birth_year), avg(cd1.cd_dep_count)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0, ca_country@1, ca_state@2, ca_county@3, __grouping_id@4], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[(NULL as i_item_id, NULL as ca_country, NULL as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, NULL as ca_country, NULL as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, ca_country@9 as ca_country, NULL as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, ca_country@9 as ca_country, ca_state@8 as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, ca_country@9 as ca_country, ca_state@8 as ca_state, ca_county@7 as ca_county)], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price), avg(catalog_sales.cs_net_profit), avg(customer.c_birth_year), avg(cd1.cd_dep_count)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_quantity@1, cs_list_price@2, cs_sales_price@3, cs_coupon_amt@4, cs_net_profit@5, cd_dep_count@6, c_birth_year@7, ca_county@8, ca_state@9, ca_country@10, i_item_id@12] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0, ca_country@1, ca_state@2, ca_county@3, __grouping_id@4], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[(NULL as i_item_id, NULL as ca_country, NULL as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, NULL as ca_country, NULL as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, ca_country@9 as ca_country, NULL as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, ca_country@9 as ca_country, ca_state@8 as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, ca_country@9 as ca_country, ca_state@8 as ca_state, ca_county@7 as ca_county)], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price), avg(catalog_sales.cs_net_profit), avg(customer.c_birth_year), avg(cd1.cd_dep_count)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_quantity@1, cs_list_price@2, cs_sales_price@3, cs_coupon_amt@4, cs_net_profit@5, cd_dep_count@6, c_birth_year@7, ca_county@8, ca_state@9, ca_country@10, i_item_id@12] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_quantity@4, cs_list_price@5, cs_sales_price@6, cs_coupon_amt@7, cs_net_profit@8, cd_dep_count@9, c_birth_year@10, ca_county@11, ca_state@12, ca_country@13] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, cs_sales_price@7 as cs_sales_price, cs_coupon_amt@8 as cs_coupon_amt, cs_net_profit@9 as cs_net_profit, cd_dep_count@10 as cd_dep_count, c_birth_year@11 as c_birth_year, ca_county@0 as ca_county, ca_state@1 as ca_state, ca_country@2 as ca_country] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@8)], projection=[ca_county@1, ca_state@2, ca_country@3, cs_sold_date_sk@4, cs_item_sk@5, cs_quantity@6, cs_list_price@7, cs_sales_price@8, cs_coupon_amt@9, cs_net_profit@10, cd_dep_count@11, c_birth_year@13] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_current_cdemo_sk@8, CAST(cd2.cd_demo_sk AS Float64)@1)], projection=[cs_sold_date_sk@0, cs_item_sk@1, cs_quantity@2, cs_list_price@3, cs_sales_price@4, cs_coupon_amt@5, cs_net_profit@6, cd_dep_count@7, c_current_addr_sk@9, c_birth_year@10] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_quantity@4, cs_list_price@5, cs_sales_price@6, cs_coupon_amt@7, cs_net_profit@8, cd_dep_count@9, c_birth_year@10, ca_county@11, ca_state@12, ca_country@13] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, cs_sales_price@7 as cs_sales_price, cs_coupon_amt@8 as cs_coupon_amt, cs_net_profit@9 as cs_net_profit, cd_dep_count@10 as cd_dep_count, c_birth_year@11 as c_birth_year, ca_county@0 as ca_county, ca_state@1 as ca_state, ca_country@2 as ca_country] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@8)], projection=[ca_county@1, ca_state@2, ca_country@3, cs_sold_date_sk@4, cs_item_sk@5, cs_quantity@6, cs_list_price@7, cs_sales_price@8, cs_coupon_amt@9, cs_net_profit@10, cd_dep_count@11, c_birth_year@13] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_current_cdemo_sk@8, CAST(cd2.cd_demo_sk AS Float64)@1)], projection=[cs_sold_date_sk@0, cs_item_sk@1, cs_quantity@2, cs_list_price@3, cs_sales_price@4, cs_coupon_amt@5, cs_net_profit@6, cd_dep_count@7, c_current_addr_sk@9, c_birth_year@10] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1998, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 1998, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1, required_guarantees=[d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1998, projection=[d_date_sk@0] + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: ca_state@2 IN (SET) ([MS, IN, ND, OK, NM, VA, MS]) + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county, ca_state, ca_country], file_type=parquet, predicate=ca_state@8 IN (SET) ([MS, IN, ND, OK, NM, VA, MS]), pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= ND AND ND <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OK AND OK <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NM AND NM <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1, required_guarantees=[ca_state in (IN, MS, ND, NM, OK, VA)] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_current_cdemo_sk@8], 12), input_partitions=3 + │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, cs_sales_price@7 as cs_sales_price, cs_coupon_amt@8 as cs_coupon_amt, cs_net_profit@9 as cs_net_profit, cd_dep_count@10 as cd_dep_count, c_current_cdemo_sk@0 as c_current_cdemo_sk, c_current_addr_sk@1 as c_current_addr_sk, c_birth_year@2 as c_birth_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, cs_bill_customer_sk@1)], projection=[c_current_cdemo_sk@1, c_current_addr_sk@2, c_birth_year@3, cs_sold_date_sk@5, cs_item_sk@7, cs_quantity@8, cs_list_price@9, cs_sales_price@10, cs_coupon_amt@11, cs_net_profit@12, cd_dep_count@13] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=3 + │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_bill_customer_sk@2 as cs_bill_customer_sk, cs_item_sk@3 as cs_item_sk, cs_quantity@4 as cs_quantity, cs_list_price@5 as cs_list_price, cs_sales_price@6 as cs_sales_price, cs_coupon_amt@7 as cs_coupon_amt, cs_net_profit@8 as cs_net_profit, cd_dep_count@0 as cd_dep_count] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(cd1.cd_demo_sk AS Float64)@2, cs_bill_cdemo_sk@2)], projection=[cd_dep_count@1, cs_sold_date_sk@3, cs_bill_customer_sk@4, cs_item_sk@6, cs_quantity@7, cs_list_price@8, cs_sales_price@9, cs_coupon_amt@10, cs_net_profit@11] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_addr_sk@2 as c_current_addr_sk, c_birth_year@3 as c_birth_year, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ FilterExec: c_birth_month@3 IN (SET) ([1, 6, 8, 9, 12, 2]), projection=[c_customer_sk@0, c_current_cdemo_sk@1, c_current_addr_sk@2, c_birth_year@4] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 1998, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1, required_guarantees=[d_year in (1998)] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk, c_birth_month, c_birth_year], file_type=parquet, predicate=c_birth_month@12 IN (SET) ([1, 6, 8, 9, 12, 2]) AND DynamicFilter [ empty ], pruning_predicate=c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 1 AND 1 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 6 AND 6 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 8 AND 8 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 9 AND 9 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 12 AND 12 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 2 AND 2 <= c_birth_month_max@1, required_guarantees=[c_birth_month in (1, 12, 2, 6, 8, 9)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: ca_state@2 IN (SET) ([MS, IN, ND, OK, NM, VA, MS]) - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county, ca_state, ca_country], file_type=parquet, predicate=ca_state@8 IN (SET) ([MS, IN, ND, OK, NM, VA, MS]), pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= ND AND ND <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OK AND OK <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NM AND NM <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1, required_guarantees=[ca_state in (IN, MS, ND, NM, OK, VA)] + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(cd1.cd_demo_sk AS Float64)@2], 12), input_partitions=2 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_dep_count@1 as cd_dep_count, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)] + │ FilterExec: cd_gender@1 = F AND cd_education_status@2 = Unknown, projection=[cd_demo_sk@0, cd_dep_count@3] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_education_status, cd_dep_count], file_type=parquet, predicate=cd_gender@1 = F AND cd_education_status@3 = Unknown, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= F AND F <= cd_gender_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Unknown AND Unknown <= cd_education_status_max@5, required_guarantees=[cd_education_status in (Unknown), cd_gender in (F)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([c_current_cdemo_sk@8], 9), input_partitions=3 - │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, cs_sales_price@7 as cs_sales_price, cs_coupon_amt@8 as cs_coupon_amt, cs_net_profit@9 as cs_net_profit, cd_dep_count@10 as cd_dep_count, c_current_cdemo_sk@0 as c_current_cdemo_sk, c_current_addr_sk@1 as c_current_addr_sk, c_birth_year@2 as c_birth_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, cs_bill_customer_sk@1)], projection=[c_current_cdemo_sk@1, c_current_addr_sk@2, c_birth_year@3, cs_sold_date_sk@5, cs_item_sk@7, cs_quantity@8, cs_list_price@9, cs_sales_price@10, cs_coupon_amt@11, cs_net_profit@12, cd_dep_count@13] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_bill_customer_sk@2 as cs_bill_customer_sk, cs_item_sk@3 as cs_item_sk, cs_quantity@4 as cs_quantity, cs_list_price@5 as cs_list_price, cs_sales_price@6 as cs_sales_price, cs_coupon_amt@7 as cs_coupon_amt, cs_net_profit@8 as cs_net_profit, cd_dep_count@0 as cd_dep_count] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(cd1.cd_demo_sk AS Float64)@2, cs_bill_cdemo_sk@2)], projection=[cd_dep_count@1, cs_sold_date_sk@3, cs_bill_customer_sk@4, cs_item_sk@6, cs_quantity@7, cs_list_price@8, cs_sales_price@9, cs_coupon_amt@10, cs_net_profit@11] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_addr_sk@2 as c_current_addr_sk, c_birth_year@3 as c_birth_year, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ FilterExec: c_birth_month@3 IN (SET) ([1, 6, 8, 9, 12, 2]), projection=[c_customer_sk@0, c_current_cdemo_sk@1, c_current_addr_sk@2, c_birth_year@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk, c_birth_month, c_birth_year], file_type=parquet, predicate=c_birth_month@12 IN (SET) ([1, 6, 8, 9, 12, 2]) AND DynamicFilter [ empty ], pruning_predicate=c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 1 AND 1 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 6 AND 6 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 8 AND 8 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 9 AND 9 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 12 AND 12 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 2 AND 2 <= c_birth_month_max@1, required_guarantees=[c_birth_month in (1, 12, 2, 6, 8, 9)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(cd1.cd_demo_sk AS Float64)@2], 9), input_partitions=2 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_dep_count@1 as cd_dep_count, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)] - │ FilterExec: cd_gender@1 = F AND cd_education_status@2 = Unknown, projection=[cd_demo_sk@0, cd_dep_count@3] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_education_status, cd_dep_count], file_type=parquet, predicate=cd_gender@1 = F AND cd_education_status@3 = Unknown, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= F AND F <= cd_gender_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Unknown AND Unknown <= cd_education_status_max@5, required_guarantees=[cd_education_status in (Unknown), cd_gender in (F)] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_bill_cdemo_sk, cs_item_sk, cs_quantity, cs_list_price, cs_sales_price, cs_coupon_amt, cs_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(cd2.cd_demo_sk AS Float64)@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_bill_cdemo_sk, cs_item_sk, cs_quantity, cs_list_price, cs_sales_price, cs_coupon_amt, cs_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(cd2.cd_demo_sk AS Float64)@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── "); Ok(()) } @@ -2215,71 +2123,65 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ext_price@4 DESC, brand@1 ASC NULLS LAST, brand_id@0 ASC NULLS LAST, i_manufact_id@2 ASC NULLS LAST, i_manufact@3 ASC NULLS LAST], fetch=100 - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[ext_price@4 DESC, brand@1 ASC NULLS LAST, brand_id@0 ASC NULLS LAST, i_manufact_id@2 ASC NULLS LAST, i_manufact@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, i_manufact_id@2 as i_manufact_id, i_manufact@3 as i_manufact, sum(store_sales.ss_ext_sales_price)@4 as ext_price] + │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id, i_manufact_id@2 as i_manufact_id, i_manufact@3 as i_manufact], aggr=[sum(store_sales.ss_ext_sales_price)] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[ext_price@4 DESC, brand@1 ASC NULLS LAST, brand_id@0 ASC NULLS LAST, i_manufact_id@2 ASC NULLS LAST, i_manufact@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, i_manufact_id@2 as i_manufact_id, i_manufact@3 as i_manufact, sum(store_sales.ss_ext_sales_price)@4 as ext_price] - │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id, i_manufact_id@2 as i_manufact_id, i_manufact@3 as i_manufact], aggr=[sum(store_sales.ss_ext_sales_price)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1, i_manufact_id@2, i_manufact@3], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_brand@2 as i_brand, i_brand_id@1 as i_brand_id, i_manufact_id@3 as i_manufact_id, i_manufact@4 as i_manufact], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@0)], filter=substr(ca_zip@0, 1, 5) != substr(s_zip@1, 1, 5), projection=[ss_ext_sales_price@4, i_brand_id@5, i_brand@6, i_manufact_id@7, i_manufact@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_store_sk@1 as ss_store_sk, ss_ext_sales_price@2 as ss_ext_sales_price, i_brand_id@3 as i_brand_id, i_brand@4 as i_brand, i_manufact_id@5 as i_manufact_id, i_manufact@6 as i_manufact, ca_zip@0 as ca_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@6)], projection=[ca_zip@1, ss_store_sk@2, ss_ext_sales_price@3, i_brand_id@4, i_brand@5, i_manufact_id@6, i_manufact@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_store_sk@1, ss_ext_sales_price@2, i_brand_id@3, i_brand@4, i_manufact_id@5, i_manufact@6, c_current_addr_sk@8] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1, i_manufact_id@2, i_manufact@3], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_brand@2 as i_brand, i_brand_id@1 as i_brand_id, i_manufact_id@3 as i_manufact_id, i_manufact@4 as i_manufact], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@0)], filter=substr(ca_zip@0, 1, 5) != substr(s_zip@1, 1, 5), projection=[ss_ext_sales_price@4, i_brand_id@5, i_brand@6, i_manufact_id@7, i_manufact@8] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[ss_store_sk@1 as ss_store_sk, ss_ext_sales_price@2 as ss_ext_sales_price, i_brand_id@3 as i_brand_id, i_brand@4 as i_brand, i_manufact_id@5 as i_manufact_id, i_manufact@6 as i_manufact, ca_zip@0 as ca_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@6)], projection=[ca_zip@1, ss_store_sk@2, ss_ext_sales_price@3, i_brand_id@4, i_brand@5, i_manufact_id@6, i_manufact@7] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_store_sk@1, ss_ext_sales_price@2, i_brand_id@3, i_brand@4, i_manufact_id@5, i_manufact@6, c_current_addr_sk@8] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_zip], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 12), input_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_store_sk@2, ss_ext_sales_price@3, i_brand_id@5, i_brand@6, i_manufact_id@7, i_manufact@8] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ FilterExec: i_manager_id@5 = 8, projection=[i_item_sk@0, i_brand_id@1, i_brand@2, i_manufact_id@3, i_manufact@4] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_brand, i_manufact_id, i_manufact, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 8 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 8 AND 8 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (8)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_zip], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 9), input_partitions=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_store_sk@2, ss_ext_sales_price@3, i_brand_id@5, i_brand@6, i_manufact_id@7, i_manufact@8] + ┌───── Stage 4 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_customer_sk@4, ss_store_sk@5, ss_ext_sales_price@6] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ FilterExec: i_manager_id@5 = 8, projection=[i_item_sk@0, i_brand_id@1, i_brand@2, i_manufact_id@3, i_manufact@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_brand, i_manufact_id, i_manufact, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 8 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 8 AND 8 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (8)] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_customer_sk@4, ss_store_sk@5, ss_ext_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 = 11 AND d_year@1 = 1998, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1998, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 = 11 AND d_year@1 = 1998, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1998, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 9), input_partitions=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 12), input_partitions=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── "); Ok(()) } @@ -2289,45 +2191,47 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_category@2 ASC, i_class@3 ASC, i_item_id@0 ASC, i_item_desc@1 ASC, revenueratio@6 ASC], fetch=100 - │ SortExec: TopK(fetch=100), expr=[i_category@2 ASC, i_class@3 ASC, i_item_id@0 ASC, i_item_desc@1 ASC, revenueratio@6 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price, sum(catalog_sales.cs_ext_sales_price)@5 as itemrevenue, CAST(sum(catalog_sales.cs_ext_sales_price)@5 AS Float64) * 100 / CAST(sum(sum(catalog_sales.cs_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@6 AS Float64) as revenueratio] - │ WindowAggExec: wdw=[sum(sum(catalog_sales.cs_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(sum(catalog_sales.cs_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(27, 2), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] - │ SortExec: expr=[i_class@3 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_class@3], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_category@2, i_class@3, i_current_price@4], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id, i_item_desc@2 as i_item_desc, i_category@5 as i_category, i_class@4 as i_class, i_current_price@3 as i_current_price], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ext_sales_price@3, i_item_id@4, i_item_desc@5, i_current_price@6, i_class@7, i_category@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@5 as cs_sold_date_sk, cs_ext_sales_price@6 as cs_ext_sales_price, i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price, i_class@3 as i_class, i_category@4 as i_category] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3, i_class@4, i_category@5, cs_sold_date_sk@6, cs_ext_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[i_category@2 ASC, i_class@3 ASC, i_item_id@0 ASC, i_item_desc@1 ASC, revenueratio@6 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price, sum(catalog_sales.cs_ext_sales_price)@5 as itemrevenue, CAST(sum(catalog_sales.cs_ext_sales_price)@5 AS Float64) * 100 / CAST(sum(sum(catalog_sales.cs_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@6 AS Float64) as revenueratio] + │ WindowAggExec: wdw=[sum(sum(catalog_sales.cs_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(sum(catalog_sales.cs_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(27, 2), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] + │ SortExec: expr=[i_class@3 ASC NULLS LAST], preserve_partitioning=[true] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_class@3], 12), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_category@2, i_class@3, i_current_price@4], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id, i_item_desc@2 as i_item_desc, i_category@5 as i_category, i_class@4 as i_class, i_current_price@3 as i_current_price], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ext_sales_price@3, i_item_id@4, i_item_desc@5, i_current_price@6, i_class@7, i_category@8] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@5 as cs_sold_date_sk, cs_ext_sales_price@6 as cs_ext_sales_price, i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price, i_class@3 as i_class, i_category@4 as i_category] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3, i_class@4, i_category@5, cs_sold_date_sk@6, cs_ext_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] + └────────────────────────────────────────────────── "#); Ok(()) } @@ -2337,53 +2241,47 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [w_warehouse_name@0 ASC, i_item_id@1 ASC], fetch=100 - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[w_warehouse_name@0 ASC, i_item_id@1 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 as inv_before, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 as inv_after] + │ FilterExec: __common_expr_4@0 >= 0.6666666666666666 AND __common_expr_4@0 <= 1.5, projection=[w_warehouse_name@1, i_item_id@2, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@4] + │ ProjectionExec: expr=[CASE WHEN sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 > 0 THEN sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 / sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 END as __common_expr_4, w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 as sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 as sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] + │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[w_warehouse_name@0 ASC, i_item_id@1 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 as inv_before, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 as inv_after] - │ FilterExec: __common_expr_4@0 >= 0.6666666666666666 AND __common_expr_4@0 <= 1.5, projection=[w_warehouse_name@1, i_item_id@2, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@4] - │ ProjectionExec: expr=[CASE WHEN sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 > 0 THEN sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 / sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 END as __common_expr_4, w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 as sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 as sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] - │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, i_item_id@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[w_warehouse_name@2 as w_warehouse_name, i_item_id@3 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] - │ ProjectionExec: expr=[d_date@0 as __common_expr_2, inv_quantity_on_hand@1 as inv_quantity_on_hand, w_warehouse_name@2 as w_warehouse_name, i_item_id@3 as i_item_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[d_date@1, inv_quantity_on_hand@3, w_warehouse_name@4, i_item_id@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_quantity_on_hand@2 as inv_quantity_on_hand, w_warehouse_name@3 as w_warehouse_name, i_item_id@0 as i_item_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_id@1, inv_date_sk@2, inv_quantity_on_hand@4, w_warehouse_name@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_item_sk@2 as inv_item_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, w_warehouse_name@0 as w_warehouse_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@2)], projection=[w_warehouse_name@1, inv_date_sk@2, inv_item_sk@3, inv_quantity_on_hand@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, i_item_id@1], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[w_warehouse_name@2 as w_warehouse_name, i_item_id@3 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] + │ ProjectionExec: expr=[d_date@0 as __common_expr_2, inv_quantity_on_hand@1 as inv_quantity_on_hand, w_warehouse_name@2 as w_warehouse_name, i_item_id@3 as i_item_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[d_date@1, inv_quantity_on_hand@3, w_warehouse_name@4, i_item_id@5] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_quantity_on_hand@2 as inv_quantity_on_hand, w_warehouse_name@3 as w_warehouse_name, i_item_id@0 as i_item_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_id@1, inv_date_sk@2, inv_quantity_on_hand@4, w_warehouse_name@5] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_item_sk@2 as inv_item_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, w_warehouse_name@0 as w_warehouse_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@2)], projection=[w_warehouse_name@1, inv_date_sk@2, inv_item_sk@3, inv_quantity_on_hand@5] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: d_date@1 >= 2000-02-10 AND d_date@1 <= 2000-04-10 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-02-10 AND d_date@2 <= 2000-04-10, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-10 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-10, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_current_price@2 >= Some(99),4,2 AND i_current_price@2 <= Some(149),4,2, projection=[i_item_sk@0, i_item_id@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_current_price], file_type=parquet, predicate=i_current_price@5 >= Some(99),4,2 AND i_current_price@5 <= Some(149),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(99),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(149),4,2, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_date@1 >= 2000-02-10 AND d_date@1 <= 2000-04-10 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-02-10 AND d_date@2 <= 2000-04-10, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-10 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-10, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@2 >= Some(99),4,2 AND i_current_price@2 <= Some(149),4,2, projection=[i_item_sk@0, i_item_id@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_current_price], file_type=parquet, predicate=i_current_price@5 >= Some(99),4,2 AND i_current_price@5 <= Some(149),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(99),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(149),4,2, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet - └────────────────────────────────────────────────── "#); Ok(()) } @@ -2393,38 +2291,36 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [qoh@4 ASC, i_product_name@0 ASC, i_brand@1 ASC, i_class@2 ASC, i_category@3 ASC], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[qoh@4 ASC, i_product_name@0 ASC, i_brand@1 ASC, i_class@2 ASC, i_category@3 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[i_product_name@0 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category, avg(inventory.inv_quantity_on_hand)@5 as qoh] │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category, __grouping_id@4 as __grouping_id], aggr=[avg(inventory.inv_quantity_on_hand)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_product_name@0, i_brand@1, i_class@2, i_category@3, __grouping_id@4], 6), input_partitions=2 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_product_name@0, i_brand@1, i_class@2, i_category@3, __grouping_id@4], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[(NULL as i_product_name, NULL as i_brand, NULL as i_class, NULL as i_category), (i_product_name@4 as i_product_name, NULL as i_brand, NULL as i_class, NULL as i_category), (i_product_name@4 as i_product_name, i_brand@1 as i_brand, NULL as i_class, NULL as i_category), (i_product_name@4 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, NULL as i_category), (i_product_name@4 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category)], aggr=[avg(inventory.inv_quantity_on_hand)] │ ProjectionExec: expr=[inv_quantity_on_hand@4 as inv_quantity_on_hand, i_brand@0 as i_brand, i_class@1 as i_class, i_category@2 as i_category, i_product_name@3 as i_product_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@0)], projection=[i_brand@1, i_class@2, i_category@3, i_product_name@4, inv_quantity_on_hand@6] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[inv_item_sk@2, inv_quantity_on_hand@3] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_product_name], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_product_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -2435,296 +2331,281 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, sales@2 ASC], fetch=100 - │ [Stage 33] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, c_first_name@1 ASC, sales@2 ASC], preserve_partitioning=[true] + │ InterleaveExec + │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)@2 as sales] + │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name], aggr=[sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)] + │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, sum(web_sales.ws_quantity * web_sales.ws_list_price)@2 as sales] + │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name], aggr=[sum(web_sales.ws_quantity * web_sales.ws_list_price)] + │ [Stage 32] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 33 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, c_first_name@1 ASC, sales@2 ASC], preserve_partitioning=[true] - │ InterleaveExec - │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)@2 as sales] - │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name], aggr=[sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)] - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, sum(web_sales.ws_quantity * web_sales.ws_list_price)@2 as sales] - │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name], aggr=[sum(web_sales.ws_quantity * web_sales.ws_list_price)] - │ [Stage 32] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_last_name@3 as c_last_name, c_first_name@2 as c_first_name], aggr=[sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@0, CAST(best_ss_customer.c_customer_sk AS Float64)@1)], projection=[cs_quantity@1, cs_list_price@2, c_first_name@3, c_last_name@4] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=1 - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@0], 9), input_partitions=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, item_sk@0)], projection=[cs_bill_customer_sk@0, cs_quantity@2, cs_list_price@3, c_first_name@4, c_last_name@5] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_last_name@3 as c_last_name, c_first_name@2 as c_first_name], aggr=[sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@0, CAST(best_ss_customer.c_customer_sk AS Float64)@1)], projection=[cs_quantity@1, cs_list_price@2, c_first_name@3, c_last_name@4] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@0], 12), input_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, item_sk@0)], projection=[cs_bill_customer_sk@0, cs_quantity@2, cs_list_price@3, c_first_name@4, c_last_name@5] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[i_item_sk@0 as item_sk] + │ FilterExec: count(Int64(1))@1 > 4, projection=[i_item_sk@0] + │ ProjectionExec: expr=[i_item_sk@1 as i_item_sk, count(Int64(1))@3 as count(Int64(1))] + │ AggregateExec: mode=FinalPartitioned, gby=[itemdesc@0 as itemdesc, i_item_sk@1 as i_item_sk, d_date@2 as d_date], aggr=[count(Int64(1))] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_customer_sk@3, cs_item_sk@4, cs_quantity@5, cs_list_price@6, c_first_name@7, c_last_name@8] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[i_item_sk@0 as item_sk] - │ FilterExec: count(Int64(1))@1 > 4, projection=[i_item_sk@0] - │ ProjectionExec: expr=[i_item_sk@1 as i_item_sk, count(Int64(1))@3 as count(Int64(1))] - │ AggregateExec: mode=FinalPartitioned, gby=[itemdesc@0 as itemdesc, i_item_sk@1 as i_item_sk, d_date@2 as d_date], aggr=[count(Int64(1))] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_customer_sk@3, cs_item_sk@4, cs_quantity@5, cs_list_price@6, c_first_name@7, c_last_name@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_bill_customer_sk@3 as cs_bill_customer_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, c_first_name@0 as c_first_name, c_last_name@1 as c_last_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, cs_bill_customer_sk@1)], projection=[c_first_name@1, c_last_name@2, cs_sold_date_sk@4, cs_bill_customer_sk@5, cs_item_sk@6, cs_quantity@7, cs_list_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_bill_customer_sk@3 as cs_bill_customer_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, c_first_name@0 as c_first_name, c_last_name@1 as c_last_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, cs_bill_customer_sk@1)], projection=[c_first_name@1, c_last_name@2, cs_sold_date_sk@4, cs_bill_customer_sk@5, cs_item_sk@6, cs_quantity@7, cs_list_price@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2000 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000 AND d_moy@2 = 2, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2000 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2000)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([itemdesc@0, i_item_sk@1, d_date@2], 9), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[itemdesc@1 as itemdesc, i_item_sk@2 as i_item_sk, d_date@0 as d_date], aggr=[count(Int64(1))] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@1)], projection=[d_date@1, itemdesc@2, i_item_sk@3] + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([itemdesc@0, i_item_sk@1, d_date@2], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[itemdesc@1 as itemdesc, i_item_sk@2 as i_item_sk, d_date@0 as d_date], aggr=[count(Int64(1))] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@1)], projection=[d_date@1, itemdesc@2, i_item_sk@3] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[substr(i_item_desc@1, 1, 30) as itemdesc, i_item_sk@0 as i_item_sk] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[substr(i_item_desc@1, 1, 30) as itemdesc, i_item_sk@0 as i_item_sk] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@2 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0, d_date@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@2 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0, d_date@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(best_ss_customer.c_customer_sk AS Float64)@1], 9), input_partitions=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(best_ss_customer.c_customer_sk AS Float64)] - │ FilterExec: sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 > 0.5 * max(max_store_sales.tpcds_cmax)@2, projection=[c_customer_sk@0] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price), max(max_store_sales.tpcds_cmax)] - │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_customer_sk@2 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price), max(max_store_sales.tpcds_cmax)] - │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@3 as c_customer_sk, tpcds_cmax@0 as tpcds_cmax] - │ CrossJoinExec - │ ProjectionExec: expr=[max(sq2.csales)@0 as tpcds_cmax] - │ AggregateExec: mode=Final, gby=[], aggr=[max(sq2.csales)] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@0 as c_customer_sk] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@0)], projection=[c_customer_sk@0, ss_quantity@3, ss_sales_price@4] - │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 15 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(best_ss_customer.c_customer_sk AS Float64)@1], 12), input_partitions=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(best_ss_customer.c_customer_sk AS Float64)] + │ FilterExec: sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 > 0.5 * max(max_store_sales.tpcds_cmax)@2, projection=[c_customer_sk@0] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price), max(max_store_sales.tpcds_cmax)] + │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_customer_sk@2 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price), max(max_store_sales.tpcds_cmax)] + │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@3 as c_customer_sk, tpcds_cmax@0 as tpcds_cmax] + │ CrossJoinExec + │ ProjectionExec: expr=[max(sq2.csales)@0 as tpcds_cmax] + │ AggregateExec: mode=Final, gby=[], aggr=[max(sq2.csales)] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@0 as c_customer_sk] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@0)], projection=[c_customer_sk@0, ss_quantity@3, ss_sales_price@4] + │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 3), input_partitions=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet + │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ AggregateExec: mode=Partial, gby=[], aggr=[max(sq2.csales)] │ ProjectionExec: expr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 as csales] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 6), input_partitions=3 + ┌───── Stage 11 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_customer_sk@2 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_sales_price@4, c_customer_sk@5] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_quantity@2 as ss_quantity, ss_sales_price@3 as ss_sales_price, c_customer_sk@0 as c_customer_sk] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@1)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_quantity@4, ss_sales_price@5] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 9), input_partitions=3 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 12), input_partitions=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 3), input_partitions=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_customer_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 32 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_last_name@3 as c_last_name, c_first_name@2 as c_first_name], aggr=[sum(web_sales.ws_quantity * web_sales.ws_list_price)] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ws_bill_customer_sk@0, CAST(best_ss_customer.c_customer_sk AS Float64)@1)], projection=[ws_quantity@1, ws_list_price@2, c_first_name@3, c_last_name@4] - │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 31] => NetworkShuffleExec: output_partitions=3, input_tasks=1 - └────────────────────────────────────────────────── - ┌───── Stage 23 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@0], 9), input_partitions=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, item_sk@0)], projection=[ws_bill_customer_sk@1, ws_quantity@2, ws_list_price@3, c_first_name@4, c_last_name@5] + ┌───── Stage 32 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_last_name@3 as c_last_name, c_first_name@2 as c_first_name], aggr=[sum(web_sales.ws_quantity * web_sales.ws_list_price)] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ws_bill_customer_sk@0, CAST(best_ss_customer.c_customer_sk AS Float64)@1)], projection=[ws_quantity@1, ws_list_price@2, c_first_name@3, c_last_name@4] + │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 31] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 23 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@0], 12), input_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, item_sk@0)], projection=[ws_bill_customer_sk@1, ws_quantity@2, ws_list_price@3, c_first_name@4, c_last_name@5] + │ CoalescePartitionsExec + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[i_item_sk@0 as item_sk] + │ FilterExec: count(Int64(1))@1 > 4, projection=[i_item_sk@0] + │ ProjectionExec: expr=[i_item_sk@1 as i_item_sk, count(Int64(1))@3 as count(Int64(1))] + │ AggregateExec: mode=FinalPartitioned, gby=[itemdesc@0 as itemdesc, i_item_sk@1 as i_item_sk, d_date@2 as d_date], aggr=[count(Int64(1))] + │ [Stage 22] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 19 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_bill_customer_sk@4, ws_quantity@5, ws_list_price@6, c_first_name@7, c_last_name@8] │ CoalescePartitionsExec - │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[i_item_sk@0 as item_sk] - │ FilterExec: count(Int64(1))@1 > 4, projection=[i_item_sk@0] - │ ProjectionExec: expr=[i_item_sk@1 as i_item_sk, count(Int64(1))@3 as count(Int64(1))] - │ AggregateExec: mode=FinalPartitioned, gby=[itemdesc@0 as itemdesc, i_item_sk@1 as i_item_sk, d_date@2 as d_date], aggr=[count(Int64(1))] - │ [Stage 22] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_bill_customer_sk@4, ws_quantity@5, ws_list_price@6, c_first_name@7, c_last_name@8] - │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_bill_customer_sk@4 as ws_bill_customer_sk, ws_quantity@5 as ws_quantity, ws_list_price@6 as ws_list_price, c_first_name@0 as c_first_name, c_last_name@1 as c_last_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, ws_bill_customer_sk@2)], projection=[c_first_name@1, c_last_name@2, ws_sold_date_sk@4, ws_item_sk@5, ws_bill_customer_sk@6, ws_quantity@7, ws_list_price@8] - │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_customer_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_bill_customer_sk@4 as ws_bill_customer_sk, ws_quantity@5 as ws_quantity, ws_list_price@6 as ws_list_price, c_first_name@0 as c_first_name, c_last_name@1 as c_last_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, ws_bill_customer_sk@2)], projection=[c_first_name@1, c_last_name@2, ws_sold_date_sk@4, ws_item_sk@5, ws_bill_customer_sk@6, ws_quantity@7, ws_list_price@8] + │ CoalescePartitionsExec + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_customer_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 17 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2000 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000 AND d_moy@2 = 2, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2000 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2000)] - └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([itemdesc@0, i_item_sk@1, d_date@2], 9), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[itemdesc@1 as itemdesc, i_item_sk@2 as i_item_sk, d_date@0 as d_date], aggr=[count(Int64(1))] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@1)], projection=[d_date@1, itemdesc@2, i_item_sk@3] + ┌───── Stage 18 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 22 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([itemdesc@0, i_item_sk@1, d_date@2], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[itemdesc@1 as itemdesc, i_item_sk@2 as i_item_sk, d_date@0 as d_date], aggr=[count(Int64(1))] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@1)], projection=[d_date@1, itemdesc@2, i_item_sk@3] + │ CoalescePartitionsExec + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[substr(i_item_desc@1, 1, 30) as itemdesc, i_item_sk@0 as i_item_sk] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 21 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4] │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[substr(i_item_desc@1, 1, 30) as itemdesc, i_item_sk@0 as i_item_sk] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4] - │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 20 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@2 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0, d_date@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@2 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0, d_date@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] - └────────────────────────────────────────────────── - ┌───── Stage 31 ── Tasks: t0:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(best_ss_customer.c_customer_sk AS Float64)@1], 9), input_partitions=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(best_ss_customer.c_customer_sk AS Float64)] - │ FilterExec: sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 > 0.5 * max(max_store_sales.tpcds_cmax)@2, projection=[c_customer_sk@0] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price), max(max_store_sales.tpcds_cmax)] - │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_customer_sk@2 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price), max(max_store_sales.tpcds_cmax)] - │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@3 as c_customer_sk, tpcds_cmax@0 as tpcds_cmax] - │ CrossJoinExec - │ ProjectionExec: expr=[max(sq2.csales)@0 as tpcds_cmax] - │ AggregateExec: mode=Final, gby=[], aggr=[max(sq2.csales)] - │ CoalescePartitionsExec - │ [Stage 28] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@0 as c_customer_sk] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@0)], projection=[c_customer_sk@0, ss_quantity@3, ss_sales_price@4] - │ [Stage 29] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 30] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 28 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 31 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(best_ss_customer.c_customer_sk AS Float64)@1], 12), input_partitions=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(best_ss_customer.c_customer_sk AS Float64)] + │ FilterExec: sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 > 0.5 * max(max_store_sales.tpcds_cmax)@2, projection=[c_customer_sk@0] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price), max(max_store_sales.tpcds_cmax)] + │ [Stage 30] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 30 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_customer_sk@2 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price), max(max_store_sales.tpcds_cmax)] + │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@3 as c_customer_sk, tpcds_cmax@0 as tpcds_cmax] + │ CrossJoinExec + │ ProjectionExec: expr=[max(sq2.csales)@0 as tpcds_cmax] + │ AggregateExec: mode=Final, gby=[], aggr=[max(sq2.csales)] + │ CoalescePartitionsExec + │ [Stage 28] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@0 as c_customer_sk] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@0)], projection=[c_customer_sk@0, ss_quantity@3, ss_sales_price@4] + │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 3), input_partitions=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet + │ [Stage 29] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 28 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ AggregateExec: mode=Partial, gby=[], aggr=[max(sq2.csales)] │ ProjectionExec: expr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 as csales] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)] - │ [Stage 27] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 27] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 27 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 6), input_partitions=3 + ┌───── Stage 27 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_customer_sk@2 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_sales_price@4, c_customer_sk@5] │ CoalescePartitionsExec - │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_quantity@2 as ss_quantity, ss_sales_price@3 as ss_sales_price, c_customer_sk@0 as c_customer_sk] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@1)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_quantity@4, ss_sales_price@5] - │ [Stage 25] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 26] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 25] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 26] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 24 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] └────────────────────────────────────────────────── - ┌───── Stage 25 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 9), input_partitions=3 + ┌───── Stage 25 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 12), input_partitions=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 26 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + ┌───── Stage 26 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 29 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 3), input_partitions=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 30 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 29 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_customer_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -2741,140 +2622,124 @@ mod tests { │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, sum(ssales.netpaid)@3 as sum(ssales.netpaid), CAST(sum(ssales.netpaid)@3 AS Decimal128(38, 15)) as join_proj_push_down_1] │ CoalescePartitionsExec │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name], aggr=[sum(ssales.netpaid)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, s_store_name@2], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name], aggr=[sum(ssales.netpaid)] + │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, sum(store_sales.ss_net_paid)@10 as netpaid] + │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, ca_state@3 as ca_state, s_state@4 as s_state, i_color@5 as i_color, i_current_price@6 as i_current_price, i_manager_id@7 as i_manager_id, i_units@8 as i_units, i_size@9 as i_size], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([5]) + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, s_store_name@2, ca_state@3, s_state@4, i_color@5, i_current_price@6, i_manager_id@7, i_units@8, i_size@9], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_last_name@9 as c_last_name, c_first_name@8 as c_first_name, s_store_name@1 as s_store_name, ca_state@10 as ca_state, s_state@2 as s_state, i_color@5 as i_color, i_current_price@3 as i_current_price, i_manager_id@7 as i_manager_id, i_units@6 as i_units, i_size@4 as i_size], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([5]) + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@9, ca_address_sk@0), (s_zip@3, ca_zip@2)], filter=c_birth_country@0 != CAST(upper(ca_country@1) AS Utf8View), projection=[ss_net_paid@0, s_store_name@1, s_state@2, i_current_price@4, i_size@5, i_color@6, i_units@7, i_manager_id@8, c_first_name@10, c_last_name@11, ca_state@14] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip, ca_country], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[CAST(0.05 * CAST(avg(ssales.netpaid)@0 AS Float64) AS Decimal128(38, 15)) as Float64(0.05) * avg(ssales.netpaid)] │ AggregateExec: mode=Final, gby=[], aggr=[avg(ssales.netpaid)] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(ssales.netpaid)] + │ ProjectionExec: expr=[sum(store_sales.ss_net_paid)@10 as netpaid] + │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, ca_state@3 as ca_state, s_state@4 as s_state, i_color@5 as i_color, i_current_price@6 as i_current_price, i_manager_id@7 as i_manager_id, i_units@8 as i_units, i_size@9 as i_size], aggr=[sum(store_sales.ss_net_paid)] + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, s_store_name@2, ca_state@3, s_state@4, i_color@5, i_current_price@6, i_manager_id@7, i_units@8, i_size@9], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_last_name@9 as c_last_name, c_first_name@8 as c_first_name, s_store_name@1 as s_store_name, ca_state@10 as ca_state, s_state@2 as s_state, i_color@5 as i_color, i_current_price@3 as i_current_price, i_manager_id@7 as i_manager_id, i_units@6 as i_units, i_size@4 as i_size], aggr=[sum(store_sales.ss_net_paid)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@9, ca_address_sk@0), (s_zip@3, ca_zip@2)], filter=c_birth_country@0 != CAST(upper(ca_country@1) AS Utf8View), projection=[ss_net_paid@0, s_store_name@1, s_state@2, i_current_price@4, i_size@5, i_color@6, i_units@7, i_manager_id@8, c_first_name@10, c_last_name@11, ca_state@14] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip, ca_country], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, s_store_name@2], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name], aggr=[sum(ssales.netpaid)] - │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, sum(store_sales.ss_net_paid)@10 as netpaid] - │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, ca_state@3 as ca_state, s_state@4 as s_state, i_color@5 as i_color, i_current_price@6 as i_current_price, i_manager_id@7 as i_manager_id, i_units@8 as i_units, i_size@9 as i_size], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([5]) - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, s_store_name@2, ca_state@3, s_state@4, i_color@5, i_current_price@6, i_manager_id@7, i_units@8, i_size@9], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_last_name@9 as c_last_name, c_first_name@8 as c_first_name, s_store_name@1 as s_store_name, ca_state@10 as ca_state, s_state@2 as s_state, i_color@5 as i_color, i_current_price@3 as i_current_price, i_manager_id@7 as i_manager_id, i_units@6 as i_units, i_size@4 as i_size], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([5]) - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@9, ca_address_sk@0), (s_zip@3, ca_zip@2)], filter=c_birth_country@0 != CAST(upper(ca_country@1) AS Utf8View), projection=[ss_net_paid@0, s_store_name@1, s_state@2, i_current_price@4, i_size@5, i_color@6, i_units@7, i_manager_id@8, c_first_name@10, c_last_name@11, ca_state@14] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip, ca_country], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_net_paid@1, s_store_name@2, s_state@3, s_zip@4, i_current_price@5, i_size@6, i_color@7, i_units@8, i_manager_id@9, c_current_addr_sk@11, c_first_name@12, c_last_name@13, c_birth_country@14] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_birth_country@4 as c_birth_country, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name, c_birth_country], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_net_paid@2, s_store_name@3, s_state@4, s_zip@5, i_current_price@7, i_size@8, i_color@9, i_units@10, i_manager_id@11] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_color@3 = peach + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_net_paid@1, s_store_name@2, s_state@3, s_zip@4, i_current_price@5, i_size@6, i_color@7, i_units@8, i_manager_id@9, c_current_addr_sk@11, c_first_name@12, c_last_name@13, c_birth_country@14] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_birth_country@4 as c_birth_country, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name, c_birth_country], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] t3:[p18..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_net_paid@2, s_store_name@3, s_state@4, s_zip@5, i_current_price@7, i_size@8, i_color@9, i_units@10, i_manager_id@11] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ FilterExec: i_color@3 = peach + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_size, i_color, i_units, i_manager_id], file_type=parquet, predicate=i_color@17 = peach AND DynamicFilter [ empty ], pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= peach AND peach <= i_color_max@1, required_guarantees=[i_color in (peach)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_customer_sk@4 as ss_customer_sk, ss_net_paid@5 as ss_net_paid, s_store_name@0 as s_store_name, s_state@1 as s_state, s_zip@2 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@4, ss_store_sk@2)], projection=[s_store_name@1, s_state@2, s_zip@3, ss_item_sk@5, ss_customer_sk@6, ss_net_paid@8] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_state@2 as s_state, s_zip@3 as s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_market_id@2 = 8, projection=[s_store_sk@0, s_store_name@1, s_state@3, s_zip@4] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_size, i_color, i_units, i_manager_id], file_type=parquet, predicate=i_color@17 = peach AND DynamicFilter [ empty ], pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= peach AND peach <= i_color_max@1, required_guarantees=[i_color in (peach)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@10 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_customer_sk@4 as ss_customer_sk, ss_net_paid@5 as ss_net_paid, s_store_name@0 as s_store_name, s_state@1 as s_state, s_zip@2 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@4, ss_store_sk@2)], projection=[s_store_name@1, s_state@2, s_zip@3, ss_item_sk@5, ss_customer_sk@6, ss_net_paid@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_state@2 as s_state, s_zip@3 as s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_market_id@2 = 8, projection=[s_store_sk@0, s_store_name@1, s_state@3, s_zip@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@10 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@0], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(ssales.netpaid)] - │ ProjectionExec: expr=[sum(store_sales.ss_net_paid)@10 as netpaid] - │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, ca_state@3 as ca_state, s_state@4 as s_state, i_color@5 as i_color, i_current_price@6 as i_current_price, i_manager_id@7 as i_manager_id, i_units@8 as i_units, i_size@9 as i_size], aggr=[sum(store_sales.ss_net_paid)] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, s_store_name@2, ca_state@3, s_state@4, i_color@5, i_current_price@6, i_manager_id@7, i_units@8, i_size@9], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_last_name@9 as c_last_name, c_first_name@8 as c_first_name, s_store_name@1 as s_store_name, ca_state@10 as ca_state, s_state@2 as s_state, i_color@5 as i_color, i_current_price@3 as i_current_price, i_manager_id@7 as i_manager_id, i_units@6 as i_units, i_size@4 as i_size], aggr=[sum(store_sales.ss_net_paid)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@9, ca_address_sk@0), (s_zip@3, ca_zip@2)], filter=c_birth_country@0 != CAST(upper(ca_country@1) AS Utf8View), projection=[ss_net_paid@0, s_store_name@1, s_state@2, i_current_price@4, i_size@5, i_color@6, i_units@7, i_manager_id@8, c_first_name@10, c_last_name@11, ca_state@14] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip, ca_country], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 12), input_partitions=3 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@0], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_net_paid@1, s_store_name@2, s_state@3, s_zip@4, i_current_price@5, i_size@6, i_color@7, i_units@8, i_manager_id@9, c_current_addr_sk@11, c_first_name@12, c_last_name@13, c_birth_country@14] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_birth_country@4 as c_birth_country, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name, c_birth_country], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] t3:[p18..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_net_paid@2, s_store_name@3, s_state@4, s_zip@5, i_current_price@7, i_size@8, i_color@9, i_units@10, i_manager_id@11] + │ CoalescePartitionsExec + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_size, i_color, i_units, i_manager_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_net_paid@1, s_store_name@2, s_state@3, s_zip@4, i_current_price@5, i_size@6, i_color@7, i_units@8, i_manager_id@9, c_current_addr_sk@11, c_first_name@12, c_last_name@13, c_birth_country@14] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_birth_country@4 as c_birth_country, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name, c_birth_country], file_type=parquet + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_customer_sk@4 as ss_customer_sk, ss_net_paid@5 as ss_net_paid, s_store_name@0 as s_store_name, s_state@1 as s_state, s_zip@2 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@4, ss_store_sk@2)], projection=[s_store_name@1, s_state@2, s_zip@3, ss_item_sk@5, ss_customer_sk@6, ss_net_paid@8] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_net_paid@2, s_store_name@3, s_state@4, s_zip@5, i_current_price@7, i_size@8, i_color@9, i_units@10, i_manager_id@11] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_size, i_color, i_units, i_manager_id], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_state@2 as s_state, s_zip@3 as s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_market_id@2 = 8, projection=[s_store_sk@0, s_store_name@1, s_state@3, s_zip@4] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@10 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 12), input_partitions=3 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@0], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_customer_sk@4 as ss_customer_sk, ss_net_paid@5 as ss_net_paid, s_store_name@0 as s_store_name, s_state@1 as s_state, s_zip@2 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@4, ss_store_sk@2)], projection=[s_store_name@1, s_state@2, s_zip@3, ss_item_sk@5, ss_customer_sk@6, ss_net_paid@8] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_state@2 as s_state, s_zip@3 as s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_market_id@2 = 8, projection=[s_store_sk@0, s_store_name@1, s_state@3, s_zip@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@10 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@0], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── "); Ok(()) } @@ -2884,95 +2749,90 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name, sum(store_sales.ss_net_profit)@4 as store_sales_profit, sum(store_returns.sr_net_loss)@5 as store_returns_loss, sum(catalog_sales.cs_net_profit)@6 as catalog_sales_profit] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name], aggr=[sum(store_sales.ss_net_profit), sum(store_returns.sr_net_loss), sum(catalog_sales.cs_net_profit)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name, sum(store_sales.ss_net_profit)@4 as store_sales_profit, sum(store_returns.sr_net_loss)@5 as store_returns_loss, sum(catalog_sales.cs_net_profit)@6 as catalog_sales_profit] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name], aggr=[sum(store_sales.ss_net_profit), sum(store_returns.sr_net_loss), sum(catalog_sales.cs_net_profit)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_store_id@2, s_store_name@3], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@5 as i_item_id, i_item_desc@6 as i_item_desc, s_store_id@3 as s_store_id, s_store_name@4 as s_store_name], aggr=[sum(store_sales.ss_net_profit), sum(store_returns.sr_net_loss), sum(catalog_sales.cs_net_profit)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_net_profit@1, sr_net_loss@2, cs_net_profit@3, s_store_id@4, s_store_name@5, i_item_id@7, i_item_desc@8] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_store_id@2, s_store_name@3], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@5 as i_item_id, i_item_desc@6 as i_item_desc, s_store_id@3 as s_store_id, s_store_name@4 as s_store_name], aggr=[sum(store_sales.ss_net_profit), sum(store_returns.sr_net_loss), sum(catalog_sales.cs_net_profit)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_net_profit@1, sr_net_loss@2, cs_net_profit@3, s_store_id@4, s_store_name@5, i_item_id@7, i_item_desc@8] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_net_profit@3 as ss_net_profit, sr_net_loss@4 as sr_net_loss, cs_net_profit@5 as cs_net_profit, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, ss_item_sk@4, ss_net_profit@6, sr_net_loss@7, cs_net_profit@8] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_net_profit@3 as ss_net_profit, sr_net_loss@4 as sr_net_loss, cs_net_profit@5 as cs_net_profit, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, ss_item_sk@4, ss_net_profit@6, sr_net_loss@7, cs_net_profit@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@4, CAST(d3.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_net_profit@2, sr_net_loss@3, cs_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 >= 4 AND d_moy@2 <= 10 AND d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 >= 4 AND d_moy@8 <= 10 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 4 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 10 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(sr_returned_date_sk@3, CAST(d2.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_net_profit@2, sr_net_loss@4, cs_sold_date_sk@5, cs_net_profit@6] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@4, CAST(d3.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_net_profit@2, sr_net_loss@3, cs_net_profit@5] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 >= 4 AND d_moy@2 <= 10 AND d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 >= 4 AND d_moy@8 <= 10 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 4 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 10 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(sr_returned_date_sk@3, CAST(d2.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_net_profit@2, sr_net_loss@4, cs_sold_date_sk@5, cs_net_profit@6] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 >= 4 AND d_moy@2 <= 10 AND d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 >= 4 AND d_moy@8 <= 10 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 4 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 10 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_net_profit@5, sr_returned_date_sk@6, sr_net_loss@7, cs_sold_date_sk@8, cs_net_profit@9] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 >= 4 AND d_moy@2 <= 10 AND d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 >= 4 AND d_moy@8 <= 10 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 4 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 10 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_net_profit@5, sr_returned_date_sk@6, sr_net_loss@7, cs_sold_date_sk@8, cs_net_profit@9] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_net_profit@5 as ss_net_profit, sr_returned_date_sk@6 as sr_returned_date_sk, sr_net_loss@7 as sr_net_loss, cs_sold_date_sk@0 as cs_sold_date_sk, cs_net_profit@1 as cs_net_profit] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_net_profit@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_net_profit@7, sr_returned_date_sk@8, sr_net_loss@11] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_net_profit@5 as ss_net_profit, sr_returned_date_sk@6 as sr_returned_date_sk, sr_net_loss@7 as sr_net_loss, cs_sold_date_sk@0 as cs_sold_date_sk, cs_net_profit@1 as cs_net_profit] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_net_profit@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_net_profit@7, sr_returned_date_sk@8, sr_net_loss@11] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 = 4 AND d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 4 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 4 AND 4 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (4), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 = 4 AND d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 4 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 4 AND 4 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (4), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_net_profit], file_type=parquet + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_net_profit], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@6, sr_item_sk@5], 12), input_partitions=3 + │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_store_sk@6 as ss_store_sk, ss_net_profit@7 as ss_net_profit, sr_returned_date_sk@0 as sr_returned_date_sk, sr_item_sk@1 as sr_item_sk, sr_customer_sk@2 as sr_customer_sk, sr_net_loss@3 as sr_net_loss] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_customer_sk@2, ss_customer_sk@2), (sr_item_sk@1, ss_item_sk@1), (sr_ticket_number@3, ss_ticket_number@4)], projection=[sr_returned_date_sk@0, sr_item_sk@1, sr_customer_sk@2, sr_net_loss@4, ss_sold_date_sk@5, ss_item_sk@6, ss_store_sk@8, ss_net_profit@10] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number, sr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_customer_sk@6, sr_item_sk@5], 9), input_partitions=3 - │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_store_sk@6 as ss_store_sk, ss_net_profit@7 as ss_net_profit, sr_returned_date_sk@0 as sr_returned_date_sk, sr_item_sk@1 as sr_item_sk, sr_customer_sk@2 as sr_customer_sk, sr_net_loss@3 as sr_net_loss] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_customer_sk@2, ss_customer_sk@2), (sr_item_sk@1, ss_item_sk@1), (sr_ticket_number@3, ss_ticket_number@4)], projection=[sr_returned_date_sk@0, sr_item_sk@1, sr_customer_sk@2, sr_net_loss@4, ss_sold_date_sk@5, ss_item_sk@6, ss_store_sk@8, ss_net_profit@10] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number, sr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── "); Ok(()) } @@ -2982,64 +2842,59 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, avg(catalog_sales.cs_quantity)@1 as agg1, avg(catalog_sales.cs_list_price)@2 as agg2, avg(catalog_sales.cs_coupon_amt)@3 as agg3, avg(catalog_sales.cs_sales_price)@4 as agg4] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, avg(catalog_sales.cs_quantity)@1 as agg1, avg(catalog_sales.cs_list_price)@2 as agg2, avg(catalog_sales.cs_coupon_amt)@3 as agg3, avg(catalog_sales.cs_sales_price)@4 as agg4] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@4 as i_item_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, cs_promo_sk@0)], projection=[cs_quantity@3, cs_list_price@4, cs_sales_price@5, cs_coupon_amt@6, i_item_id@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_promo_sk@1, cs_quantity@2, cs_list_price@3, cs_sales_price@4, cs_coupon_amt@5, i_item_id@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@4 as i_item_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, cs_promo_sk@0)], projection=[cs_quantity@3, cs_list_price@4, cs_sales_price@5, cs_coupon_amt@6, i_item_id@7] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_promo_sk@1, cs_quantity@2, cs_list_price@3, cs_sales_price@4, cs_coupon_amt@5, i_item_id@7] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_email@1 = N OR p_channel_event@2 = N, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_email, p_channel_event], file_type=parquet, predicate=p_channel_email@9 = N OR p_channel_event@14 = N, pruning_predicate=p_channel_email_null_count@2 != row_count@3 AND p_channel_email_min@0 <= N AND N <= p_channel_email_max@1 OR p_channel_event_null_count@6 != row_count@3 AND p_channel_event_min@4 <= N AND N <= p_channel_event_max@5, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_email@1 = N OR p_channel_event@2 = N, projection=[p_promo_sk@0] + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_promo_sk@4, cs_quantity@5, cs_list_price@6, cs_sales_price@7, cs_coupon_amt@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, cs_bill_cdemo_sk@1)], projection=[cs_sold_date_sk@2, cs_item_sk@4, cs_promo_sk@5, cs_quantity@6, cs_list_price@7, cs_sales_price@8, cs_coupon_amt@9] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_email, p_channel_event], file_type=parquet, predicate=p_channel_email@9 = N OR p_channel_event@14 = N, pruning_predicate=p_channel_email_null_count@2 != row_count@3 AND p_channel_email_min@0 <= N AND N <= p_channel_email_max@1 OR p_channel_event_null_count@6 != row_count@3 AND p_channel_event_min@4 <= N AND N <= p_channel_event_max@5, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_promo_sk@4, cs_quantity@5, cs_list_price@6, cs_sales_price@7, cs_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, cs_bill_cdemo_sk@1)], projection=[cs_sold_date_sk@2, cs_item_sk@4, cs_promo_sk@5, cs_quantity@6, cs_list_price@7, cs_sales_price@8, cs_coupon_amt@9] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 12), input_partitions=2 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_cdemo_sk, cs_item_sk, cs_promo_sk, cs_quantity, cs_list_price, cs_sales_price, cs_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_cdemo_sk, cs_item_sk, cs_promo_sk, cs_quantity, cs_list_price, cs_sales_price, cs_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── "); Ok(()) } @@ -3049,175 +2904,166 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC, s_state@1 ASC], fetch=100 - │ [Stage 19] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC], preserve_partitioning=[true] + │ DistributedUnionExec: t0:[c0, c1, c2] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, CAST(s_state@1 AS Utf8) as s_state, 0 as g_state, avg(results.agg1)@2 as agg1, avg(results.agg2)@3 as agg2, avg(results.agg3)@4 as agg3, avg(results.agg4)@5 as agg4] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, s_state@1 as s_state], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)], ordering_mode=PartiallySorted([1]) + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, NULL as s_state, 1 as g_state, avg(results.agg1)@1 as agg1, avg(results.agg2)@2 as agg2, avg(results.agg3)@3 as agg3, avg(results.agg4)@4 as agg4] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[NULL as i_item_id, NULL as s_state, 1 as g_state, avg(results.agg1)@0 as agg1, avg(results.agg2)@1 as agg2, avg(results.agg3)@2 as agg3, avg(results.agg4)@3 as agg4] + │ AggregateExec: mode=Final, gby=[], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] + │ CoalescePartitionsExec + │ [Stage 18] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC], preserve_partitioning=[true] - │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] t3:[c2] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, CAST(s_state@1 AS Utf8) as s_state, 0 as g_state, avg(results.agg1)@2 as agg1, avg(results.agg2)@3 as agg2, avg(results.agg3)@4 as agg3, avg(results.agg4)@5 as agg4] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, s_state@1 as s_state], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)], ordering_mode=PartiallySorted([1]) - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, NULL as s_state, 1 as g_state, avg(results.agg1)@1 as agg1, avg(results.agg2)@2 as agg2, avg(results.agg3)@3 as agg3, avg(results.agg4)@4 as agg4] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ ProjectionExec: expr=[NULL as i_item_id, NULL as s_state, 1 as g_state, avg(results.agg1)@0 as agg1, avg(results.agg2)@1 as agg2, avg(results.agg3)@2 as agg3, avg(results.agg4)@3 as agg4] - │ AggregateExec: mode=Final, gby=[], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0, s_state@1], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@0 as i_item_id, s_state@1 as s_state], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)], ordering_mode=PartiallySorted([1]) + │ ProjectionExec: expr=[i_item_id@5 as i_item_id, s_state@4 as s_state, ss_quantity@0 as agg1, ss_list_price@1 as agg2, ss_coupon_amt@3 as agg3, ss_sales_price@2 as agg4] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, ss_sales_price@3, ss_coupon_amt@4, s_state@5, i_item_id@7] │ CoalescePartitionsExec - │ [Stage 18] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_item_id@0, s_state@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@0 as i_item_id, s_state@1 as s_state], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)], ordering_mode=PartiallySorted([1]) - │ ProjectionExec: expr=[i_item_id@5 as i_item_id, s_state@4 as s_state, ss_quantity@0 as agg1, ss_list_price@1 as agg2, ss_coupon_amt@3 as agg3, ss_sales_price@2 as agg4] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, ss_sales_price@3, ss_coupon_amt@4, s_state@5, i_item_id@7] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, ss_list_price@3 as ss_list_price, ss_sales_price@4 as ss_sales_price, ss_coupon_amt@5 as ss_coupon_amt, s_state@0 as s_state] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_state@1, ss_item_sk@3, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, ss_list_price@3 as ss_list_price, ss_sales_price@4 as ss_sales_price, ss_coupon_amt@5 as ss_coupon_amt, s_state@0 as s_state] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_state@1, ss_item_sk@3, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_state@1 as s_state, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_state@1 as s_state, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_store_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@0 as i_item_id], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] - │ ProjectionExec: expr=[i_item_id@4 as i_item_id, ss_quantity@0 as agg1, ss_list_price@1 as agg2, ss_coupon_amt@3 as agg3, ss_sales_price@2 as agg4] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, ss_sales_price@3, ss_coupon_amt@4, i_item_id@6] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 12), input_partitions=2 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_store_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@0 as i_item_id], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] + │ ProjectionExec: expr=[i_item_id@4 as i_item_id, ss_quantity@0 as agg1, ss_list_price@1 as agg2, ss_coupon_amt@3 as agg3, ss_sales_price@2 as agg4] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, ss_sales_price@3, ss_coupon_amt@4, i_item_id@6] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[ss_item_sk@2, ss_quantity@4, ss_list_price@5, ss_sales_price@6, ss_coupon_amt@7] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[ss_item_sk@2, ss_quantity@4, ss_list_price@5, ss_sales_price@6, ss_coupon_amt@7] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_store_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] - │ ProjectionExec: expr=[ss_quantity@0 as agg1, ss_list_price@1 as agg2, ss_coupon_amt@3 as agg3, ss_sales_price@2 as agg4] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, ss_sales_price@3, ss_coupon_amt@4] + ┌───── Stage 8 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 12), input_partitions=2 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_store_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] + │ ProjectionExec: expr=[ss_quantity@0 as agg1, ss_list_price@1 as agg2, ss_coupon_amt@3 as agg3, ss_sales_price@2 as agg4] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, ss_sales_price@3, ss_coupon_amt@4] + │ CoalescePartitionsExec + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 17 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[ss_item_sk@2, ss_quantity@4, ss_list_price@5, ss_sales_price@6, ss_coupon_amt@7] + │ CoalescePartitionsExec + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[ss_item_sk@2, ss_quantity@4, ss_list_price@5, ss_sales_price@6, ss_coupon_amt@7] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 13 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 15 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 12), input_partitions=2 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] + └────────────────────────────────────────────────── + ┌───── Stage 16 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_store_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] - └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_store_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── "); Ok(()) } @@ -3232,69 +3078,69 @@ mod tests { │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b6_lp, count(store_sales.ss_list_price)@1 as b6_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b6_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[b1_lp@3 as b1_lp, b1_cnt@4 as b1_cnt, b1_cntd@5 as b1_cntd, b2_lp@6 as b2_lp, b2_cnt@7 as b2_cnt, b2_cntd@8 as b2_cntd, b3_lp@9 as b3_lp, b3_cnt@10 as b3_cnt, b3_cntd@11 as b3_cntd, b4_lp@12 as b4_lp, b4_cnt@13 as b4_cnt, b4_cntd@14 as b4_cntd, b5_lp@0 as b5_lp, b5_cnt@1 as b5_cnt, b5_cntd@2 as b5_cntd] │ CrossJoinExec │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b5_lp, count(store_sales.ss_list_price)@1 as b5_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b5_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[b1_lp@3 as b1_lp, b1_cnt@4 as b1_cnt, b1_cntd@5 as b1_cntd, b2_lp@6 as b2_lp, b2_cnt@7 as b2_cnt, b2_cntd@8 as b2_cntd, b3_lp@9 as b3_lp, b3_cnt@10 as b3_cnt, b3_cntd@11 as b3_cntd, b4_lp@0 as b4_lp, b4_cnt@1 as b4_cnt, b4_cntd@2 as b4_cntd] │ CrossJoinExec │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b4_lp, count(store_sales.ss_list_price)@1 as b4_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b4_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[b1_lp@3 as b1_lp, b1_cnt@4 as b1_cnt, b1_cntd@5 as b1_cntd, b2_lp@6 as b2_lp, b2_cnt@7 as b2_cnt, b2_cntd@8 as b2_cntd, b3_lp@0 as b3_lp, b3_cnt@1 as b3_cnt, b3_cntd@2 as b3_cntd] │ CrossJoinExec │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b3_lp, count(store_sales.ss_list_price)@1 as b3_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b3_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ CrossJoinExec │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b1_lp, count(store_sales.ss_list_price)@1 as b1_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b1_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b2_lp, count(store_sales.ss_list_price)@1 as b2_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b2_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ FilterExec: ss_quantity@0 >= 26 AND ss_quantity@0 <= 30 AND (ss_list_price@2 >= Some(15400),5,2 AND ss_list_price@2 <= Some(16400),5,2 OR ss_coupon_amt@3 >= Some(732600),7,2 AND ss_coupon_amt@3 <= Some(832600),7,2 OR ss_wholesale_cost@1 >= Some(700),5,2 AND ss_wholesale_cost@1 <= Some(2700),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@10 >= 26 AND ss_quantity@10 <= 30 AND (ss_list_price@12 >= Some(15400),5,2 AND ss_list_price@12 <= Some(16400),5,2 OR ss_coupon_amt@19 >= Some(732600),7,2 AND ss_coupon_amt@19 <= Some(832600),7,2 OR ss_wholesale_cost@11 >= Some(700),5,2 AND ss_wholesale_cost@11 <= Some(2700),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 26 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 30 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(15400),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(16400),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(732600),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(832600),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(700),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(2700),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 25 AND (ss_list_price@2 >= Some(12200),5,2 AND ss_list_price@2 <= Some(13200),5,2 OR ss_coupon_amt@3 >= Some(83600),7,2 AND ss_coupon_amt@3 <= Some(183600),7,2 OR ss_wholesale_cost@1 >= Some(1700),5,2 AND ss_wholesale_cost@1 <= Some(3700),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@10 >= 21 AND ss_quantity@10 <= 25 AND (ss_list_price@12 >= Some(12200),5,2 AND ss_list_price@12 <= Some(13200),5,2 OR ss_coupon_amt@19 >= Some(83600),7,2 AND ss_coupon_amt@19 <= Some(183600),7,2 OR ss_wholesale_cost@11 >= Some(1700),5,2 AND ss_wholesale_cost@11 <= Some(3700),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 21 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 25 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(12200),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(13200),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(83600),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(183600),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(1700),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(3700),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ FilterExec: ss_quantity@0 >= 16 AND ss_quantity@0 <= 20 AND (ss_list_price@2 >= Some(13500),5,2 AND ss_list_price@2 <= Some(14500),5,2 OR ss_coupon_amt@3 >= Some(607100),7,2 AND ss_coupon_amt@3 <= Some(707100),7,2 OR ss_wholesale_cost@1 >= Some(3800),5,2 AND ss_wholesale_cost@1 <= Some(5800),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@10 >= 16 AND ss_quantity@10 <= 20 AND (ss_list_price@12 >= Some(13500),5,2 AND ss_list_price@12 <= Some(14500),5,2 OR ss_coupon_amt@19 >= Some(607100),7,2 AND ss_coupon_amt@19 <= Some(707100),7,2 OR ss_wholesale_cost@11 >= Some(3800),5,2 AND ss_wholesale_cost@11 <= Some(5800),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 16 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 20 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(13500),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(14500),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(607100),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(707100),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(3800),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(5800),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ FilterExec: ss_quantity@0 >= 11 AND ss_quantity@0 <= 15 AND (ss_list_price@2 >= Some(14200),5,2 AND ss_list_price@2 <= Some(15200),5,2 OR ss_coupon_amt@3 >= Some(1221400),7,2 AND ss_coupon_amt@3 <= Some(1321400),7,2 OR ss_wholesale_cost@1 >= Some(7900),5,2 AND ss_wholesale_cost@1 <= Some(9900),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@10 >= 11 AND ss_quantity@10 <= 15 AND (ss_list_price@12 >= Some(14200),5,2 AND ss_list_price@12 <= Some(15200),5,2 OR ss_coupon_amt@19 >= Some(1221400),7,2 AND ss_coupon_amt@19 <= Some(1321400),7,2 OR ss_wholesale_cost@11 >= Some(7900),5,2 AND ss_wholesale_cost@11 <= Some(9900),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 11 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 15 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(14200),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(15200),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(1221400),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(1321400),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(7900),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(9900),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ FilterExec: ss_quantity@0 >= 0 AND ss_quantity@0 <= 5 AND (ss_list_price@2 >= Some(800),5,2 AND ss_list_price@2 <= Some(1800),5,2 OR ss_coupon_amt@3 >= Some(45900),7,2 AND ss_coupon_amt@3 <= Some(145900),7,2 OR ss_wholesale_cost@1 >= Some(5700),5,2 AND ss_wholesale_cost@1 <= Some(7700),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@10 >= 0 AND ss_quantity@10 <= 5 AND (ss_list_price@12 >= Some(800),5,2 AND ss_list_price@12 <= Some(1800),5,2 OR ss_coupon_amt@19 >= Some(45900),7,2 AND ss_coupon_amt@19 <= Some(145900),7,2 OR ss_wholesale_cost@11 >= Some(5700),5,2 AND ss_wholesale_cost@11 <= Some(7700),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 0 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 5 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(800),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(1800),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(45900),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(145900),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(5700),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(7700),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ FilterExec: ss_quantity@0 >= 6 AND ss_quantity@0 <= 10 AND (ss_list_price@2 >= Some(9000),5,2 AND ss_list_price@2 <= Some(10000),5,2 OR ss_coupon_amt@3 >= Some(232300),7,2 AND ss_coupon_amt@3 <= Some(332300),7,2 OR ss_wholesale_cost@1 >= Some(3100),5,2 AND ss_wholesale_cost@1 <= Some(5100),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@10 >= 6 AND ss_quantity@10 <= 10 AND (ss_list_price@12 >= Some(9000),5,2 AND ss_list_price@12 <= Some(10000),5,2 OR ss_coupon_amt@19 >= Some(232300),7,2 AND ss_coupon_amt@19 <= Some(332300),7,2 OR ss_wholesale_cost@11 >= Some(3100),5,2 AND ss_wholesale_cost@11 <= Some(5100),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 6 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 10 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(9000),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(10000),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(232300),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(332300),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(3100),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(5100),5,2), required_guarantees=[] └────────────────────────────────────────────────── "); @@ -3306,95 +3152,89 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name, sum(store_sales.ss_quantity)@4 as store_sales_quantity, sum(store_returns.sr_return_quantity)@5 as store_returns_quantity, sum(catalog_sales.cs_quantity)@6 as catalog_sales_quantity] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name], aggr=[sum(store_sales.ss_quantity), sum(store_returns.sr_return_quantity), sum(catalog_sales.cs_quantity)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name, sum(store_sales.ss_quantity)@4 as store_sales_quantity, sum(store_returns.sr_return_quantity)@5 as store_returns_quantity, sum(catalog_sales.cs_quantity)@6 as catalog_sales_quantity] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name], aggr=[sum(store_sales.ss_quantity), sum(store_returns.sr_return_quantity), sum(catalog_sales.cs_quantity)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_store_id@2, s_store_name@3], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@5 as i_item_id, i_item_desc@6 as i_item_desc, s_store_id@3 as s_store_id, s_store_name@4 as s_store_name], aggr=[sum(store_sales.ss_quantity), sum(store_returns.sr_return_quantity), sum(catalog_sales.cs_quantity)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, sr_return_quantity@2, cs_quantity@3, s_store_id@4, s_store_name@5, i_item_id@7, i_item_desc@8] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_store_id@2, s_store_name@3], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@5 as i_item_id, i_item_desc@6 as i_item_desc, s_store_id@3 as s_store_id, s_store_name@4 as s_store_name], aggr=[sum(store_sales.ss_quantity), sum(store_returns.sr_return_quantity), sum(catalog_sales.cs_quantity)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, sr_return_quantity@2, cs_quantity@3, s_store_id@4, s_store_name@5, i_item_id@7, i_item_desc@8] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_quantity@3 as ss_quantity, sr_return_quantity@4 as sr_return_quantity, cs_quantity@5 as cs_quantity, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, ss_item_sk@4, ss_quantity@6, sr_return_quantity@7, cs_quantity@8] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_quantity@3 as ss_quantity, sr_return_quantity@4 as sr_return_quantity, cs_quantity@5 as cs_quantity, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, ss_item_sk@4, ss_quantity@6, sr_return_quantity@7, cs_quantity@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@4, CAST(d3.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_quantity@2, sr_return_quantity@3, cs_quantity@5] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (1999, 2000, 2001)] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, sr_returned_date_sk@3)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@6, cs_sold_date_sk@7, cs_quantity@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, sr_returned_date_sk@6, sr_return_quantity@7, cs_sold_date_sk@8, cs_quantity@9] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@4, CAST(d3.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_quantity@2, sr_return_quantity@3, cs_quantity@5] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (1999, 2000, 2001)] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, sr_returned_date_sk@3)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@6, cs_sold_date_sk@7, cs_quantity@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, sr_returned_date_sk@6, sr_return_quantity@7, cs_sold_date_sk@8, cs_quantity@9] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 >= 9 AND d_moy@2 <= 12 AND d_year@1 = 1999, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 >= 9 AND d_moy@8 <= 12 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 9 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 12 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_year in (1999)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 = 9 AND d_year@1 = 1999, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 9 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 9 AND 9 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (9), d_year in (1999)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_quantity], file_type=parquet + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 >= 9 AND d_moy@2 <= 12 AND d_year@1 = 1999, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 >= 9 AND d_moy@8 <= 12 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 9 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 12 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_year in (1999)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 = 9 AND d_year@1 = 1999, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 9 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 9 AND 9 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (9), d_year in (1999)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_quantity], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@6, sr_item_sk@5], 12), input_partitions=3 + │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_store_sk@6 as ss_store_sk, ss_quantity@7 as ss_quantity, sr_returned_date_sk@0 as sr_returned_date_sk, sr_item_sk@1 as sr_item_sk, sr_customer_sk@2 as sr_customer_sk, sr_return_quantity@3 as sr_return_quantity] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_customer_sk@2, ss_customer_sk@2), (sr_item_sk@1, ss_item_sk@1), (sr_ticket_number@3, ss_ticket_number@4)], projection=[sr_returned_date_sk@0, sr_item_sk@1, sr_customer_sk@2, sr_return_quantity@4, ss_sold_date_sk@5, ss_item_sk@6, ss_store_sk@8, ss_quantity@10] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number, sr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_customer_sk@6, sr_item_sk@5], 9), input_partitions=3 - │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_store_sk@6 as ss_store_sk, ss_quantity@7 as ss_quantity, sr_returned_date_sk@0 as sr_returned_date_sk, sr_item_sk@1 as sr_item_sk, sr_customer_sk@2 as sr_customer_sk, sr_return_quantity@3 as sr_return_quantity] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_customer_sk@2, ss_customer_sk@2), (sr_item_sk@1, ss_item_sk@1), (sr_ticket_number@3, ss_ticket_number@4)], projection=[sr_returned_date_sk@0, sr_item_sk@1, sr_customer_sk@2, sr_return_quantity@4, ss_sold_date_sk@5, ss_item_sk@6, ss_store_sk@8, ss_quantity@10] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number, sr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── "); Ok(()) } @@ -3411,229 +3251,223 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ca_county@0 ASC NULLS LAST] - │ [Stage 24] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 24] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 24 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: expr=[ca_county@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[ca_county@2 as ca_county, d_year@3 as d_year, __common_expr_1@0 / CAST(web_sales@6 AS Float64) as web_q1_q2_increase, __common_expr_2@1 / CAST(store_sales@4 AS Float64) as store_q1_q2_increase, CAST(web_sales@7 AS Float64) / __common_expr_1@0 as web_q2_q3_increase, CAST(store_sales@5 AS Float64) / __common_expr_2@1 as store_q2_q3_increase] │ ProjectionExec: expr=[CAST(web_sales@6 AS Float64) as __common_expr_1, CAST(store_sales@3 AS Float64) as __common_expr_2, ca_county@0 as ca_county, d_year@1 as d_year, store_sales@2 as store_sales, store_sales@4 as store_sales, web_sales@5 as web_sales, web_sales@7 as web_sales] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@5, ca_county@0)], filter=CASE WHEN web_sales@2 > Some(0),17,2 THEN CAST(web_sales@3 AS Float64) / CAST(web_sales@2 AS Float64) END > CASE WHEN store_sales@0 > Some(0),17,2 THEN CAST(store_sales@1 AS Float64) / CAST(store_sales@0 AS Float64) END, projection=[ca_county@0, d_year@1, store_sales@2, store_sales@3, store_sales@4, web_sales@6, web_sales@7, web_sales@9] │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(web_sales.ws_ext_sales_price)@3 as web_sales] │ AggregateExec: mode=FinalPartitioned, gby=[ca_county@0 as ca_county, d_qoy@1 as d_qoy, d_year@2 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 20 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@5, ca_county@0)], filter=CASE WHEN web_sales@2 > Some(0),17,2 THEN CAST(web_sales@3 AS Float64) / CAST(web_sales@2 AS Float64) END > CASE WHEN store_sales@0 > Some(0),17,2 THEN CAST(store_sales@1 AS Float64) / CAST(store_sales@0 AS Float64) END, projection=[ca_county@0, d_year@1, store_sales@2, store_sales@3, store_sales@4, ca_county@5, web_sales@6, web_sales@8] │ CoalescePartitionsExec - │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(web_sales.ws_ext_sales_price)@3 as web_sales] │ AggregateExec: mode=FinalPartitioned, gby=[ca_county@0 as ca_county, d_qoy@1 as d_qoy, d_year@2 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 19] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 19] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 16 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ca_county@2 as ca_county, d_year@3 as d_year, store_sales@4 as store_sales, store_sales@5 as store_sales, store_sales@6 as store_sales, ca_county@0 as ca_county, web_sales@1 as web_sales] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@0, ca_county@0)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@3, ca_county@0)], projection=[ca_county@0, d_year@1, store_sales@2, store_sales@4, store_sales@6] │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(store_sales.ss_ext_sales_price)@3 as store_sales] │ AggregateExec: mode=FinalPartitioned, gby=[ca_county@0 as ca_county, d_qoy@1 as d_qoy, d_year@2 as d_year], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(web_sales.ws_ext_sales_price)@3 as web_sales] │ AggregateExec: mode=FinalPartitioned, gby=[ca_county@0 as ca_county, d_qoy@1 as d_qoy, d_year@2 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ws_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ws_bill_addr_sk@2 as ws_bill_addr_sk, ws_ext_sales_price@3 as ws_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_bill_addr_sk@5, ws_ext_sales_price@6] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_qoy@2 = 1 AND d_year@1 = 2000 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 1 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 1 AND 1 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (1), d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 1 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 1 AND 1 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (1), d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 12 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@0, ca_county@0)] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(store_sales.ss_ext_sales_price)@3 as store_sales] │ AggregateExec: mode=FinalPartitioned, gby=[ca_county@0 as ca_county, d_qoy@1 as d_qoy, d_year@2 as d_year], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ca_county@0 as ca_county, d_year@2 as d_year, sum(store_sales.ss_ext_sales_price)@3 as store_sales] │ AggregateExec: mode=FinalPartitioned, gby=[ca_county@0 as ca_county, d_qoy@1 as d_qoy, d_year@2 as d_year], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 6 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ss_addr_sk@2 as ss_addr_sk, ss_ext_sales_price@3 as ss_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_addr_sk@5, ss_ext_sales_price@6] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_qoy@2 = 1 AND d_year@1 = 2000 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 1 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 1 AND 1 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (1), d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 1 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 1 AND 1 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (1), d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 + ┌───── Stage 11 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 10 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ss_addr_sk@2 as ss_addr_sk, ss_ext_sales_price@3 as ss_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_addr_sk@5, ss_ext_sales_price@6] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 2000 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 + ┌───── Stage 15 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 14 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ss_addr_sk@2 as ss_addr_sk, ss_ext_sales_price@3 as ss_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_addr_sk@5, ss_ext_sales_price@6] │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 13 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_qoy@2 = 3 AND d_year@1 = 2000 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 3 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 3 AND 3 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (3), d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 3 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 3 AND 3 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (3), d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 + ┌───── Stage 19 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ws_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 18 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ws_bill_addr_sk@2 as ws_bill_addr_sk, ws_ext_sales_price@3 as ws_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_bill_addr_sk@5, ws_ext_sales_price@6] │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 17 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 2000 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 23 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 + ┌───── Stage 23 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ws_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] │ CoalescePartitionsExec - │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 22 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ws_bill_addr_sk@2 as ws_bill_addr_sk, ws_ext_sales_price@3 as ws_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_bill_addr_sk@5, ws_ext_sales_price@6] │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 21 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_qoy@2 = 3 AND d_year@1 = 2000 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 3 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 3 AND 3 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (3), d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 3 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 3 AND 3 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (3), d_year in (2000)] └────────────────────────────────────────────────── "); Ok(()) @@ -3647,60 +3481,57 @@ mod tests { │ GlobalLimitExec: skip=0, fetch=100 │ AggregateExec: mode=Final, gby=[], aggr=[sum(catalog_sales.cs_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(catalog_sales.cs_ext_discount_amt)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, i_item_sk@1)], filter=CAST(cs_ext_discount_amt@0 AS Decimal128(30, 15)) > Float64(1.3) * avg(catalog_sales.cs_ext_discount_amt)@1, projection=[cs_ext_discount_amt@2] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ext_discount_amt@3, i_item_sk@4] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ext_discount_amt@2 as cs_ext_discount_amt, i_item_sk@0 as i_item_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_sk@0, cs_sold_date_sk@1, cs_ext_discount_amt@3] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_ext_discount_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[CAST(1.3 * CAST(avg(catalog_sales.cs_ext_discount_amt)@1 AS Float64) AS Decimal128(30, 15)) as Float64(1.3) * avg(catalog_sales.cs_ext_discount_amt), cs_item_sk@0 as cs_item_sk] │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[avg(catalog_sales.cs_ext_discount_amt)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 6), input_partitions=2 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[avg(catalog_sales.cs_ext_discount_amt)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_ext_discount_amt@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_ext_discount_amt], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: i_manufact_id@1 = 977, projection=[i_item_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=i_manufact_id@13 = 977 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 977 AND 977 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (977)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=i_manufact_id@13 = 977 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 977 AND 977 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (977)] └────────────────────────────────────────────────── "); Ok(()) @@ -3711,146 +3542,155 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [total_sales@1 ASC NULLS LAST], fetch=100 - │ SortExec: TopK(fetch=100), expr=[total_sales@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(tmp1.total_sales)@1 as total_sales] - │ AggregateExec: mode=SinglePartitioned, gby=[i_manufact_id@0 as i_manufact_id], aggr=[sum(tmp1.total_sales)] - │ InterleaveExec - │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(store_sales.ss_ext_sales_price)@1 as total_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact_id@0 as i_manufact_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_manufact_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_manufact_id@1 as i_manufact_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_manufact_id@1, i_manufact_id@0)] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_category@0 = Electronics, projection=[i_manufact_id@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Electronics, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1, required_guarantees=[i_category in (Electronics)] - │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(catalog_sales.cs_ext_sales_price)@1 as total_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact_id@0 as i_manufact_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_manufact_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_manufact_id@1 as i_manufact_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_manufact_id@1, i_manufact_id@0)] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_category@0 = Electronics, projection=[i_manufact_id@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Electronics, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1, required_guarantees=[i_category in (Electronics)] - │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(web_sales.ws_ext_sales_price)@1 as total_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact_id@0 as i_manufact_id], aggr=[sum(web_sales.ws_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_manufact_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_manufact_id@1 as i_manufact_id], aggr=[sum(web_sales.ws_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_manufact_id@1, i_manufact_id@0)] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_category@0 = Electronics, projection=[i_manufact_id@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Electronics, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1, required_guarantees=[i_category in (Electronics)] + │ [Stage 16] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_manufact_id@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[total_sales@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(tmp1.total_sales)@1 as total_sales] + │ AggregateExec: mode=SinglePartitioned, gby=[i_manufact_id@0 as i_manufact_id], aggr=[sum(tmp1.total_sales)] + │ InterleaveExec + │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(store_sales.ss_ext_sales_price)@1 as total_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact_id@0 as i_manufact_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(catalog_sales.cs_ext_sales_price)@1 as total_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact_id@0 as i_manufact_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(web_sales.ws_ext_sales_price)@1 as total_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact_id@0 as i_manufact_id], aggr=[sum(web_sales.ws_ext_sales_price)] + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_manufact_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_manufact_id@1 as i_manufact_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_manufact_id@1, i_manufact_id@0)] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_category@0 = Electronics, projection=[i_manufact_id@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Electronics, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1, required_guarantees=[i_category in (Electronics)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_addr_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_manufact_id@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 5, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 5 AND 5 <= d_moy_max@5, required_guarantees=[d_moy in (5), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_manufact_id@3] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_addr_sk@4, ss_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 5, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 5 AND 5 <= d_moy_max@5, required_guarantees=[d_moy in (5), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_manufact_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_manufact_id@1 as i_manufact_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_manufact_id@1, i_manufact_id@0)] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_category@0 = Electronics, projection=[i_manufact_id@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Electronics, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1, required_guarantees=[i_category in (Electronics)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_manufact_id@3] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 5, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 5 AND 5 <= d_moy_max@5, required_guarantees=[d_moy in (5), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_manufact_id@3] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 5, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 5 AND 5 <= d_moy_max@5, required_guarantees=[d_moy in (5), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 15 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_manufact_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_manufact_id@1 as i_manufact_id], aggr=[sum(web_sales.ws_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_manufact_id@1, i_manufact_id@0)] + │ CoalescePartitionsExec + │ [Stage 14] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_category@0 = Electronics, projection=[i_manufact_id@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Electronics, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1, required_guarantees=[i_category in (Electronics)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_bill_addr_sk@4, ws_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_manufact_id@3] + │ CoalescePartitionsExec + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 5, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 5 AND 5 <= d_moy_max@5, required_guarantees=[d_moy in (5), d_year in (1998)] - └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_bill_addr_sk@4, ws_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 5, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 5 AND 5 <= d_moy_max@5, required_guarantees=[d_moy in (5), d_year in (1998)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -3860,59 +3700,57 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, c_salutation@2 ASC, c_preferred_cust_flag@3 DESC, ss_ticket_number@4 ASC] - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] │ SortExec: expr=[c_last_name@0 ASC, c_first_name@1 ASC, c_salutation@2 ASC, c_preferred_cust_flag@3 DESC, ss_ticket_number@4 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[c_last_name@4 as c_last_name, c_first_name@3 as c_first_name, c_salutation@2 as c_salutation, c_preferred_cust_flag@5 as c_preferred_cust_flag, ss_ticket_number@0 as ss_ticket_number, cnt@1 as cnt] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_ticket_number@0, cnt@2, c_salutation@4, c_first_name@5, c_last_name@6, c_preferred_cust_flag@7] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_salutation@1 as c_salutation, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, count(Int64(1))@2 as cnt] │ FilterExec: count(Int64(1))@2 >= 15 AND count(Int64(1))@2 <= 20 │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk], aggr=[count(Int64(1))] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1], 6), input_partitions=2 + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@1 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk], aggr=[count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_ticket_number@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@2)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_ticket_number@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_store_sk@5, ss_ticket_number@6] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_hdemo_sk, ss_store_sk, ss_ticket_number], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: (hd_buy_potential@1 = >10000 OR hd_buy_potential@1 = Unknown) AND hd_vehicle_count@3 > 0 AND CASE WHEN hd_vehicle_count@3 > 0 THEN CAST(hd_dep_count@2 AS Float64) / CAST(hd_vehicle_count@3 AS Float64) END > 1.2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=(hd_buy_potential@2 = >10000 OR hd_buy_potential@2 = Unknown) AND hd_vehicle_count@4 > 0 AND CASE WHEN hd_vehicle_count@4 > 0 THEN CAST(hd_dep_count@3 AS Float64) / CAST(hd_vehicle_count@4 AS Float64) END > 1.2, pruning_predicate=(hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1 OR hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknown AND Unknown <= hd_buy_potential_max@1) AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 0, required_guarantees=[hd_buy_potential in (>10000, Unknown)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=(hd_buy_potential@2 = >10000 OR hd_buy_potential@2 = Unknown) AND hd_vehicle_count@4 > 0 AND CASE WHEN hd_vehicle_count@4 > 0 THEN CAST(hd_dep_count@3 AS Float64) / CAST(hd_vehicle_count@4 AS Float64) END > 1.2, pruning_predicate=(hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1 OR hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknown AND Unknown <= hd_buy_potential_max@1) AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 0, required_guarantees=[hd_buy_potential in (>10000, Unknown)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_county@1 = Williamson County, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_county], file_type=parquet, predicate=s_county@23 = Williamson County, pruning_predicate=s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Williamson County AND Williamson County <= s_county_max@1, required_guarantees=[s_county in (Williamson County)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_county], file_type=parquet, predicate=s_county@23 = Williamson County, pruning_predicate=s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Williamson County AND Williamson County <= s_county_max@1, required_guarantees=[s_county in (Williamson County)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: (d_dom@2 >= 1 AND d_dom@2 <= 3 OR d_dom@2 >= 25 AND d_dom@2 <= 28) AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -3928,96 +3766,95 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ca_state@0 ASC, cd_gender@1 ASC, cd_marital_status@2 ASC, cd_dep_count@3 ASC, cd_dep_employed_count@8 ASC, cd_dep_college_count@13 ASC], fetch=100 - │ SortExec: TopK(fetch=100), expr=[ca_state@0 ASC, cd_gender@1 ASC, cd_marital_status@2 ASC, cd_dep_count@3 ASC, cd_dep_employed_count@8 ASC, cd_dep_college_count@13 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[ca_state@0 as ca_state, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_dep_count@3 as cd_dep_count, count(Int64(1))@6 as cnt1, min(customer_demographics.cd_dep_count)@7 as min1, max(customer_demographics.cd_dep_count)@8 as max1, avg(customer_demographics.cd_dep_count)@9 as avg1, cd_dep_employed_count@4 as cd_dep_employed_count, count(Int64(1))@6 as cnt2, min(customer_demographics.cd_dep_employed_count)@10 as min2, max(customer_demographics.cd_dep_employed_count)@11 as max2, avg(customer_demographics.cd_dep_employed_count)@12 as avg2, cd_dep_college_count@5 as cd_dep_college_count, count(Int64(1))@6 as cnt3, min(customer_demographics.cd_dep_college_count)@13 as min(customer_demographics.cd_dep_college_count), max(customer_demographics.cd_dep_college_count)@14 as max(customer_demographics.cd_dep_college_count), avg(customer_demographics.cd_dep_college_count)@15 as avg(customer_demographics.cd_dep_college_count)] - │ AggregateExec: mode=FinalPartitioned, gby=[ca_state@0 as ca_state, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_dep_count@3 as cd_dep_count, cd_dep_employed_count@4 as cd_dep_employed_count, cd_dep_college_count@5 as cd_dep_college_count], aggr=[count(Int64(1)), min(customer_demographics.cd_dep_count), max(customer_demographics.cd_dep_count), avg(customer_demographics.cd_dep_count), min(customer_demographics.cd_dep_employed_count), max(customer_demographics.cd_dep_employed_count), avg(customer_demographics.cd_dep_employed_count), min(customer_demographics.cd_dep_college_count), max(customer_demographics.cd_dep_college_count), avg(customer_demographics.cd_dep_college_count)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([ca_state@0, cd_gender@1, cd_marital_status@2, cd_dep_count@3, cd_dep_employed_count@4, cd_dep_college_count@5], 3), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[ca_state@0 as ca_state, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_dep_count@3 as cd_dep_count, cd_dep_employed_count@4 as cd_dep_employed_count, cd_dep_college_count@5 as cd_dep_college_count], aggr=[count(Int64(1)), min(customer_demographics.cd_dep_count), max(customer_demographics.cd_dep_count), avg(customer_demographics.cd_dep_count), min(customer_demographics.cd_dep_employed_count), max(customer_demographics.cd_dep_employed_count), avg(customer_demographics.cd_dep_employed_count), min(customer_demographics.cd_dep_college_count), max(customer_demographics.cd_dep_college_count), avg(customer_demographics.cd_dep_college_count)] - │ FilterExec: mark@6 OR mark@7, projection=[ca_state@0, cd_gender@1, cd_marital_status@2, cd_dep_count@3, cd_dep_employed_count@4, cd_dep_college_count@5] - │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(cs_ship_customer_sk@0, CAST(c.c_customer_sk AS Float64)@8)], projection=[ca_state@1, cd_gender@2, cd_marital_status@3, cd_dep_count@4, cd_dep_employed_count@5, cd_dep_college_count@6, mark@7, mark@9] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ca_state@1 as ca_state, cd_gender@2 as cd_gender, cd_marital_status@3 as cd_marital_status, cd_dep_count@4 as cd_dep_count, cd_dep_employed_count@5 as cd_dep_employed_count, cd_dep_college_count@6 as cd_dep_college_count, mark@7 as mark, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] - │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(ws_bill_customer_sk@0, CAST(c.c_customer_sk AS Float64)@7)], projection=[c_customer_sk@0, ca_state@1, cd_gender@2, cd_marital_status@3, cd_dep_count@4, cd_dep_employed_count@5, cd_dep_college_count@6, mark@8] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ca_state@1 as ca_state, cd_gender@2 as cd_gender, cd_marital_status@3 as cd_marital_status, cd_dep_count@4 as cd_dep_count, cd_dep_employed_count@5 as cd_dep_employed_count, cd_dep_college_count@6 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(ss_customer_sk@0, CAST(c.c_customer_sk AS Float64)@7)], projection=[c_customer_sk@0, ca_state@1, cd_gender@2, cd_marital_status@3, cd_dep_count@4, cd_dep_employed_count@5, cd_dep_college_count@6] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ca_state@1 as ca_state, cd_gender@2 as cd_gender, cd_marital_status@3 as cd_marital_status, cd_dep_count@4 as cd_dep_count, cd_dep_employed_count@5 as cd_dep_employed_count, cd_dep_college_count@6 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@6)], projection=[c_customer_sk@0, ca_state@2, cd_gender@4, cd_marital_status@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_dep_count, cd_dep_employed_count, cd_dep_college_count, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[ca_state@0 ASC, cd_gender@1 ASC, cd_marital_status@2 ASC, cd_dep_count@3 ASC, cd_dep_employed_count@8 ASC, cd_dep_college_count@13 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[ca_state@0 as ca_state, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_dep_count@3 as cd_dep_count, count(Int64(1))@6 as cnt1, min(customer_demographics.cd_dep_count)@7 as min1, max(customer_demographics.cd_dep_count)@8 as max1, avg(customer_demographics.cd_dep_count)@9 as avg1, cd_dep_employed_count@4 as cd_dep_employed_count, count(Int64(1))@6 as cnt2, min(customer_demographics.cd_dep_employed_count)@10 as min2, max(customer_demographics.cd_dep_employed_count)@11 as max2, avg(customer_demographics.cd_dep_employed_count)@12 as avg2, cd_dep_college_count@5 as cd_dep_college_count, count(Int64(1))@6 as cnt3, min(customer_demographics.cd_dep_college_count)@13 as min(customer_demographics.cd_dep_college_count), max(customer_demographics.cd_dep_college_count)@14 as max(customer_demographics.cd_dep_college_count), avg(customer_demographics.cd_dep_college_count)@15 as avg(customer_demographics.cd_dep_college_count)] + │ AggregateExec: mode=FinalPartitioned, gby=[ca_state@0 as ca_state, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_dep_count@3 as cd_dep_count, cd_dep_employed_count@4 as cd_dep_employed_count, cd_dep_college_count@5 as cd_dep_college_count], aggr=[count(Int64(1)), min(customer_demographics.cd_dep_count), max(customer_demographics.cd_dep_count), avg(customer_demographics.cd_dep_count), min(customer_demographics.cd_dep_employed_count), max(customer_demographics.cd_dep_employed_count), avg(customer_demographics.cd_dep_employed_count), min(customer_demographics.cd_dep_college_count), max(customer_demographics.cd_dep_college_count), avg(customer_demographics.cd_dep_college_count)] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ca_state@0, cd_gender@1, cd_marital_status@2, cd_dep_count@3, cd_dep_employed_count@4, cd_dep_college_count@5], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[ca_state@0 as ca_state, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_dep_count@3 as cd_dep_count, cd_dep_employed_count@4 as cd_dep_employed_count, cd_dep_college_count@5 as cd_dep_college_count], aggr=[count(Int64(1)), min(customer_demographics.cd_dep_count), max(customer_demographics.cd_dep_count), avg(customer_demographics.cd_dep_count), min(customer_demographics.cd_dep_employed_count), max(customer_demographics.cd_dep_employed_count), avg(customer_demographics.cd_dep_employed_count), min(customer_demographics.cd_dep_college_count), max(customer_demographics.cd_dep_college_count), avg(customer_demographics.cd_dep_college_count)] + │ FilterExec: mark@6 OR mark@7, projection=[ca_state@0, cd_gender@1, cd_marital_status@2, cd_dep_count@3, cd_dep_employed_count@4, cd_dep_college_count@5] + │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(cs_ship_customer_sk@0, CAST(c.c_customer_sk AS Float64)@8)], projection=[ca_state@1, cd_gender@2, cd_marital_status@3, cd_dep_count@4, cd_dep_employed_count@5, cd_dep_college_count@6, mark@7, mark@9] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ca_state@1 as ca_state, cd_gender@2 as cd_gender, cd_marital_status@3 as cd_marital_status, cd_dep_count@4 as cd_dep_count, cd_dep_employed_count@5 as cd_dep_employed_count, cd_dep_college_count@6 as cd_dep_college_count, mark@7 as mark, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] + │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(ws_bill_customer_sk@0, CAST(c.c_customer_sk AS Float64)@7)], projection=[c_customer_sk@0, ca_state@1, cd_gender@2, cd_marital_status@3, cd_dep_count@4, cd_dep_employed_count@5, cd_dep_college_count@6, mark@8] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ca_state@1 as ca_state, cd_gender@2 as cd_gender, cd_marital_status@3 as cd_marital_status, cd_dep_count@4 as cd_dep_count, cd_dep_employed_count@5 as cd_dep_employed_count, cd_dep_college_count@6 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(ss_customer_sk@0, CAST(c.c_customer_sk AS Float64)@7)], projection=[c_customer_sk@0, ca_state@1, cd_gender@2, cd_marital_status@3, cd_dep_count@4, cd_dep_employed_count@5, cd_dep_college_count@6] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ca_state@1 as ca_state, cd_gender@2 as cd_gender, cd_marital_status@3 as cd_marital_status, cd_dep_count@4 as cd_dep_count, cd_dep_employed_count@5 as cd_dep_employed_count, cd_dep_college_count@6 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@6)], projection=[c_customer_sk@0, ca_state@2, cd_gender@4, cd_marital_status@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_dep_count, cd_dep_employed_count, cd_dep_college_count, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_sk@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[c_customer_sk@1 as c_customer_sk, c_current_cdemo_sk@2 as c_current_cdemo_sk, ca_state@0 as ca_state] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[ca_state@1, c_customer_sk@2, c_current_cdemo_sk@3] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@1 as c_customer_sk, c_current_cdemo_sk@2 as c_current_cdemo_sk, ca_state@0 as ca_state] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[ca_state@1, c_customer_sk@2, c_current_cdemo_sk@3] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet + └────────────────────────────────────────────────── "); Ok(()) } @@ -4034,155 +3871,137 @@ mod tests { │ RepartitionExec: partitioning=Hash([lochierarchy@4, CASE WHEN t_class@3 = 0 THEN i_category@1 END], 3), input_partitions=3 │ ProjectionExec: expr=[gross_margin@0 as gross_margin, i_category@1 as i_category, i_class@2 as i_class, t_class@4 as t_class, lochierarchy@5 as lochierarchy] │ AggregateExec: mode=FinalPartitioned, gby=[gross_margin@0 as gross_margin, i_category@1 as i_category, i_class@2 as i_class, t_category@3 as t_category, t_class@4 as t_class, lochierarchy@5 as lochierarchy], aggr=[] - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([gross_margin@0, i_category@1, i_class@2, t_category@3, t_class@4, lochierarchy@5], 3), input_partitions=4 + │ AggregateExec: mode=Partial, gby=[gross_margin@0 as gross_margin, i_category@1 as i_category, i_class@2 as i_class, t_category@3 as t_category, t_class@4 as t_class, lochierarchy@5 as lochierarchy], aggr=[], ordering_mode=PartiallySorted([3]) + │ DistributedUnionExec: t0:[c0, c1] + │ ProjectionExec: expr=[gross_margin@0 as gross_margin, CAST(i_category@1 AS Utf8) as i_category, i_class@2 as i_class, t_category@3 as t_category, t_class@4 as t_class, lochierarchy@5 as lochierarchy] + │ AggregateExec: mode=FinalPartitioned, gby=[gross_margin@0 as gross_margin, i_category@1 as i_category, i_class@2 as i_class, t_category@3 as t_category, t_class@4 as t_class, lochierarchy@5 as lochierarchy], aggr=[], ordering_mode=PartiallySorted([3]) + │ RepartitionExec: partitioning=Hash([gross_margin@0, i_category@1, i_class@2, t_category@3, t_class@4, lochierarchy@5], 3), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[gross_margin@0 as gross_margin, i_category@1 as i_category, i_class@2 as i_class, t_category@3 as t_category, t_class@4 as t_class, lochierarchy@5 as lochierarchy], aggr=[], ordering_mode=PartiallySorted([3, 4, 5]) + │ DistributedUnionExec: t0:[c0, c1] + │ ProjectionExec: expr=[CAST(sum(store_sales.ss_net_profit)@2 AS Float64) / CAST(sum(store_sales.ss_ext_sales_price)@3 AS Float64) as gross_margin, i_category@0 as i_category, CAST(i_class@1 AS Utf8) as i_class, 0 as t_category, 0 as t_class, 0 as lochierarchy] + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[CAST(sum(results.ss_net_profit)@1 AS Float64) / CAST(sum(results.ss_ext_sales_price)@2 AS Float64) as gross_margin, i_category@0 as i_category, NULL as i_class, 0 as t_category, 1 as t_class, 1 as lochierarchy] + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] + │ RepartitionExec: partitioning=Hash([i_category@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_category@2 as i_category], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] + │ ProjectionExec: expr=[sum(store_sales.ss_net_profit)@2 as ss_net_profit, sum(store_sales.ss_ext_sales_price)@3 as ss_ext_sales_price, i_category@0 as i_category] + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[CAST(sum(results.ss_net_profit)@0 AS Float64) / CAST(sum(results.ss_ext_sales_price)@1 AS Float64) as gross_margin, NULL as i_category, NULL as i_class, 1 as t_category, 1 as t_class, 2 as lochierarchy] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] + │ ProjectionExec: expr=[sum(store_sales.ss_net_profit)@2 as ss_net_profit, sum(store_sales.ss_ext_sales_price)@3 as ss_ext_sales_price] + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([gross_margin@0, i_category@1, i_class@2, t_category@3, t_class@4, lochierarchy@5], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[gross_margin@0 as gross_margin, i_category@1 as i_category, i_class@2 as i_class, t_category@3 as t_category, t_class@4 as t_class, lochierarchy@5 as lochierarchy], aggr=[], ordering_mode=PartiallySorted([3]) - │ DistributedUnionExec: t0:[c0] t1:[c1] - │ ProjectionExec: expr=[gross_margin@0 as gross_margin, CAST(i_category@1 AS Utf8) as i_category, i_class@2 as i_class, t_category@3 as t_category, t_class@4 as t_class, lochierarchy@5 as lochierarchy] - │ AggregateExec: mode=FinalPartitioned, gby=[gross_margin@0 as gross_margin, i_category@1 as i_category, i_class@2 as i_class, t_category@3 as t_category, t_class@4 as t_class, lochierarchy@5 as lochierarchy], aggr=[], ordering_mode=PartiallySorted([3]) - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ ProjectionExec: expr=[CAST(sum(results.ss_net_profit)@0 AS Float64) / CAST(sum(results.ss_ext_sales_price)@1 AS Float64) as gross_margin, NULL as i_category, NULL as i_class, 1 as t_category, 1 as t_class, 2 as lochierarchy] - │ AggregateExec: mode=Final, gby=[], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] - │ CoalescePartitionsExec - │ [Stage 15] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([gross_margin@0, i_category@1, i_class@2, t_category@3, t_class@4, lochierarchy@5], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[gross_margin@0 as gross_margin, i_category@1 as i_category, i_class@2 as i_class, t_category@3 as t_category, t_class@4 as t_class, lochierarchy@5 as lochierarchy], aggr=[], ordering_mode=PartiallySorted([3, 4, 5]) - │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1] - │ ProjectionExec: expr=[CAST(sum(store_sales.ss_net_profit)@2 AS Float64) / CAST(sum(store_sales.ss_ext_sales_price)@3 AS Float64) as gross_margin, i_category@0 as i_category, CAST(i_class@1 AS Utf8) as i_class, 0 as t_category, 0 as t_class, 0 as lochierarchy] - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ ProjectionExec: expr=[CAST(sum(results.ss_net_profit)@1 AS Float64) / CAST(sum(results.ss_ext_sales_price)@2 AS Float64) as gross_margin, i_category@0 as i_category, NULL as i_class, 0 as t_category, 1 as t_class, 1 as lochierarchy] - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_category@3 as i_category, i_class@2 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_store_sk@1, ss_ext_sales_price@2, ss_net_profit@3, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_category@3 as i_category, i_class@2 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, i_class@5, i_category@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_store_sk@1, ss_ext_sales_price@2, ss_net_profit@3, i_class@5, i_category@6] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_category@2 as i_category], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] - │ ProjectionExec: expr=[sum(store_sales.ss_net_profit)@2 as ss_net_profit, sum(store_sales.ss_ext_sales_price)@3 as ss_ext_sales_price, i_category@0 as i_category] - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_category@3 as i_category, i_class@2 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_store_sk@1, ss_ext_sales_price@2, ss_net_profit@3, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] - │ ProjectionExec: expr=[sum(store_sales.ss_net_profit)@2 as ss_net_profit, sum(store_sales.ss_ext_sales_price)@3 as ss_ext_sales_price] - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_category@3 as i_category, i_class@2 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_store_sk@1, ss_ext_sales_price@2, ss_net_profit@3, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_category@3 as i_category, i_class@2 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, i_class@5, i_category@6] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_store_sk@1, ss_ext_sales_price@2, ss_net_profit@3, i_class@5, i_category@6] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_category@3 as i_category, i_class@2 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, i_class@5, i_category@6] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_store_sk@1, ss_ext_sales_price@2, ss_net_profit@3, i_class@5, i_category@6] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] + │ CoalescePartitionsExec + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── "#); Ok(()) } @@ -4192,48 +4011,45 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_current_price@2], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@0)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_current_price@2], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@0)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ FilterExec: d_date@1 >= 2000-02-01 AND d_date@1 <= 2000-04-01, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-02-01 AND d_date@2 <= 2000-04-01 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-01, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3, inv_date_sk@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ FilterExec: d_date@1 >= 2000-02-01 AND d_date@1 <= 2000-04-01, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-02-01 AND d_date@2 <= 2000-04-01 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-01, required_guarantees=[] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@3 >= 100 AND inv_quantity_on_hand@3 <= 500 AND DynamicFilter [ empty ], pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3, inv_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@3 >= 100 AND inv_quantity_on_hand@3 <= 500 AND DynamicFilter [ empty ], pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@3 >= Some(6800),4,2 AND i_current_price@3 <= Some(9800),4,2 AND i_manufact_id@4 IN (SET) ([677, 940, 694, 808]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_manufact_id], file_type=parquet, predicate=i_current_price@5 >= Some(6800),4,2 AND i_current_price@5 <= Some(9800),4,2 AND i_manufact_id@13 IN (SET) ([677, 940, 694, 808]) AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(6800),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(9800),4,2 AND (i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 677 AND 677 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 940 AND 940 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 694 AND 694 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 808 AND 808 <= i_manufact_id_max@5), required_guarantees=[i_manufact_id in (677, 694, 808, 940)] - └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_current_price@3 >= Some(6800),4,2 AND i_current_price@3 <= Some(9800),4,2 AND i_manufact_id@4 IN (SET) ([677, 940, 694, 808]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_manufact_id], file_type=parquet, predicate=i_current_price@5 >= Some(6800),4,2 AND i_current_price@5 <= Some(9800),4,2 AND i_manufact_id@13 IN (SET) ([677, 940, 694, 808]) AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(6800),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(9800),4,2 AND (i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 677 AND 677 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 940 AND 940 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 694 AND 694 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 808 AND 808 <= i_manufact_id_max@5), required_guarantees=[i_manufact_id in (677, 694, 808, 940)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -4246,118 +4062,115 @@ mod tests { │ GlobalLimitExec: skip=0, fetch=100 │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=1 │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=1 │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p8] + │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ws_bill_customer_sk@1 as ws_bill_customer_sk, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_bill_customer_sk@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 8 ── Tasks: t0:[p0..p8] + │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 6 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[cs_bill_customer_sk@1 as cs_bill_customer_sk, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_date@1, cs_bill_customer_sk@4] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 + ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] t3:[p0..p8] + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 9), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 10 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ss_customer_sk@1 as ss_customer_sk, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_customer_sk@4] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -4372,95 +4185,83 @@ mod tests { │ SortExec: expr=[wsk1@0 ASC, isk1@1 ASC, mean1@3 ASC, cov1@4 ASC, mean@8 ASC, cov@9 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[w_warehouse_sk@0 as wsk1, i_item_sk@1 as isk1, d_moy@2 as dmoy1, mean@3 as mean1, cov@4 as cov1, w_warehouse_sk@5 as w_warehouse_sk, i_item_sk@6 as i_item_sk, d_moy@7 as d_moy, mean@8 as mean, cov@9 as cov, w_warehouse_sk@0 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@2 as d_moy, mean@3 as mean, cov@4 as cov] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@1, i_item_sk@1), (w_warehouse_sk@0, w_warehouse_sk@0)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_item_sk@1, w_warehouse_sk@0], 3), input_partitions=3 + │ ProjectionExec: expr=[w_warehouse_sk@0 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@2 as d_moy, avg(inventory.inv_quantity_on_hand)@4 as mean, CASE avg(inventory.inv_quantity_on_hand)@4 WHEN 0 THEN NULL ELSE stddev(inventory.inv_quantity_on_hand)@3 / avg(inventory.inv_quantity_on_hand)@4 END as cov] + │ FilterExec: CASE avg(inventory.inv_quantity_on_hand)@4 WHEN 0 THEN 0 ELSE stddev(inventory.inv_quantity_on_hand)@3 / avg(inventory.inv_quantity_on_hand)@4 END > 1 + │ ProjectionExec: expr=[w_warehouse_sk@1 as w_warehouse_sk, i_item_sk@2 as i_item_sk, d_moy@3 as d_moy, stddev(inventory.inv_quantity_on_hand)@4 as stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)@5 as avg(inventory.inv_quantity_on_hand)] + │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sk@1 as w_warehouse_sk, i_item_sk@2 as i_item_sk, d_moy@3 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ RepartitionExec: partitioning=Hash([i_item_sk@1, w_warehouse_sk@0], 3), input_partitions=3 + │ ProjectionExec: expr=[w_warehouse_sk@0 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@2 as d_moy, avg(inventory.inv_quantity_on_hand)@4 as mean, CASE avg(inventory.inv_quantity_on_hand)@4 WHEN 0 THEN NULL ELSE stddev(inventory.inv_quantity_on_hand)@3 / avg(inventory.inv_quantity_on_hand)@4 END as cov] + │ FilterExec: CASE avg(inventory.inv_quantity_on_hand)@4 WHEN 0 THEN 0 ELSE stddev(inventory.inv_quantity_on_hand)@3 / avg(inventory.inv_quantity_on_hand)@4 END > 1 + │ ProjectionExec: expr=[w_warehouse_sk@1 as w_warehouse_sk, i_item_sk@2 as i_item_sk, d_moy@3 as d_moy, stddev(inventory.inv_quantity_on_hand)@4 as stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)@5 as avg(inventory.inv_quantity_on_hand)] + │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sk@1 as w_warehouse_sk, i_item_sk@2 as i_item_sk, d_moy@3 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_item_sk@1, w_warehouse_sk@0], 3), input_partitions=3 - │ ProjectionExec: expr=[w_warehouse_sk@0 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@2 as d_moy, avg(inventory.inv_quantity_on_hand)@4 as mean, CASE avg(inventory.inv_quantity_on_hand)@4 WHEN 0 THEN NULL ELSE stddev(inventory.inv_quantity_on_hand)@3 / avg(inventory.inv_quantity_on_hand)@4 END as cov] - │ FilterExec: CASE avg(inventory.inv_quantity_on_hand)@4 WHEN 0 THEN 0 ELSE stddev(inventory.inv_quantity_on_hand)@3 / avg(inventory.inv_quantity_on_hand)@4 END > 1 - │ ProjectionExec: expr=[w_warehouse_sk@1 as w_warehouse_sk, i_item_sk@2 as i_item_sk, d_moy@3 as d_moy, stddev(inventory.inv_quantity_on_hand)@4 as stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)@5 as avg(inventory.inv_quantity_on_hand)] - │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sk@1 as w_warehouse_sk, i_item_sk@2 as i_item_sk, d_moy@3 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sk@1, i_item_sk@2, d_moy@3], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[w_warehouse_name@3 as w_warehouse_name, w_warehouse_sk@2 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@4 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) - │ ProjectionExec: expr=[inv_quantity_on_hand@1 as inv_quantity_on_hand, i_item_sk@2 as i_item_sk, w_warehouse_sk@3 as w_warehouse_sk, w_warehouse_name@4 as w_warehouse_name, d_moy@0 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[d_moy@1, inv_quantity_on_hand@3, i_item_sk@4, w_warehouse_sk@5, w_warehouse_name@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[inv_date_sk@2 as inv_date_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@4 as i_item_sk, w_warehouse_sk@0 as w_warehouse_sk, w_warehouse_name@1 as w_warehouse_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@1)], projection=[w_warehouse_sk@0, w_warehouse_name@1, inv_date_sk@2, inv_quantity_on_hand@4, i_item_sk@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_warehouse_sk@2 as inv_warehouse_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@0 as i_item_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, inv_date_sk@1, inv_warehouse_sk@3, inv_quantity_on_hand@4] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sk@1, i_item_sk@2, d_moy@3], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[w_warehouse_name@3 as w_warehouse_name, w_warehouse_sk@2 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@4 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) + │ ProjectionExec: expr=[inv_quantity_on_hand@1 as inv_quantity_on_hand, i_item_sk@2 as i_item_sk, w_warehouse_sk@3 as w_warehouse_sk, w_warehouse_name@4 as w_warehouse_name, d_moy@0 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[d_moy@1, inv_quantity_on_hand@3, i_item_sk@4, w_warehouse_sk@5, w_warehouse_name@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[inv_date_sk@2 as inv_date_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@4 as i_item_sk, w_warehouse_sk@0 as w_warehouse_sk, w_warehouse_name@1 as w_warehouse_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@1)], projection=[w_warehouse_sk@0, w_warehouse_name@1, inv_date_sk@2, inv_quantity_on_hand@4, i_item_sk@5] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_warehouse_sk@2 as inv_warehouse_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@0 as i_item_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, inv_date_sk@1, inv_warehouse_sk@3, inv_quantity_on_hand@4] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: d_moy@2 = 1 AND d_year@1 = 2001, projection=[d_date_sk@0, d_moy@2] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 1 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 1 AND 1 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (1), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_moy@2 = 1 AND d_year@1 = 2001, projection=[d_date_sk@0, d_moy@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 1 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 1 AND 1 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (1), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_item_sk@1, w_warehouse_sk@0], 3), input_partitions=3 - │ ProjectionExec: expr=[w_warehouse_sk@0 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@2 as d_moy, avg(inventory.inv_quantity_on_hand)@4 as mean, CASE avg(inventory.inv_quantity_on_hand)@4 WHEN 0 THEN NULL ELSE stddev(inventory.inv_quantity_on_hand)@3 / avg(inventory.inv_quantity_on_hand)@4 END as cov] - │ FilterExec: CASE avg(inventory.inv_quantity_on_hand)@4 WHEN 0 THEN 0 ELSE stddev(inventory.inv_quantity_on_hand)@3 / avg(inventory.inv_quantity_on_hand)@4 END > 1 - │ ProjectionExec: expr=[w_warehouse_sk@1 as w_warehouse_sk, i_item_sk@2 as i_item_sk, d_moy@3 as d_moy, stddev(inventory.inv_quantity_on_hand)@4 as stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)@5 as avg(inventory.inv_quantity_on_hand)] - │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sk@1 as w_warehouse_sk, i_item_sk@2 as i_item_sk, d_moy@3 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sk@1, i_item_sk@2, d_moy@3], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[w_warehouse_name@3 as w_warehouse_name, w_warehouse_sk@2 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@4 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) - │ ProjectionExec: expr=[inv_quantity_on_hand@1 as inv_quantity_on_hand, i_item_sk@2 as i_item_sk, w_warehouse_sk@3 as w_warehouse_sk, w_warehouse_name@4 as w_warehouse_name, d_moy@0 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[d_moy@1, inv_quantity_on_hand@3, i_item_sk@4, w_warehouse_sk@5, w_warehouse_name@6] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[inv_date_sk@2 as inv_date_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@4 as i_item_sk, w_warehouse_sk@0 as w_warehouse_sk, w_warehouse_name@1 as w_warehouse_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@1)], projection=[w_warehouse_sk@0, w_warehouse_name@1, inv_date_sk@2, inv_quantity_on_hand@4, i_item_sk@5] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_warehouse_sk@2 as inv_warehouse_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@0 as i_item_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, inv_date_sk@1, inv_warehouse_sk@3, inv_quantity_on_hand@4] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sk@1, i_item_sk@2, d_moy@3], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[w_warehouse_name@3 as w_warehouse_name, w_warehouse_sk@2 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@4 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) + │ ProjectionExec: expr=[inv_quantity_on_hand@1 as inv_quantity_on_hand, i_item_sk@2 as i_item_sk, w_warehouse_sk@3 as w_warehouse_sk, w_warehouse_name@4 as w_warehouse_name, d_moy@0 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[d_moy@1, inv_quantity_on_hand@3, i_item_sk@4, w_warehouse_sk@5, w_warehouse_name@6] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[inv_date_sk@2 as inv_date_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@4 as i_item_sk, w_warehouse_sk@0 as w_warehouse_sk, w_warehouse_name@1 as w_warehouse_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@1)], projection=[w_warehouse_sk@0, w_warehouse_name@1, inv_date_sk@2, inv_quantity_on_hand@4, i_item_sk@5] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_warehouse_sk@2 as inv_warehouse_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@0 as i_item_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, inv_date_sk@1, inv_warehouse_sk@3, inv_quantity_on_hand@4] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: d_moy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0, d_moy@2] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 2 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 2 AND 2 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_moy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0, d_moy@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 2 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 2 AND 2 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet - └────────────────────────────────────────────────── "); Ok(()) } @@ -4470,105 +4271,66 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [w_state@0 ASC NULLS LAST, i_item_id@1 ASC NULLS LAST], fetch=100 - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[w_state@0 ASC NULLS LAST, i_item_id@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[w_state@0 as w_state, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)@2 as sales_before, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)@3 as sales_after] + │ AggregateExec: mode=FinalPartitioned, gby=[w_state@0 as w_state, i_item_id@1 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[w_state@0 ASC NULLS LAST, i_item_id@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[w_state@0 as w_state, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)@2 as sales_before, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)@3 as sales_after] - │ AggregateExec: mode=FinalPartitioned, gby=[w_state@0 as w_state, i_item_id@1 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([w_state@0, i_item_id@1], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[w_state@3 as w_state, i_item_id@4 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)] - │ ProjectionExec: expr=[d_date@4 as __common_expr_1, cs_sales_price@0 as cs_sales_price, cr_refunded_cash@1 as cr_refunded_cash, w_state@2 as w_state, i_item_id@3 as i_item_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[cs_sales_price@1, cr_refunded_cash@2, w_state@3, i_item_id@4, d_date@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 2000-02-10 AND d_date@1 <= 2000-04-10 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-02-10 AND d_date@2 <= 2000-04-10, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-10 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-10, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, i_item_sk@0)], projection=[cs_sold_date_sk@0, cs_sales_price@2, cr_refunded_cash@3, w_state@4, i_item_id@6] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_current_price@2 >= Some(99),4,2 AND i_current_price@2 <= Some(149),4,2, projection=[i_item_sk@0, i_item_id@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_current_price], file_type=parquet, predicate=i_current_price@5 >= Some(99),4,2 AND i_current_price@5 <= Some(149),4,2 AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(99),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(149),4,2, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_item_sk@2 as cs_item_sk, cs_sales_price@3 as cs_sales_price, cr_refunded_cash@4 as cr_refunded_cash, w_state@0 as w_state] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@2, cs_warehouse_sk@1)], projection=[w_state@1, cs_sold_date_sk@3, cs_item_sk@5, cs_sales_price@6, cr_refunded_cash@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_warehouse_sk@2 as cs_warehouse_sk, cs_item_sk@3 as cs_item_sk, cs_sales_price@4 as cs_sales_price, cr_refunded_cash@0 as cr_refunded_cash] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@3), (cr_item_sk@0, cs_item_sk@2)], projection=[cr_refunded_cash@2, cs_sold_date_sk@3, cs_warehouse_sk@4, cs_item_sk@5, cs_sales_price@7] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_state, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_order_number@3, cs_item_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_warehouse_sk, cs_item_sk, cs_order_number, cs_sales_price], file_type=parquet - └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([w_state@0, i_item_id@1], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[w_state@3 as w_state, i_item_id@4 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)] + │ ProjectionExec: expr=[d_date@4 as __common_expr_1, cs_sales_price@0 as cs_sales_price, cr_refunded_cash@1 as cr_refunded_cash, w_state@2 as w_state, i_item_id@3 as i_item_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[cs_sales_price@1, cr_refunded_cash@2, w_state@3, i_item_id@4, d_date@6] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 2000-02-10 AND d_date@1 <= 2000-04-10 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-02-10 AND d_date@2 <= 2000-04-10, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-10 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-10, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, i_item_sk@0)], projection=[cs_sold_date_sk@0, cs_sales_price@2, cr_refunded_cash@3, w_state@4, i_item_id@6] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ FilterExec: i_current_price@2 >= Some(99),4,2 AND i_current_price@2 <= Some(149),4,2, projection=[i_item_sk@0, i_item_id@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_current_price], file_type=parquet, predicate=i_current_price@5 >= Some(99),4,2 AND i_current_price@5 <= Some(149),4,2 AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(99),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(149),4,2, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_item_sk@2 as cs_item_sk, cs_sales_price@3 as cs_sales_price, cr_refunded_cash@4 as cr_refunded_cash, w_state@0 as w_state] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@2, cs_warehouse_sk@1)], projection=[w_state@1, cs_sold_date_sk@3, cs_item_sk@5, cs_sales_price@6, cr_refunded_cash@7] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_warehouse_sk@2 as cs_warehouse_sk, cs_item_sk@3 as cs_item_sk, cs_sales_price@4 as cs_sales_price, cr_refunded_cash@0 as cr_refunded_cash] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@3), (cr_item_sk@0, cs_item_sk@2)], projection=[cr_refunded_cash@2, cs_sold_date_sk@3, cs_warehouse_sk@4, cs_item_sk@5, cs_sales_price@7] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_state, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_order_number@3, cs_item_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_warehouse_sk, cs_item_sk, cs_order_number, cs_sales_price], file_type=parquet + └────────────────────────────────────────────────── "#); Ok(()) } #[tokio::test] async fn test_tpcds_41() -> Result<()> { let display = test_tpcds_query("q41").await?; - assert_snapshot!(display, @r" - ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [i_product_name@0 ASC NULLS LAST], fetch=100 - │ SortExec: TopK(fetch=100), expr=[i_product_name@0 ASC NULLS LAST], preserve_partitioning=[true] - │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name], aggr=[] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_product_name@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_product_name@0 as i_product_name], aggr=[] - │ FilterExec: CASE WHEN __always_true@2 IS NULL THEN 0 ELSE item_cnt@1 END > 0, projection=[i_product_name@0] - │ ProjectionExec: expr=[i_product_name@2 as i_product_name, item_cnt@0 as item_cnt, __always_true@1 as __always_true] - │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(i_manufact@1, i_manufact@0)], projection=[item_cnt@0, __always_true@2, i_product_name@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 - │ FilterExec: i_manufact_id@0 >= 738 AND i_manufact_id@0 <= 778, projection=[i_manufact@1, i_product_name@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_manufact_id, i_manufact, i_product_name], file_type=parquet, predicate=i_manufact_id@13 >= 738 AND i_manufact_id@13 <= 778, pruning_predicate=i_manufact_id_null_count@1 != row_count@2 AND i_manufact_id_max@0 >= 738 AND i_manufact_id_null_count@1 != row_count@2 AND i_manufact_id_min@3 <= 778, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[count(Int64(1))@1 as item_cnt, i_manufact@0 as i_manufact, true as __always_true] - │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact@0 as i_manufact], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_manufact@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_manufact@0 as i_manufact], aggr=[count(Int64(1))] - │ FilterExec: __common_expr_4@0 AND ((i_color@4 = powder OR i_color@4 = khaki) AND (i_units@5 = Ounce OR i_units@5 = Oz) AND (i_size@3 = medium OR i_size@3 = extra large) OR (i_color@4 = brown OR i_color@4 = honeydew) AND (i_units@5 = Bunch OR i_units@5 = Ton) AND (i_size@3 = N/A OR i_size@3 = small)) OR i_category@1 = Men AND (i_color@4 = floral OR i_color@4 = deep) AND (i_units@5 = N/A OR i_units@5 = Dozen) AND i_size@3 = petite OR i_category@1 = Men AND (i_color@4 = light OR i_color@4 = cornflower) AND (i_units@5 = Box OR i_units@5 = Pound) AND (i_size@3 = medium OR i_size@3 = extra large) OR __common_expr_4@0 AND ((i_color@4 = midnight OR i_color@4 = snow) AND (i_units@5 = Pallet OR i_units@5 = Gross) AND (i_size@3 = medium OR i_size@3 = extra large) OR (i_color@4 = cyan OR i_color@4 = papaya) AND (i_units@5 = Cup OR i_units@5 = Dram) AND (i_size@3 = N/A OR i_size@3 = small)) OR i_category@1 = Men AND (i_color@4 = orange OR i_color@4 = frosted) AND (i_units@5 = Each OR i_units@5 = Tbl) AND i_size@3 = petite OR i_category@1 = Men AND (i_color@4 = forest OR i_color@4 = ghost) AND (i_units@5 = Lb OR i_units@5 = Bundle) AND (i_size@3 = medium OR i_size@3 = extra large), projection=[i_manufact@2] - │ ProjectionExec: expr=[i_category@0 = Women as __common_expr_4, i_category@0 as i_category, i_manufact@1 as i_manufact, i_size@2 as i_size, i_color@3 as i_color, i_units@4 as i_units] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_category, i_manufact, i_size, i_color, i_units], file_type=parquet - └────────────────────────────────────────────────── - "); + assert_snapshot!(display, @""); Ok(()) } #[tokio::test] @@ -4577,39 +4339,38 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [sum(store_sales.ss_ext_sales_price)@3 DESC, d_year@0 ASC NULLS LAST, i_category_id@1 ASC NULLS LAST, i_category@2 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[sum(store_sales.ss_ext_sales_price)@3 DESC, i_category_id@1 ASC NULLS LAST, i_category@2 ASC NULLS LAST], preserve_partitioning=[true] │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_category_id@1 as i_category_id, i_category@2 as i_category], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_year@0, i_category_id@1, i_category@2], 6), input_partitions=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([d_year@0, i_category_id@1, i_category@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_category_id@2 as i_category_id, i_category@3 as i_category], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@1, i_item_sk@0)], projection=[d_year@0, ss_ext_sales_price@2, i_category_id@4, i_category@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ FilterExec: i_manager_id@3 = 1, projection=[i_item_sk@0, i_category_id@1, i_category@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category_id, i_category, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 1 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 1 AND 1 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (1)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(dt.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ext_sales_price@5] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(dt.d_date_sk AS Float64)] │ FilterExec: d_moy@2 = 11 AND d_year@1 = 2000, projection=[d_date_sk@0, d_year@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 2000, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 2000, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (2000)] └────────────────────────────────────────────────── "); Ok(()) @@ -4620,43 +4381,38 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_store_name@0 ASC NULLS LAST, s_store_id@1 ASC NULLS LAST, sun_sales@2 ASC NULLS LAST, mon_sales@3 ASC NULLS LAST, tue_sales@4 ASC NULLS LAST, wed_sales@5 ASC NULLS LAST, thu_sales@6 ASC NULLS LAST, fri_sales@7 ASC NULLS LAST, sat_sales@8 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC NULLS LAST, s_store_id@1 ASC NULLS LAST, sun_sales@2 ASC NULLS LAST, mon_sales@3 ASC NULLS LAST, tue_sales@4 ASC NULLS LAST, wed_sales@5 ASC NULLS LAST, thu_sales@6 ASC NULLS LAST, fri_sales@7 ASC NULLS LAST, sat_sales@8 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[s_store_name@0 as s_store_name, s_store_id@1 as s_store_id, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[s_store_name@0 as s_store_name, s_store_id@1 as s_store_id], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC NULLS LAST, s_store_id@1 ASC NULLS LAST, sun_sales@2 ASC NULLS LAST, mon_sales@3 ASC NULLS LAST, tue_sales@4 ASC NULLS LAST, wed_sales@5 ASC NULLS LAST, thu_sales@6 ASC NULLS LAST, fri_sales@7 ASC NULLS LAST, sat_sales@8 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[s_store_name@0 as s_store_name, s_store_id@1 as s_store_id, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[s_store_name@0 as s_store_name, s_store_id@1 as s_store_id], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([s_store_name@0, s_store_id@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[s_store_name@3 as s_store_name, s_store_id@2 as s_store_id], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] - │ ProjectionExec: expr=[d_day_name@2 as d_day_name, ss_sales_price@3 as ss_sales_price, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, d_day_name@4, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_day_name@1, ss_store_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([s_store_name@0, s_store_id@1], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[s_store_name@3 as s_store_name, s_store_id@2 as s_store_id], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] + │ ProjectionExec: expr=[d_day_name@2 as d_day_name, ss_sales_price@3 as ss_sales_price, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, d_day_name@4, ss_sales_price@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_day_name@1, ss_store_sk@4, ss_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_id@1 as s_store_id, s_store_name@2 as s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_gmt_offset@3 = Some(-500),3,2, projection=[s_store_sk@0, s_store_id@1, s_store_name@2] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, s_gmt_offset], file_type=parquet, predicate=s_gmt_offset@27 = Some(-500),3,2, pruning_predicate=s_gmt_offset_null_count@2 != row_count@3 AND s_gmt_offset_min@0 <= Some(-500),3,2 AND Some(-500),3,2 <= s_gmt_offset_max@1, required_guarantees=[s_gmt_offset in (Some(-500),3,2)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_day_name@1 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0, d_day_name@2] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_day_name], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_id@1 as s_store_id, s_store_name@2 as s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_gmt_offset@3 = Some(-500),3,2, projection=[s_store_sk@0, s_store_id@1, s_store_name@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, s_gmt_offset], file_type=parquet, predicate=s_gmt_offset@27 = Some(-500),3,2, pruning_predicate=s_gmt_offset_null_count@2 != row_count@3 AND s_gmt_offset_min@0 <= Some(-500),3,2 AND Some(-500),3,2 <= s_gmt_offset_max@1, required_guarantees=[s_gmt_offset in (Some(-500),3,2)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_day_name@1 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0, d_day_name@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_day_name], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] - └────────────────────────────────────────────────── "#); Ok(()) } @@ -4666,34 +4422,32 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [rnk@0 ASC NULLS LAST], fetch=100 - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[rnk@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[rnk@1 as rnk, i_product_name@2 as best_performing, i_product_name@0 as worst_performing] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@1)], projection=[i_product_name@1, rnk@2, i_product_name@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ ProjectionExec: expr=[rnk@1 as rnk, item_sk@2 as item_sk, i_product_name@0 as i_product_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@0)], projection=[i_product_name@1, rnk@3, item_sk@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(rnk@1, rnk@1)], projection=[item_sk@0, rnk@1, item_sk@2] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=1 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_product_name], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_product_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_product_name], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_product_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] - │ RepartitionExec: partitioning=Hash([rnk@1], 6), input_partitions=3 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([rnk@1], 12), input_partitions=3 │ ProjectionExec: expr=[item_sk@0 as item_sk, rank() ORDER BY [v1.rank_col ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rnk] │ FilterExec: rank() ORDER BY [v1.rank_col ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 < 11 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1, maintains_sort_order=true @@ -4705,28 +4459,31 @@ mod tests { │ FilterExec: CAST(avg(ss1.ss_net_profit)@1 AS Decimal128(30, 15)) > CAST(0.9 * rank_col@2 AS Decimal128(30, 15)), projection=[ss_item_sk@0, avg(ss1.ss_net_profit)@1] │ NestedLoopJoinExec: join_type=Left │ CoalescePartitionsExec - │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 │ ProjectionExec: expr=[CAST(avg(store_sales.ss_net_profit)@1 AS Float64) as rank_col] │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(store_sales.ss_net_profit)], ordering_mode=Sorted - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] - │ FilterExec: ss_store_sk@1 = 4, projection=[ss_item_sk@0, ss_net_profit@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@7 = 4, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1, required_guarantees=[ss_store_sk in (4)] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] + │ FilterExec: ss_store_sk@1 = 4, projection=[ss_item_sk@0, ss_net_profit@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@7 = 4, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1, required_guarantees=[ss_store_sk in (4)] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ss_store_sk@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(store_sales.ss_net_profit)], ordering_mode=Sorted │ FilterExec: ss_store_sk@1 = 4 AND ss_addr_sk@0 IS NULL, projection=[ss_store_sk@1, ss_net_profit@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_addr_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@7 = 4 AND ss_addr_sk@6 IS NULL, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1 AND ss_addr_sk_null_count@4 > 0, required_guarantees=[ss_store_sk in (4)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] - │ RepartitionExec: partitioning=Hash([rnk@1], 6), input_partitions=3 + ┌───── Stage 10 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([rnk@1], 12), input_partitions=3 │ ProjectionExec: expr=[item_sk@0 as item_sk, rank() ORDER BY [v2.rank_col DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rnk] │ FilterExec: rank() ORDER BY [v2.rank_col DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 < 11 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1, maintains_sort_order=true @@ -4738,24 +4495,27 @@ mod tests { │ FilterExec: CAST(avg(ss1.ss_net_profit)@1 AS Decimal128(30, 15)) > CAST(0.9 * rank_col@2 AS Decimal128(30, 15)), projection=[ss_item_sk@0, avg(ss1.ss_net_profit)@1] │ NestedLoopJoinExec: join_type=Left │ CoalescePartitionsExec - │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 │ ProjectionExec: expr=[CAST(avg(store_sales.ss_net_profit)@1 AS Float64) as rank_col] │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(store_sales.ss_net_profit)], ordering_mode=Sorted - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] - │ FilterExec: ss_store_sk@1 = 4, projection=[ss_item_sk@0, ss_net_profit@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@7 = 4, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1, required_guarantees=[ss_store_sk in (4)] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] + │ FilterExec: ss_store_sk@1 = 4, projection=[ss_item_sk@0, ss_net_profit@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@7 = 4, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1, required_guarantees=[ss_store_sk in (4)] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ss_store_sk@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(store_sales.ss_net_profit)], ordering_mode=Sorted │ FilterExec: ss_store_sk@1 = 4 AND ss_addr_sk@0 IS NULL, projection=[ss_store_sk@1, ss_net_profit@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_addr_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@7 = 4 AND ss_addr_sk@6 IS NULL, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1 AND ss_addr_sk_null_count@4 > 0, required_guarantees=[ss_store_sk in (4)] └────────────────────────────────────────────────── "#); @@ -4767,65 +4527,69 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ca_zip@0 ASC NULLS LAST, ca_city@1 ASC NULLS LAST], fetch=100 - │ SortExec: TopK(fetch=100), expr=[ca_zip@0 ASC NULLS LAST, ca_city@1 ASC NULLS LAST], preserve_partitioning=[true] - │ AggregateExec: mode=FinalPartitioned, gby=[ca_zip@0 as ca_zip, ca_city@1 as ca_city], aggr=[sum(web_sales.ws_sales_price)] - │ RepartitionExec: partitioning=Hash([ca_zip@0, ca_city@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[ca_zip@2 as ca_zip, ca_city@1 as ca_city], aggr=[sum(web_sales.ws_sales_price)] - │ FilterExec: substr(ca_zip@2, 1, 5) IN (SET) ([85669, 86197, 88274, 83405, 86475, 85392, 85460, 80348, 81792]) OR mark@3, projection=[ws_sales_price@0, ca_city@1, ca_zip@2] - │ HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(i_item_id@3, i_item_id@0)], projection=[ws_sales_price@0, ca_city@1, ca_zip@2, mark@4] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_item_sk@0 IN (SET) ([2, 3, 5, 7, 11, 13, 17, 19, 23, 29]), projection=[i_item_id@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=i_item_sk@0 IN (SET) ([2, 3, 5, 7, 11, 13, 17, 19, 23, 29]), pruning_predicate=i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 2 AND 2 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 3 AND 3 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 5 AND 5 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 7 AND 7 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 11 AND 11 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 13 AND 13 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 17 AND 17 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 19 AND 19 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 23 AND 23 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 29 AND 29 <= i_item_sk_max@1, required_guarantees=[i_item_sk in (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)] + │ [Stage 8] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_sales_price@1, ca_city@2, ca_zip@3, i_item_id@5] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_sales_price@4, ca_city@5, ca_zip@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_sales_price@4 as ws_sales_price, ca_city@0 as ca_city, ca_zip@1 as ca_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[ca_city@1, ca_zip@2, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@5] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[ca_zip@0 ASC NULLS LAST, ca_city@1 ASC NULLS LAST], preserve_partitioning=[true] + │ AggregateExec: mode=FinalPartitioned, gby=[ca_zip@0 as ca_zip, ca_city@1 as ca_city], aggr=[sum(web_sales.ws_sales_price)] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([ca_zip@0, ca_city@1], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[ca_zip@2 as ca_zip, ca_city@1 as ca_city], aggr=[sum(web_sales.ws_sales_price)] + │ FilterExec: substr(ca_zip@2, 1, 5) IN (SET) ([85669, 86197, 88274, 83405, 86475, 85392, 85460, 80348, 81792]) OR mark@3, projection=[ws_sales_price@0, ca_city@1, ca_zip@2] + │ HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(i_item_id@3, i_item_id@0)], projection=[ws_sales_price@0, ca_city@1, ca_zip@2, mark@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_item_sk@2 as ws_item_sk, ws_sales_price@3 as ws_sales_price, c_current_addr_sk@0 as c_current_addr_sk] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@2, ws_bill_customer_sk@2)], projection=[c_current_addr_sk@1, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@6] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_item_sk@0 IN (SET) ([2, 3, 5, 7, 11, 13, 17, 19, 23, 29]), projection=[i_item_id@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=i_item_sk@0 IN (SET) ([2, 3, 5, 7, 11, 13, 17, 19, 23, 29]), pruning_predicate=i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 2 AND 2 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 3 AND 3 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 5 AND 5 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 7 AND 7 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 11 AND 11 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 13 AND 13 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 17 AND 17 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 19 AND 19 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 23 AND 23 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 29 AND 29 <= i_item_sk_max@1, required_guarantees=[i_item_sk in (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2001, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_city, ca_zip], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 9), input_partitions=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_customer_sk, ws_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_sales_price@1, ca_city@2, ca_zip@3, i_item_id@5] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_sales_price@4, ca_city@5, ca_zip@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_sales_price@4 as ws_sales_price, ca_city@0 as ca_city, ca_zip@1 as ca_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[ca_city@1, ca_zip@2, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_item_sk@2 as ws_item_sk, ws_sales_price@3 as ws_sales_price, c_current_addr_sk@0 as c_current_addr_sk] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@2, ws_bill_customer_sk@2)], projection=[c_current_addr_sk@1, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@6] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2001, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_city, ca_zip], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 12), input_partitions=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_customer_sk, ws_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── "); Ok(()) } @@ -4851,61 +4615,59 @@ mod tests { │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@4)], projection=[ss_ticket_number@0, bought_city@2, amt@3, profit@4, c_current_addr_sk@6, c_first_name@7, c_last_name@8] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] + ┌───── Stage 6 ── Tasks: t0:[p0..p5] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ca_city@3 as bought_city, sum(store_sales.ss_coupon_amt)@4 as amt, sum(store_sales.ss_net_profit)@5 as profit] │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ss_addr_sk@2 as ss_addr_sk, ca_city@3 as ca_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1, ss_addr_sk@2, ca_city@3], 6), input_partitions=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1, ss_addr_sk@2, ca_city@3], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@2 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk, ss_addr_sk@1 as ss_addr_sk, ca_city@5 as ca_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_customer_sk@0, ss_addr_sk@1, ss_ticket_number@2, ss_coupon_amt@3, ss_net_profit@4, ca_city@6] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_city@1 as ca_city, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_city], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_addr_sk@4, ss_ticket_number@5, ss_coupon_amt@6, ss_net_profit@7] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_ticket_number@6, ss_coupon_amt@7, ss_net_profit@8] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_store_sk@6, ss_ticket_number@7, ss_coupon_amt@8, ss_net_profit@9] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_ticket_number, ss_coupon_amt, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 OR hd_vehicle_count@2 = 3, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 OR hd_vehicle_count@4 = 3, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 OR hd_vehicle_count_null_count@6 != row_count@3 AND hd_vehicle_count_min@4 <= 3 AND 3 <= hd_vehicle_count_max@5, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 OR hd_vehicle_count@4 = 3, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 OR hd_vehicle_count_null_count@6 != row_count@3 AND hd_vehicle_count_min@4 <= 3 AND 3 <= hd_vehicle_count_max@5, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_city@1 = Fairview OR s_city@1 = Midway, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_city], file_type=parquet, predicate=s_city@22 = Fairview OR s_city@22 = Midway, pruning_predicate=s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Fairview AND Fairview <= s_city_max@1 OR s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Midway AND Midway <= s_city_max@1, required_guarantees=[s_city in (Fairview, Midway)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_city], file_type=parquet, predicate=s_city@22 = Fairview OR s_city@22 = Midway, pruning_predicate=s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Fairview AND Fairview <= s_city_max@1 OR s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Midway AND Midway <= s_city_max@1, required_guarantees=[s_city in (Fairview, Midway)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: (d_dow@2 = 6 OR d_dow@2 = 0) AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -4937,136 +4699,121 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3], 3), input_partitions=3 │ BoundedWindowAggExec: wdw=[avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Decimal128(19, 6) }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, s_store_name@2 ASC NULLS LAST, s_company_name@3 ASC NULLS LAST, d_year@4 ASC NULLS LAST, d_moy@5 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4], 3), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, d_year@4 as d_year, d_moy@5 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, sum(store_sales.ss_sales_price)@6 as sum_sales, rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 AS Decimal128(20, 0)) + Some(1),20,0 as v1_lag.rn + Decimal128(Some(1),20,0)] │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, s_store_name@2 ASC NULLS LAST, s_company_name@3 ASC NULLS LAST, d_year@4 ASC NULLS LAST, d_moy@5 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3], 3), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, d_year@4 as d_year, d_moy@5 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, sum(store_sales.ss_sales_price)@6 as sum_sales, rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 AS Decimal128(20, 0)) - Some(1),20,0 as v1_lead.rn - Decimal128(Some(1),20,0)] │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, s_store_name@2 ASC NULLS LAST, s_company_name@3 ASC NULLS LAST, d_year@4 ASC NULLS LAST, d_moy@5 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3], 3), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, d_year@4 as d_year, d_moy@5 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, d_year@4 as d_year, d_moy@5 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4, d_moy@5], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_sales_price@4 as ss_sales_price, d_year@5 as d_year, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@2)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_category@5, ss_sales_price@7, d_year@8, d_moy@9] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, ss_store_sk@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4, d_moy@5], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_sales_price@4 as ss_sales_price, d_year@5 as d_year, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@2)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_category@5, ss_sales_price@7, d_year@8, d_moy@9] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, ss_store_sk@7, ss_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, d_year@4 as d_year, d_moy@5 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4, d_moy@5], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_sales_price@4 as ss_sales_price, d_year@5 as d_year, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@2)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_category@5, ss_sales_price@7, d_year@8, d_moy@9] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, ss_store_sk@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, d_year@4 as d_year, d_moy@5 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4, d_moy@5], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_sales_price@4 as ss_sales_price, d_year@5 as d_year, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@2)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_category@5, ss_sales_price@7, d_year@8, d_moy@9] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, ss_store_sk@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4, d_moy@5], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_sales_price@4 as ss_sales_price, d_year@5 as d_year, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@2)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_category@5, ss_sales_price@7, d_year@8, d_moy@9] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, ss_store_sk@7, ss_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4, d_moy@5], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_sales_price@4 as ss_sales_price, d_year@5 as d_year, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@2)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_category@5, ss_sales_price@7, d_year@8, d_moy@9] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, ss_store_sk@7, ss_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet - └────────────────────────────────────────────────── "#); Ok(()) } @@ -5077,54 +4824,49 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ AggregateExec: mode=Final, gby=[], aggr=[sum(store_sales.ss_quantity)] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_quantity)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_quantity@1] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_quantity)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_quantity@1] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = CO OR ca_state@1 = OH OR ca_state@1 = TX) AND ss_net_profit@0 >= Some(0),6,2 AND ss_net_profit@0 <= Some(200000),6,2 OR (ca_state@1 = OR OR ca_state@1 = MN OR ca_state@1 = KY) AND ss_net_profit@0 >= Some(15000),6,2 AND ss_net_profit@0 <= Some(300000),6,2 OR (ca_state@1 = VA OR ca_state@1 = CA OR ca_state@1 = MS) AND ss_net_profit@0 >= Some(5000),6,2 AND CAST(ss_net_profit@0 AS Decimal128(22, 2)) <= Some(2500000),22,2, projection=[ss_sold_date_sk@0, ss_quantity@2] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = CO OR ca_state@1 = OH OR ca_state@1 = TX) AND ss_net_profit@0 >= Some(0),6,2 AND ss_net_profit@0 <= Some(200000),6,2 OR (ca_state@1 = OR OR ca_state@1 = MN OR ca_state@1 = KY) AND ss_net_profit@0 >= Some(15000),6,2 AND ss_net_profit@0 <= Some(300000),6,2 OR (ca_state@1 = VA OR ca_state@1 = CA OR ca_state@1 = MS) AND ss_net_profit@0 >= Some(5000),6,2 AND CAST(ss_net_profit@0 AS Decimal128(22, 2)) <= Some(2500000),22,2, projection=[ss_sold_date_sk@0, ss_quantity@2] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: (ca_state@1 = CO OR ca_state@1 = OH OR ca_state@1 = TX OR ca_state@1 = OR OR ca_state@1 = MN OR ca_state@1 = KY OR ca_state@1 = VA OR ca_state@1 = CA OR ca_state@1 = MS) AND ca_state@1 IN (SET) ([CO, OH, TX, OR, MN, KY, VA, CA, MS]) AND ca_country@2 = United States, projection=[ca_address_sk@0, ca_state@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=(ca_state@8 = CO OR ca_state@8 = OH OR ca_state@8 = TX OR ca_state@8 = OR OR ca_state@8 = MN OR ca_state@8 = KY OR ca_state@8 = VA OR ca_state@8 = CA OR ca_state@8 = MS) AND ca_state@8 IN (SET) ([CO, OH, TX, OR, MN, KY, VA, CA, MS]) AND ca_country@10 = United States, pruning_predicate=(ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CO AND CO <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= TX AND TX <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OR AND OR <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MN AND MN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CA AND CA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1) AND (ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CO AND CO <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= TX AND TX <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OR AND OR <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MN AND MN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CA AND CA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1) AND ca_country_null_count@6 != row_count@3 AND ca_country_min@4 <= United States AND United States <= ca_country_max@5, required_guarantees=[ca_country in (United States), ca_state in (CA, CO, KY, MN, MS, OH, OR, TX, VA)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = 4 yr Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = D AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2, projection=[ss_sold_date_sk@0, ss_addr_sk@2, ss_quantity@3, ss_net_profit@5] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: (ca_state@1 = CO OR ca_state@1 = OH OR ca_state@1 = TX OR ca_state@1 = OR OR ca_state@1 = MN OR ca_state@1 = KY OR ca_state@1 = VA OR ca_state@1 = CA OR ca_state@1 = MS) AND ca_state@1 IN (SET) ([CO, OH, TX, OR, MN, KY, VA, CA, MS]) AND ca_country@2 = United States, projection=[ca_address_sk@0, ca_state@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=(ca_state@8 = CO OR ca_state@8 = OH OR ca_state@8 = TX OR ca_state@8 = OR OR ca_state@8 = MN OR ca_state@8 = KY OR ca_state@8 = VA OR ca_state@8 = CA OR ca_state@8 = MS) AND ca_state@8 IN (SET) ([CO, OH, TX, OR, MN, KY, VA, CA, MS]) AND ca_country@10 = United States, pruning_predicate=(ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CO AND CO <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= TX AND TX <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OR AND OR <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MN AND MN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CA AND CA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1) AND (ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CO AND CO <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= TX AND TX <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OR AND OR <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MN AND MN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CA AND CA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1) AND ca_country_null_count@6 != row_count@3 AND ca_country_min@4 <= United States AND United States <= ca_country_max@5, required_guarantees=[ca_country in (United States), ca_state in (CA, CO, KY, MN, MS, OH, OR, TX, VA)] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = 4 yr Degree OR cd_marital_status@1 = D AND cd_education_status@2 = 2 yr Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = 4 yr Degree OR cd_marital_status@2 = D AND cd_education_status@3 = 2 yr Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 4 yr Degree AND 4 yr Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= D AND D <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, 4 yr Degree, College), cd_marital_status in (D, M, S)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = 4 yr Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = D AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2, projection=[ss_sold_date_sk@0, ss_addr_sk@2, ss_quantity@3, ss_net_profit@5] + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_cdemo_sk@3, ss_addr_sk@4, ss_quantity@6, ss_sales_price@7, ss_net_profit@8] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = 4 yr Degree OR cd_marital_status@1 = D AND cd_education_status@2 = 2 yr Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = 4 yr Degree OR cd_marital_status@2 = D AND cd_education_status@3 = 2 yr Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 4 yr Degree AND 4 yr Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= D AND D <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, 4 yr Degree, College), cd_marital_status in (D, M, S)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_cdemo_sk@3, ss_addr_sk@4, ss_quantity@6, ss_sales_price@7, ss_net_profit@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ FilterExec: (ss_net_profit@6 >= Some(0),6,2 AND ss_net_profit@6 <= Some(200000),6,2 OR ss_net_profit@6 >= Some(15000),6,2 AND ss_net_profit@6 <= Some(300000),6,2 OR ss_net_profit@6 >= Some(5000),6,2 AND CAST(ss_net_profit@6 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price@5 >= Some(10000),5,2 AND ss_sales_price@5 <= Some(15000),5,2 OR ss_sales_price@5 >= Some(5000),5,2 AND ss_sales_price@5 <= Some(10000),5,2 OR ss_sales_price@5 >= Some(15000),5,2 AND ss_sales_price@5 <= Some(20000),5,2) - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_cdemo_sk, ss_addr_sk, ss_store_sk, ss_quantity, ss_sales_price, ss_net_profit], file_type=parquet, predicate=(ss_net_profit@22 >= Some(0),6,2 AND ss_net_profit@22 <= Some(200000),6,2 OR ss_net_profit@22 >= Some(15000),6,2 AND ss_net_profit@22 <= Some(300000),6,2 OR ss_net_profit@22 >= Some(5000),6,2 AND CAST(ss_net_profit@22 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price@13 >= Some(10000),5,2 AND ss_sales_price@13 <= Some(15000),5,2 OR ss_sales_price@13 >= Some(5000),5,2 AND ss_sales_price@13 <= Some(10000),5,2 OR ss_sales_price@13 >= Some(15000),5,2 AND ss_sales_price@13 <= Some(20000),5,2) AND DynamicFilter [ empty ], pruning_predicate=(ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(0),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(200000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(15000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(300000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(5000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND CAST(ss_net_profit_min@3 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(10000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(15000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(5000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(10000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(15000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ FilterExec: (ss_net_profit@6 >= Some(0),6,2 AND ss_net_profit@6 <= Some(200000),6,2 OR ss_net_profit@6 >= Some(15000),6,2 AND ss_net_profit@6 <= Some(300000),6,2 OR ss_net_profit@6 >= Some(5000),6,2 AND CAST(ss_net_profit@6 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price@5 >= Some(10000),5,2 AND ss_sales_price@5 <= Some(15000),5,2 OR ss_sales_price@5 >= Some(5000),5,2 AND ss_sales_price@5 <= Some(10000),5,2 OR ss_sales_price@5 >= Some(15000),5,2 AND ss_sales_price@5 <= Some(20000),5,2) + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_cdemo_sk, ss_addr_sk, ss_store_sk, ss_quantity, ss_sales_price, ss_net_profit], file_type=parquet, predicate=(ss_net_profit@22 >= Some(0),6,2 AND ss_net_profit@22 <= Some(200000),6,2 OR ss_net_profit@22 >= Some(15000),6,2 AND ss_net_profit@22 <= Some(300000),6,2 OR ss_net_profit@22 >= Some(5000),6,2 AND CAST(ss_net_profit@22 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price@13 >= Some(10000),5,2 AND ss_sales_price@13 <= Some(15000),5,2 OR ss_sales_price@13 >= Some(5000),5,2 AND ss_sales_price@13 <= Some(10000),5,2 OR ss_sales_price@13 >= Some(15000),5,2 AND ss_sales_price@13 <= Some(20000),5,2) AND DynamicFilter [ empty ], pruning_predicate=(ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(0),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(200000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(15000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(300000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(5000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND CAST(ss_net_profit_min@3 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(10000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(15000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(5000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(10000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(15000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── "); Ok(()) } @@ -5136,151 +4878,139 @@ mod tests { │ SortPreservingMergeExec: [channel@0 ASC, return_rank@3 ASC, currency_rank@4 ASC, item@1 ASC], fetch=100 │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, return_rank@3 ASC, currency_rank@4 ASC, item@1 ASC], preserve_partitioning=[true] │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, item@1 as item, return_ratio@2 as return_ratio, return_rank@3 as return_rank, currency_rank@4 as currency_rank], aggr=[] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ RepartitionExec: partitioning=Hash([channel@0, item@1, return_ratio@2, return_rank@3, currency_rank@4], 3), input_partitions=9 + │ AggregateExec: mode=Partial, gby=[channel@0 as channel, item@1 as item, return_ratio@2 as return_ratio, return_rank@3 as return_rank, currency_rank@4 as currency_rank], aggr=[], ordering_mode=PartiallySorted([0, 4]) + │ DistributedUnionExec: t0:[c0, c1, c2] + │ ProjectionExec: expr=[web as channel, item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as return_rank, rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as currency_rank] + │ FilterExec: rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 <= 10 OR rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 <= 10 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1, maintains_sort_order=true + │ ProjectionExec: expr=[item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] + │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] + │ SortExec: expr=[currency_ratio@2 ASC NULLS LAST], preserve_partitioning=[false] + │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] + │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] + │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[ws_item_sk@0 as item, CAST(sum(coalesce(wr.wr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(ws.ws_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(wr.wr_return_amt,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(ws.ws_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] + │ AggregateExec: mode=FinalPartitioned, gby=[ws_item_sk@0 as ws_item_sk], aggr=[sum(coalesce(wr.wr_return_quantity,Int64(0))), sum(coalesce(ws.ws_quantity,Int64(0))), sum(coalesce(wr.wr_return_amt,Int64(0))), sum(coalesce(ws.ws_net_paid,Int64(0)))] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[catalog as channel, item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as return_rank, rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as currency_rank] + │ FilterExec: rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 <= 10 OR rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 <= 10 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1, maintains_sort_order=true + │ ProjectionExec: expr=[item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] + │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] + │ SortExec: expr=[currency_ratio@2 ASC NULLS LAST], preserve_partitioning=[false] + │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] + │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] + │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[cs_item_sk@0 as item, CAST(sum(coalesce(cr.cr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(cs.cs_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(cr.cr_return_amount,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(cs.cs_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(coalesce(cr.cr_return_quantity,Int64(0))), sum(coalesce(cs.cs_quantity,Int64(0))), sum(coalesce(cr.cr_return_amount,Int64(0))), sum(coalesce(cs.cs_net_paid,Int64(0)))] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[store as channel, item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as return_rank, rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as currency_rank] + │ FilterExec: rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 <= 10 OR rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 <= 10 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1, maintains_sort_order=true + │ ProjectionExec: expr=[item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] + │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] + │ SortExec: expr=[currency_ratio@2 ASC NULLS LAST], preserve_partitioning=[false] + │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] + │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] + │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[ss_item_sk@0 as item, CAST(sum(coalesce(sr.sr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(sts.ss_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(sr.sr_return_amt,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(sts.ss_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[sum(coalesce(sr.sr_return_quantity,Int64(0))), sum(coalesce(sts.ss_quantity,Int64(0))), sum(coalesce(sr.sr_return_amt,Int64(0))), sum(coalesce(sts.ss_net_paid,Int64(0)))] + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([channel@0, item@1, return_ratio@2, return_rank@3, currency_rank@4], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[channel@0 as channel, item@1 as item, return_ratio@2 as return_ratio, return_rank@3 as return_rank, currency_rank@4 as currency_rank], aggr=[], ordering_mode=PartiallySorted([0, 4]) - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] - │ ProjectionExec: expr=[web as channel, item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as return_rank, rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as currency_rank] - │ FilterExec: rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 <= 10 OR rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 <= 10 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1, maintains_sort_order=true - │ ProjectionExec: expr=[item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] - │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_web.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] - │ SortExec: expr=[currency_ratio@2 ASC NULLS LAST], preserve_partitioning=[false] - │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_web.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] - │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] - │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[ws_item_sk@0 as item, CAST(sum(coalesce(wr.wr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(ws.ws_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(wr.wr_return_amt,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(ws.ws_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] - │ AggregateExec: mode=FinalPartitioned, gby=[ws_item_sk@0 as ws_item_sk], aggr=[sum(coalesce(wr.wr_return_quantity,Int64(0))), sum(coalesce(ws.ws_quantity,Int64(0))), sum(coalesce(wr.wr_return_amt,Int64(0))), sum(coalesce(ws.ws_net_paid,Int64(0)))] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ ProjectionExec: expr=[catalog as channel, item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as return_rank, rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as currency_rank] - │ FilterExec: rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 <= 10 OR rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 <= 10 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1, maintains_sort_order=true - │ ProjectionExec: expr=[item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] - │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] - │ SortExec: expr=[currency_ratio@2 ASC NULLS LAST], preserve_partitioning=[false] - │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] - │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[store as channel, item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as return_rank, rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as currency_rank] - │ FilterExec: rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 <= 10 OR rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 <= 10 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1, maintains_sort_order=true - │ ProjectionExec: expr=[item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] - │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] - │ SortExec: expr=[currency_ratio@2 ASC NULLS LAST], preserve_partitioning=[false] - │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] - │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] - │ [Stage 14] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([ws_item_sk@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[ws_item_sk@2 as ws_item_sk], aggr=[sum(coalesce(wr.wr_return_quantity,Int64(0))), sum(coalesce(ws.ws_quantity,Int64(0))), sum(coalesce(wr.wr_return_amt,Int64(0))), sum(coalesce(ws.ws_net_paid,Int64(0)))] - │ ProjectionExec: expr=[CAST(wr_return_amt@4 AS Decimal128(22, 2)) as __common_expr_1, CAST(ws_net_paid@2 AS Decimal128(22, 2)) as __common_expr_2, ws_item_sk@0 as ws_item_sk, ws_quantity@1 as ws_quantity, wr_return_quantity@3 as wr_return_quantity] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_quantity@4, ws_net_paid@5, wr_return_quantity@6, wr_return_amt@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ FilterExec: wr_return_amt@5 > Some(1000000),7,2 - │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_quantity@4 as ws_quantity, ws_net_paid@5 as ws_net_paid, wr_return_quantity@0 as wr_return_quantity, wr_return_amt@1 as wr_return_amt] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(wr_order_number@1, ws_order_number@2), (wr_item_sk@0, ws_item_sk@1)], projection=[wr_return_quantity@2, wr_return_amt@3, ws_sold_date_sk@4, ws_item_sk@5, ws_quantity@7, ws_net_paid@8] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 12, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ RepartitionExec: partitioning=Hash([wr_order_number@1, wr_item_sk@0], 6), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number, wr_return_quantity, wr_return_amt], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ws_order_number@2, ws_item_sk@1], 6), input_partitions=2 - │ FilterExec: ws_net_profit@5 > Some(100),7,2 AND ws_net_paid@4 > Some(0),7,2 AND ws_quantity@3 > 0, projection=[ws_sold_date_sk@0, ws_item_sk@1, ws_order_number@2, ws_quantity@3, ws_net_paid@4] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_order_number, ws_quantity, ws_net_paid, ws_net_profit], file_type=parquet, predicate=ws_net_profit@33 > Some(100),7,2 AND ws_net_paid@29 > Some(0),7,2 AND ws_quantity@18 > 0, pruning_predicate=ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 > Some(100),7,2 AND ws_net_paid_null_count@4 != row_count@2 AND ws_net_paid_max@3 > Some(0),7,2 AND ws_quantity_null_count@6 != row_count@2 AND ws_quantity_max@5 > 0, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[cs_item_sk@0 as item, CAST(sum(coalesce(cr.cr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(cs.cs_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(cr.cr_return_amount,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(cs.cs_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(coalesce(cr.cr_return_quantity,Int64(0))), sum(coalesce(cs.cs_quantity,Int64(0))), sum(coalesce(cr.cr_return_amount,Int64(0))), sum(coalesce(cs.cs_net_paid,Int64(0)))] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[cs_item_sk@2 as cs_item_sk], aggr=[sum(coalesce(cr.cr_return_quantity,Int64(0))), sum(coalesce(cs.cs_quantity,Int64(0))), sum(coalesce(cr.cr_return_amount,Int64(0))), sum(coalesce(cs.cs_net_paid,Int64(0)))] - │ ProjectionExec: expr=[CAST(cr_return_amount@4 AS Decimal128(22, 2)) as __common_expr_3, CAST(cs_net_paid@2 AS Decimal128(22, 2)) as __common_expr_4, cs_item_sk@0 as cs_item_sk, cs_quantity@1 as cs_quantity, cr_return_quantity@3 as cr_return_quantity] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_quantity@4, cs_net_paid@5, cr_return_quantity@6, cr_return_amount@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ FilterExec: cr_return_amount@5 > Some(1000000),7,2 - │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_item_sk@3 as cs_item_sk, cs_quantity@4 as cs_quantity, cs_net_paid@5 as cs_net_paid, cr_return_quantity@0 as cr_return_quantity, cr_return_amount@1 as cr_return_amount] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@2), (cr_item_sk@0, cs_item_sk@1)], projection=[cr_return_quantity@2, cr_return_amount@3, cs_sold_date_sk@4, cs_item_sk@5, cs_quantity@7, cs_net_paid@8] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 12, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_return_quantity, cr_return_amount], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_order_number@2, cs_item_sk@1], 9), input_partitions=2 - │ FilterExec: cs_net_profit@5 > Some(100),7,2 AND cs_net_paid@4 > Some(0),7,2 AND cs_quantity@3 > 0, projection=[cs_sold_date_sk@0, cs_item_sk@1, cs_order_number@2, cs_quantity@3, cs_net_paid@4] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_order_number, cs_quantity, cs_net_paid, cs_net_profit], file_type=parquet, predicate=cs_net_profit@33 > Some(100),7,2 AND cs_net_paid@29 > Some(0),7,2 AND cs_quantity@18 > 0, pruning_predicate=cs_net_profit_null_count@1 != row_count@2 AND cs_net_profit_max@0 > Some(100),7,2 AND cs_net_paid_null_count@4 != row_count@2 AND cs_net_paid_max@3 > Some(0),7,2 AND cs_quantity_null_count@6 != row_count@2 AND cs_quantity_max@5 > 0, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[ss_item_sk@0 as item, CAST(sum(coalesce(sr.sr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(sts.ss_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(sr.sr_return_amt,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(sts.ss_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] - │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[sum(coalesce(sr.sr_return_quantity,Int64(0))), sum(coalesce(sts.ss_quantity,Int64(0))), sum(coalesce(sr.sr_return_amt,Int64(0))), sum(coalesce(sts.ss_net_paid,Int64(0)))] - │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[ss_item_sk@2 as ss_item_sk], aggr=[sum(coalesce(sr.sr_return_quantity,Int64(0))), sum(coalesce(sts.ss_quantity,Int64(0))), sum(coalesce(sr.sr_return_amt,Int64(0))), sum(coalesce(sts.ss_net_paid,Int64(0)))] - │ ProjectionExec: expr=[CAST(sr_return_amt@4 AS Decimal128(22, 2)) as __common_expr_5, CAST(ss_net_paid@2 AS Decimal128(22, 2)) as __common_expr_6, ss_item_sk@0 as ss_item_sk, ss_quantity@1 as ss_quantity, sr_return_quantity@3 as sr_return_quantity] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_quantity@4, ss_net_paid@5, sr_return_quantity@6, sr_return_amt@7] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ FilterExec: sr_return_amt@5 > Some(1000000),7,2 - │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_quantity@4 as ss_quantity, ss_net_paid@5 as ss_net_paid, sr_return_quantity@0 as sr_return_quantity, sr_return_amt@1 as sr_return_amt] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_ticket_number@1, ss_ticket_number@2), (sr_item_sk@0, ss_item_sk@1)], projection=[sr_return_quantity@2, sr_return_amt@3, ss_sold_date_sk@4, ss_item_sk@5, ss_quantity@7, ss_net_paid@8] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 12, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number, sr_return_quantity, sr_return_amt], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@2, ss_item_sk@1], 9), input_partitions=2 - │ FilterExec: ss_net_profit@5 > Some(100),6,2 AND ss_net_paid@4 > Some(0),7,2 AND ss_quantity@3 > 0, projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_ticket_number@2, ss_quantity@3, ss_net_paid@4] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ticket_number, ss_quantity, ss_net_paid, ss_net_profit], file_type=parquet, predicate=ss_net_profit@22 > Some(100),6,2 AND ss_net_paid@20 > Some(0),7,2 AND ss_quantity@10 > 0, pruning_predicate=ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 > Some(100),6,2 AND ss_net_paid_null_count@4 != row_count@2 AND ss_net_paid_max@3 > Some(0),7,2 AND ss_quantity_null_count@6 != row_count@2 AND ss_quantity_max@5 > 0, required_guarantees=[] - └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ws_item_sk@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[ws_item_sk@2 as ws_item_sk], aggr=[sum(coalesce(wr.wr_return_quantity,Int64(0))), sum(coalesce(ws.ws_quantity,Int64(0))), sum(coalesce(wr.wr_return_amt,Int64(0))), sum(coalesce(ws.ws_net_paid,Int64(0)))] + │ ProjectionExec: expr=[CAST(wr_return_amt@4 AS Decimal128(22, 2)) as __common_expr_1, CAST(ws_net_paid@2 AS Decimal128(22, 2)) as __common_expr_2, ws_item_sk@0 as ws_item_sk, ws_quantity@1 as ws_quantity, wr_return_quantity@3 as wr_return_quantity] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_quantity@4, ws_net_paid@5, wr_return_quantity@6, wr_return_amt@7] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ FilterExec: wr_return_amt@5 > Some(1000000),7,2 + │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_quantity@4 as ws_quantity, ws_net_paid@5 as ws_net_paid, wr_return_quantity@0 as wr_return_quantity, wr_return_amt@1 as wr_return_amt] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(wr_order_number@1, ws_order_number@2), (wr_item_sk@0, ws_item_sk@1)], projection=[wr_return_quantity@2, wr_return_amt@3, ws_sold_date_sk@4, ws_item_sk@5, ws_quantity@7, ws_net_paid@8] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 12, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([wr_order_number@1, wr_item_sk@0], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number, wr_return_quantity, wr_return_amt], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ws_order_number@2, ws_item_sk@1], 12), input_partitions=2 + │ FilterExec: ws_net_profit@5 > Some(100),7,2 AND ws_net_paid@4 > Some(0),7,2 AND ws_quantity@3 > 0, projection=[ws_sold_date_sk@0, ws_item_sk@1, ws_order_number@2, ws_quantity@3, ws_net_paid@4] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_order_number, ws_quantity, ws_net_paid, ws_net_profit], file_type=parquet, predicate=ws_net_profit@33 > Some(100),7,2 AND ws_net_paid@29 > Some(0),7,2 AND ws_quantity@18 > 0, pruning_predicate=ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 > Some(100),7,2 AND ws_net_paid_null_count@4 != row_count@2 AND ws_net_paid_max@3 > Some(0),7,2 AND ws_quantity_null_count@6 != row_count@2 AND ws_quantity_max@5 > 0, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[cs_item_sk@2 as cs_item_sk], aggr=[sum(coalesce(cr.cr_return_quantity,Int64(0))), sum(coalesce(cs.cs_quantity,Int64(0))), sum(coalesce(cr.cr_return_amount,Int64(0))), sum(coalesce(cs.cs_net_paid,Int64(0)))] + │ ProjectionExec: expr=[CAST(cr_return_amount@4 AS Decimal128(22, 2)) as __common_expr_3, CAST(cs_net_paid@2 AS Decimal128(22, 2)) as __common_expr_4, cs_item_sk@0 as cs_item_sk, cs_quantity@1 as cs_quantity, cr_return_quantity@3 as cr_return_quantity] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_quantity@4, cs_net_paid@5, cr_return_quantity@6, cr_return_amount@7] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ FilterExec: cr_return_amount@5 > Some(1000000),7,2 + │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_item_sk@3 as cs_item_sk, cs_quantity@4 as cs_quantity, cs_net_paid@5 as cs_net_paid, cr_return_quantity@0 as cr_return_quantity, cr_return_amount@1 as cr_return_amount] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@2), (cr_item_sk@0, cs_item_sk@1)], projection=[cr_return_quantity@2, cr_return_amount@3, cs_sold_date_sk@4, cs_item_sk@5, cs_quantity@7, cs_net_paid@8] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 12, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_return_quantity, cr_return_amount], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_order_number@2, cs_item_sk@1], 12), input_partitions=2 + │ FilterExec: cs_net_profit@5 > Some(100),7,2 AND cs_net_paid@4 > Some(0),7,2 AND cs_quantity@3 > 0, projection=[cs_sold_date_sk@0, cs_item_sk@1, cs_order_number@2, cs_quantity@3, cs_net_paid@4] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_order_number, cs_quantity, cs_net_paid, cs_net_profit], file_type=parquet, predicate=cs_net_profit@33 > Some(100),7,2 AND cs_net_paid@29 > Some(0),7,2 AND cs_quantity@18 > 0, pruning_predicate=cs_net_profit_null_count@1 != row_count@2 AND cs_net_profit_max@0 > Some(100),7,2 AND cs_net_paid_null_count@4 != row_count@2 AND cs_net_paid_max@3 > Some(0),7,2 AND cs_quantity_null_count@6 != row_count@2 AND cs_quantity_max@5 > 0, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[ss_item_sk@2 as ss_item_sk], aggr=[sum(coalesce(sr.sr_return_quantity,Int64(0))), sum(coalesce(sts.ss_quantity,Int64(0))), sum(coalesce(sr.sr_return_amt,Int64(0))), sum(coalesce(sts.ss_net_paid,Int64(0)))] + │ ProjectionExec: expr=[CAST(sr_return_amt@4 AS Decimal128(22, 2)) as __common_expr_5, CAST(ss_net_paid@2 AS Decimal128(22, 2)) as __common_expr_6, ss_item_sk@0 as ss_item_sk, ss_quantity@1 as ss_quantity, sr_return_quantity@3 as sr_return_quantity] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_quantity@4, ss_net_paid@5, sr_return_quantity@6, sr_return_amt@7] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ FilterExec: sr_return_amt@5 > Some(1000000),7,2 + │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_quantity@4 as ss_quantity, ss_net_paid@5 as ss_net_paid, sr_return_quantity@0 as sr_return_quantity, sr_return_amt@1 as sr_return_amt] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_ticket_number@1, ss_ticket_number@2), (sr_item_sk@0, ss_item_sk@1)], projection=[sr_return_quantity@2, sr_return_amt@3, ss_sold_date_sk@4, ss_item_sk@5, ss_quantity@7, ss_net_paid@8] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 12, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number, sr_return_quantity, sr_return_amt], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@2, ss_item_sk@1], 12), input_partitions=2 + │ FilterExec: ss_net_profit@5 > Some(100),6,2 AND ss_net_paid@4 > Some(0),7,2 AND ss_quantity@3 > 0, projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_ticket_number@2, ss_quantity@3, ss_net_paid@4] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ticket_number, ss_quantity, ss_net_paid, ss_net_profit], file_type=parquet, predicate=ss_net_profit@22 > Some(100),6,2 AND ss_net_paid@20 > Some(0),7,2 AND ss_quantity@10 > 0, pruning_predicate=ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 > Some(100),6,2 AND ss_net_paid_null_count@4 != row_count@2 AND ss_net_paid_max@3 > Some(0),7,2 AND ss_quantity_null_count@6 != row_count@2 AND ss_quantity_max@5 > 0, required_guarantees=[] + └────────────────────────────────────────────────── "#); Ok(()) } @@ -5290,63 +5020,59 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_store_name@0 ASC NULLS LAST, s_company_id@1 ASC NULLS LAST, s_street_number@2 ASC NULLS LAST, s_street_name@3 ASC NULLS LAST, s_street_type@4 ASC NULLS LAST, s_suite_number@5 ASC NULLS LAST, s_city@6 ASC NULLS LAST, s_county@7 ASC NULLS LAST, s_state@8 ASC NULLS LAST, s_zip@9 ASC NULLS LAST], fetch=100 - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC NULLS LAST, s_company_id@1 ASC NULLS LAST, s_street_number@2 ASC NULLS LAST, s_street_name@3 ASC NULLS LAST, s_street_type@4 ASC NULLS LAST, s_suite_number@5 ASC NULLS LAST, s_city@6 ASC NULLS LAST, s_county@7 ASC NULLS LAST, s_state@8 ASC NULLS LAST, s_zip@9 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@10 as 30 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@11 as 31-60 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@12 as 61-90 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@13 as 91-120 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@14 as >120 days] + │ AggregateExec: mode=FinalPartitioned, gby=[s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip], aggr=[sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC NULLS LAST, s_company_id@1 ASC NULLS LAST, s_street_number@2 ASC NULLS LAST, s_street_name@3 ASC NULLS LAST, s_street_type@4 ASC NULLS LAST, s_suite_number@5 ASC NULLS LAST, s_city@6 ASC NULLS LAST, s_county@7 ASC NULLS LAST, s_state@8 ASC NULLS LAST, s_zip@9 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@10 as 30 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@11 as 31-60 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@12 as 61-90 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@13 as 91-120 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@14 as >120 days] - │ AggregateExec: mode=FinalPartitioned, gby=[s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip], aggr=[sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([s_store_name@0, s_company_id@1, s_street_number@2, s_street_name@3, s_street_type@4, s_suite_number@5, s_city@6, s_county@7, s_state@8, s_zip@9], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[s_store_name@1 as s_store_name, s_company_id@2 as s_company_id, s_street_number@3 as s_street_number, s_street_name@4 as s_street_name, s_street_type@5 as s_street_type, s_suite_number@6 as s_suite_number, s_city@7 as s_city, s_county@8 as s_county, s_state@9 as s_state, s_zip@10 as s_zip], aggr=[sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] - │ ProjectionExec: expr=[sr_returned_date_sk@1 - ss_sold_date_sk@0 as __common_expr_1, s_store_name@2 as s_store_name, s_company_id@3 as s_company_id, s_street_number@4 as s_street_number, s_street_name@5 as s_street_name, s_street_type@6 as s_street_type, s_suite_number@7 as s_suite_number, s_city@8 as s_city, s_county@9 as s_county, s_state@10 as s_state, s_zip@11 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(sr_returned_date_sk@1, CAST(d2.d_date_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, sr_returned_date_sk@1, s_store_name@2, s_company_id@3, s_street_number@4, s_street_name@5, s_street_type@6, s_suite_number@7, s_city@8, s_county@9, s_state@10, s_zip@11] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 8, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 8, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 8 AND 8 <= d_moy_max@5, required_guarantees=[d_moy in (8), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(d1.d_date_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, sr_returned_date_sk@1, s_store_name@2, s_company_id@3, s_street_number@4, s_street_name@5, s_street_type@6, s_suite_number@7, s_city@8, s_county@9, s_state@10, s_zip@11] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_sold_date_sk@10 as ss_sold_date_sk, sr_returned_date_sk@11 as sr_returned_date_sk, s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@11, ss_store_sk@1)], projection=[s_store_name@1, s_company_id@2, s_street_number@3, s_street_name@4, s_street_type@5, s_suite_number@6, s_city@7, s_county@8, s_state@9, s_zip@10, ss_sold_date_sk@12, sr_returned_date_sk@14] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_store_sk@2 as ss_store_sk, sr_returned_date_sk@0 as sr_returned_date_sk] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@3, ss_ticket_number@4), (sr_item_sk@1, ss_item_sk@1), (sr_customer_sk@2, ss_customer_sk@2)], projection=[sr_returned_date_sk@0, ss_sold_date_sk@4, ss_store_sk@7] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, s_suite_number, s_city, s_county, s_state, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_ticket_number@3, sr_item_sk@1, sr_customer_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@4, ss_item_sk@1, ss_customer_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([s_store_name@0, s_company_id@1, s_street_number@2, s_street_name@3, s_street_type@4, s_suite_number@5, s_city@6, s_county@7, s_state@8, s_zip@9], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[s_store_name@1 as s_store_name, s_company_id@2 as s_company_id, s_street_number@3 as s_street_number, s_street_name@4 as s_street_name, s_street_type@5 as s_street_type, s_suite_number@6 as s_suite_number, s_city@7 as s_city, s_county@8 as s_county, s_state@9 as s_state, s_zip@10 as s_zip], aggr=[sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] + │ ProjectionExec: expr=[sr_returned_date_sk@1 - ss_sold_date_sk@0 as __common_expr_1, s_store_name@2 as s_store_name, s_company_id@3 as s_company_id, s_street_number@4 as s_street_number, s_street_name@5 as s_street_name, s_street_type@6 as s_street_type, s_suite_number@7 as s_suite_number, s_city@8 as s_city, s_county@9 as s_county, s_state@10 as s_state, s_zip@11 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(sr_returned_date_sk@1, CAST(d2.d_date_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, sr_returned_date_sk@1, s_store_name@2, s_company_id@3, s_street_number@4, s_street_name@5, s_street_type@6, s_suite_number@7, s_city@8, s_county@9, s_state@10, s_zip@11] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 8, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 8, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 8 AND 8 <= d_moy_max@5, required_guarantees=[d_moy in (8), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(d1.d_date_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, sr_returned_date_sk@1, s_store_name@2, s_company_id@3, s_street_number@4, s_street_name@5, s_street_type@6, s_suite_number@7, s_city@8, s_county@9, s_state@10, s_zip@11] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ss_sold_date_sk@10 as ss_sold_date_sk, sr_returned_date_sk@11 as sr_returned_date_sk, s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@11, ss_store_sk@1)], projection=[s_store_name@1, s_company_id@2, s_street_number@3, s_street_name@4, s_street_type@5, s_suite_number@6, s_city@7, s_county@8, s_state@9, s_zip@10, ss_sold_date_sk@12, sr_returned_date_sk@14] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_store_sk@2 as ss_store_sk, sr_returned_date_sk@0 as sr_returned_date_sk] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@3, ss_ticket_number@4), (sr_item_sk@1, ss_item_sk@1), (sr_customer_sk@2, ss_customer_sk@2)], projection=[sr_returned_date_sk@0, ss_sold_date_sk@4, ss_store_sk@7] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, s_suite_number, s_city, s_county, s_state, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@3, sr_item_sk@1, sr_customer_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@4, ss_item_sk@1, ss_customer_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── "); Ok(()) } @@ -5368,60 +5094,52 @@ mod tests { │ ProjectionExec: expr=[ws_item_sk@0 as item_sk, d_date@1 as d_date, sum(sum(web_sales.ws_sales_price)) PARTITION BY [web_sales.ws_item_sk] ORDER BY [date_dim.d_date ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as cume_sales] │ BoundedWindowAggExec: wdw=[sum(sum(web_sales.ws_sales_price)) PARTITION BY [web_sales.ws_item_sk] ORDER BY [date_dim.d_date ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "sum(sum(web_sales.ws_sales_price)) PARTITION BY [web_sales.ws_item_sk] ORDER BY [date_dim.d_date ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Decimal128(25, 2) }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[ws_item_sk@0 ASC NULLS LAST, d_date@1 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([ws_item_sk@0], 3), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[ws_item_sk@0 as ws_item_sk, d_date@1 as d_date], aggr=[sum(web_sales.ws_sales_price)] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[ss_item_sk@0 as item_sk, d_date@1 as d_date, sum(sum(store_sales.ss_sales_price)) PARTITION BY [store_sales.ss_item_sk] ORDER BY [date_dim.d_date ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as cume_sales] │ BoundedWindowAggExec: wdw=[sum(sum(store_sales.ss_sales_price)) PARTITION BY [store_sales.ss_item_sk] ORDER BY [date_dim.d_date ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "sum(sum(store_sales.ss_sales_price)) PARTITION BY [store_sales.ss_item_sk] ORDER BY [date_dim.d_date ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Decimal128(25, 2) }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[ss_item_sk@0 ASC NULLS LAST, d_date@1 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk, d_date@1 as d_date], aggr=[sum(store_sales.ss_sales_price)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([ws_item_sk@0], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[ws_item_sk@0 as ws_item_sk, d_date@1 as d_date], aggr=[sum(web_sales.ws_sales_price)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ws_item_sk@0, d_date@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[ws_item_sk@0 as ws_item_sk, d_date@2 as d_date], aggr=[sum(web_sales.ws_sales_price)] - │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_sales_price@2 as ws_sales_price, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_item_sk@4, ws_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ FilterExec: ws_item_sk@1 IS NOT NULL - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_sales_price], file_type=parquet, predicate=ws_item_sk@3 IS NOT NULL AND DynamicFilter [ empty ], pruning_predicate=ws_item_sk_null_count@1 != row_count@0, required_guarantees=[] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ws_item_sk@0, d_date@1], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[ws_item_sk@0 as ws_item_sk, d_date@2 as d_date], aggr=[sum(web_sales.ws_sales_price)] + │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_sales_price@2 as ws_sales_price, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_item_sk@4, ws_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ FilterExec: ws_item_sk@1 IS NOT NULL + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_sales_price], file_type=parquet, predicate=ws_item_sk@3 IS NOT NULL AND DynamicFilter [ empty ], pruning_predicate=ws_item_sk_null_count@1 != row_count@0, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk, d_date@1 as d_date], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_item_sk@0, d_date@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[ss_item_sk@0 as ss_item_sk, d_date@2 as d_date], aggr=[sum(store_sales.ss_sales_price)] - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_sales_price@2 as ss_sales_price, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ FilterExec: ss_item_sk@1 IS NOT NULL - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_sales_price], file_type=parquet, predicate=ss_item_sk@2 IS NOT NULL AND DynamicFilter [ empty ], pruning_predicate=ss_item_sk_null_count@1 != row_count@0, required_guarantees=[] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@0, d_date@1], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[ss_item_sk@0 as ss_item_sk, d_date@2 as d_date], aggr=[sum(store_sales.ss_sales_price)] + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_sales_price@2 as ss_sales_price, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4, ss_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ FilterExec: ss_item_sk@1 IS NOT NULL + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_sales_price], file_type=parquet, predicate=ss_item_sk@2 IS NOT NULL AND DynamicFilter [ empty ], pruning_predicate=ss_item_sk_null_count@1 != row_count@0, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] - └────────────────────────────────────────────────── "#); Ok(()) } @@ -5431,40 +5149,39 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [d_year@0 ASC NULLS LAST, ext_price@3 DESC, brand_id@1 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[ext_price@3 DESC, brand_id@1 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[d_year@0 as d_year, i_brand_id@2 as brand_id, i_brand@1 as brand, sum(store_sales.ss_ext_sales_price)@3 as ext_price] │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand@1 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_year@0, i_brand@1, i_brand_id@2], 6), input_partitions=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([d_year@0, i_brand@1, i_brand_id@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand@3 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@1, i_item_sk@0)], projection=[d_year@0, ss_ext_sales_price@2, i_brand_id@4, i_brand@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ FilterExec: i_manager_id@3 = 1, projection=[i_item_sk@0, i_brand_id@1, i_brand@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_brand, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 1 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 1 AND 1 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (1)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(dt.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ext_sales_price@5] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(dt.d_date_sk AS Float64)] │ FilterExec: d_moy@2 = 11 AND d_year@1 = 2000, projection=[d_date_sk@0, d_year@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 2000, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 2000, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (2000)] └────────────────────────────────────────────────── "); Ok(()) @@ -5480,50 +5197,44 @@ mod tests { │ FilterExec: CASE WHEN avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manufact_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 > Some(0),19,6 THEN abs(sum(store_sales.ss_sales_price)@1 - avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manufact_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2) / avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manufact_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 END > Some(1000000000),30,10 │ WindowAggExec: wdw=[avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manufact_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manufact_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(19, 6), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] │ SortExec: expr=[i_manufact_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_manufact_id@0], 3), input_partitions=3 + │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(store_sales.ss_sales_price)@2 as sum(store_sales.ss_sales_price)] + │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact_id@0 as i_manufact_id, d_qoy@1 as d_qoy], aggr=[sum(store_sales.ss_sales_price)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_manufact_id@0], 3), input_partitions=3 - │ ProjectionExec: expr=[i_manufact_id@0 as i_manufact_id, sum(store_sales.ss_sales_price)@2 as sum(store_sales.ss_sales_price)] - │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact_id@0 as i_manufact_id, d_qoy@1 as d_qoy], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_manufact_id@0, d_qoy@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_manufact_id@0 as i_manufact_id, d_qoy@2 as d_qoy], aggr=[sum(store_sales.ss_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[i_manufact_id@2, ss_sales_price@4, d_qoy@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_manufact_id@1 as i_manufact_id, ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_qoy@0 as d_qoy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@1)], projection=[d_qoy@1, i_manufact_id@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_manufact_id@1, ss_sold_date_sk@2, ss_store_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_manufact_id@0, d_qoy@1], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_manufact_id@0 as i_manufact_id, d_qoy@2 as d_qoy], aggr=[sum(store_sales.ss_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[i_manufact_id@2, ss_sales_price@4, d_qoy@5] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[i_manufact_id@1 as i_manufact_id, ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_qoy@0 as d_qoy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@1)], projection=[d_qoy@1, i_manufact_id@3, ss_store_sk@5, ss_sales_price@6] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_manufact_id@1, ss_sold_date_sk@2, ss_store_sk@4, ss_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_qoy@1 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@1 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), projection=[d_date_sk@0, d_qoy@2] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq, d_qoy], file_type=parquet, predicate=d_month_seq@3 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), pruning_predicate=d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1200 AND 1200 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1201 AND 1201 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1202 AND 1202 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1203 AND 1203 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1204 AND 1204 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1205 AND 1205 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1206 AND 1206 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1207 AND 1207 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1208 AND 1208 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1209 AND 1209 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1210 AND 1210 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1211 AND 1211 <= d_month_seq_max@1, required_guarantees=[d_month_seq in (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: (i_category@3 = Books OR i_category@3 = Children OR i_category@3 = Electronics) AND i_class@2 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@1 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@3 = Women OR i_category@3 = Music OR i_category@3 = Men) AND i_class@2 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@1 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), projection=[i_item_sk@0, i_manufact_id@4] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_manufact_id], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Children OR i_category@12 = Electronics) AND i_class@10 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@8 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@12 = Women OR i_category@12 = Music OR i_category@12 = Men) AND i_class@10 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@8 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Children AND Children <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= personal AND personal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= portable AND portable <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= reference AND reference <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= self-help AND self-help <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #14 AND scholaramalgamalg #14 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #7 AND scholaramalgamalg #7 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiunivamalg #9 AND exportiunivamalg #9 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #9 AND scholaramalgamalg #9 <= i_brand_max@8) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= accessories AND accessories <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= classical AND classical <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= fragrances AND fragrances <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= pants AND pants <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= amalgimporto #1 AND amalgimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= edu packscholar #1 AND edu packscholar #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiimporto #1 AND exportiimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= importoamalg #1 AND importoamalg #1 <= i_brand_max@8), required_guarantees=[i_brand in (amalgimporto #1, edu packscholar #1, exportiimporto #1, exportiunivamalg #9, importoamalg #1, scholaramalgamalg #14, scholaramalgamalg #7, scholaramalgamalg #9), i_class in (accessories, classical, fragrances, pants, personal, portable, reference, self-help)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_qoy@1 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@1 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), projection=[d_date_sk@0, d_qoy@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq, d_qoy], file_type=parquet, predicate=d_month_seq@3 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), pruning_predicate=d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1200 AND 1200 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1201 AND 1201 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1202 AND 1202 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1203 AND 1203 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1204 AND 1204 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1205 AND 1205 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1206 AND 1206 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1207 AND 1207 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1208 AND 1208 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1209 AND 1209 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1210 AND 1210 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1211 AND 1211 <= d_month_seq_max@1, required_guarantees=[d_month_seq in (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: (i_category@3 = Books OR i_category@3 = Children OR i_category@3 = Electronics) AND i_class@2 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@1 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@3 = Women OR i_category@3 = Music OR i_category@3 = Men) AND i_class@2 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@1 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), projection=[i_item_sk@0, i_manufact_id@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_manufact_id], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Children OR i_category@12 = Electronics) AND i_class@10 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@8 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@12 = Women OR i_category@12 = Music OR i_category@12 = Men) AND i_class@10 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@8 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Children AND Children <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= personal AND personal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= portable AND portable <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= reference AND reference <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= self-help AND self-help <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #14 AND scholaramalgamalg #14 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #7 AND scholaramalgamalg #7 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiunivamalg #9 AND exportiunivamalg #9 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #9 AND scholaramalgamalg #9 <= i_brand_max@8) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= accessories AND accessories <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= classical AND classical <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= fragrances AND fragrances <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= pants AND pants <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= amalgimporto #1 AND amalgimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= edu packscholar #1 AND edu packscholar #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiimporto #1 AND exportiimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= importoamalg #1 AND importoamalg #1 <= i_brand_max@8), required_guarantees=[i_brand in (amalgimporto #1, edu packscholar #1, exportiimporto #1, exportiunivamalg #9, importoamalg #1, scholaramalgamalg #14, scholaramalgamalg #7, scholaramalgamalg #9), i_class in (accessories, classical, fragrances, pants, personal, portable, reference, self-help)] - └────────────────────────────────────────────────── "#); Ok(()) } @@ -5533,119 +5244,126 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [segment@0 ASC, num_customers@1 ASC, segment_base@2 ASC NULLS LAST], fetch=100 - │ SortExec: TopK(fetch=100), expr=[segment@0 ASC, num_customers@1 ASC, segment_base@2 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[segment@0 as segment, count(Int64(1))@1 as num_customers, CAST(segment@0 AS Int64) * 50 as segment_base] - │ AggregateExec: mode=FinalPartitioned, gby=[segment@0 as segment], aggr=[count(Int64(1))] - │ RepartitionExec: partitioning=Hash([segment@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[segment@0 as segment], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[CAST(round(sum(store_sales.ss_ext_sales_price)@1 / Some(50),20,0) AS Int32) as segment] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_ext_sales_price)] - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price] - │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_2@1 <= date_dim.d_month_seq + Int64(3)@0, projection=[c_customer_sk@0, ss_ext_sales_price@1, d_month_seq@2, date_dim.d_month_seq + Int64(3)@4] - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price, d_month_seq@2 as d_month_seq, CAST(d_month_seq@2 AS Int64) as join_proj_push_down_2] - │ CoalescePartitionsExec - │ ProjectionExec: expr=[c_customer_sk@1 as c_customer_sk, ss_ext_sales_price@2 as ss_ext_sales_price, d_month_seq@3 as d_month_seq] - │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 >= date_dim.d_month_seq + Int64(1)@0, projection=[date_dim.d_month_seq + Int64(1)@0, c_customer_sk@1, ss_ext_sales_price@2, d_month_seq@3] - │ CoalescePartitionsExec - │ AggregateExec: mode=FinalPartitioned, gby=[date_dim.d_month_seq + Int64(1)@0 as date_dim.d_month_seq + Int64(1)], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price, d_month_seq@2 as d_month_seq, CAST(d_month_seq@2 AS Int64) as join_proj_push_down_1] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[c_customer_sk@0, ss_ext_sales_price@2, d_month_seq@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_county@0, ca_county@3), (CAST(store.s_state AS Utf8View)@2, ca_state@4)], projection=[c_customer_sk@3, ss_sold_date_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@1, ca_address_sk@0)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_ext_sales_price@3, ca_county@5, ca_state@6] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county, ca_state], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_month_seq@1 as d_month_seq, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet - │ AggregateExec: mode=FinalPartitioned, gby=[date_dim.d_month_seq + Int64(3)@0 as date_dim.d_month_seq + Int64(3)], aggr=[] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([date_dim.d_month_seq + Int64(1)@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[date_dim.d_month_seq + Int64(1)@0 as date_dim.d_month_seq + Int64(1)], aggr=[] - │ ProjectionExec: expr=[CAST(d_month_seq@0 AS Int64) + 1 as date_dim.d_month_seq + Int64(1)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 12, projection=[d_month_seq@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_month_seq, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_county, s_state, CAST(s_state@24 AS Utf8View) as CAST(store.s_state AS Utf8View)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(my_customers.c_customer_sk AS Float64)@2, ss_customer_sk@1)], projection=[c_customer_sk@0, c_current_addr_sk@1, ss_sold_date_sk@3, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(my_customers.c_customer_sk AS Float64)] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk], aggr=[] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[segment@0 ASC, num_customers@1 ASC, segment_base@2 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[segment@0 as segment, count(Int64(1))@1 as num_customers, CAST(segment@0 AS Int64) * 50 as segment_base] + │ AggregateExec: mode=FinalPartitioned, gby=[segment@0 as segment], aggr=[count(Int64(1))] + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([segment@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[segment@0 as segment], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[CAST(round(sum(store_sales.ss_ext_sales_price)@1 / Some(50),20,0) AS Int32) as segment] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_ext_sales_price)] + │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_ext_sales_price)] + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price] + │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_2@1 <= date_dim.d_month_seq + Int64(3)@0, projection=[c_customer_sk@0, ss_ext_sales_price@1, d_month_seq@2, date_dim.d_month_seq + Int64(3)@4] + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price, d_month_seq@2 as d_month_seq, CAST(d_month_seq@2 AS Int64) as join_proj_push_down_2] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@1 as c_customer_sk, ss_ext_sales_price@2 as ss_ext_sales_price, d_month_seq@3 as d_month_seq] + │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 >= date_dim.d_month_seq + Int64(1)@0, projection=[date_dim.d_month_seq + Int64(1)@0, c_customer_sk@1, ss_ext_sales_price@2, d_month_seq@3] + │ CoalescePartitionsExec + │ AggregateExec: mode=FinalPartitioned, gby=[date_dim.d_month_seq + Int64(1)@0 as date_dim.d_month_seq + Int64(1)], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price, d_month_seq@2 as d_month_seq, CAST(d_month_seq@2 AS Int64) as join_proj_push_down_1] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[c_customer_sk@0, ss_ext_sales_price@2, d_month_seq@4] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_month_seq@1 as d_month_seq, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet + │ AggregateExec: mode=FinalPartitioned, gby=[date_dim.d_month_seq + Int64(3)@0 as date_dim.d_month_seq + Int64(3)], aggr=[] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_sk@0, c_current_addr_sk@1], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk], aggr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[c_customer_sk@1, c_current_addr_sk@2] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 - │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ RepartitionExec: partitioning=Hash([date_dim.d_month_seq + Int64(1)@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[date_dim.d_month_seq + Int64(1)@0 as date_dim.d_month_seq + Int64(1)], aggr=[] + │ ProjectionExec: expr=[CAST(d_month_seq@0 AS Int64) + 1 as date_dim.d_month_seq + Int64(1)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 12, projection=[d_month_seq@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_month_seq, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sold_date_sk@0)], projection=[customer_sk@3] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_county@0, ca_county@3), (CAST(store.s_state AS Utf8View)@2, ca_state@4)], projection=[c_customer_sk@3, ss_sold_date_sk@4, ss_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@1, ca_address_sk@0)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_ext_sales_price@3, ca_county@5, ca_state@6] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_county, ca_state], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_county, s_state, CAST(s_state@24 AS Utf8View) as CAST(store.s_state AS Utf8View)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(my_customers.c_customer_sk AS Float64)@2, ss_customer_sk@1)], projection=[c_customer_sk@0, c_current_addr_sk@1, ss_sold_date_sk@3, ss_ext_sales_price@5] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@2)], projection=[sold_date_sk@1, customer_sk@2] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 - │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk@0 as sold_date_sk, cs_bill_customer_sk@3 as customer_sk, cs_item_sk@15 as item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk@0 as sold_date_sk, ws_bill_customer_sk@4 as customer_sk, ws_item_sk@3 as item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 = 12 AND d_year@1 = 1998, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 12 AND d_year@6 = 1998, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 12 AND 12 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(my_customers.c_customer_sk AS Float64)] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk], aggr=[] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ FilterExec: i_category@2 = Women AND i_class@1 = maternity, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=i_category@12 = Women AND i_class@10 = maternity, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 AND i_class_null_count@6 != row_count@3 AND i_class_min@4 <= maternity AND maternity <= i_class_max@5, required_guarantees=[i_category in (Women), i_class in (maternity)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([date_dim.d_month_seq + Int64(3)@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[date_dim.d_month_seq + Int64(3)@0 as date_dim.d_month_seq + Int64(3)], aggr=[] - │ ProjectionExec: expr=[CAST(d_month_seq@0 AS Int64) + 3 as date_dim.d_month_seq + Int64(3)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 12, projection=[d_month_seq@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_month_seq, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] - └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([c_customer_sk@0, c_current_addr_sk@1], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk], aggr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[c_customer_sk@1, c_current_addr_sk@2] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sold_date_sk@0)], projection=[customer_sk@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@2)], projection=[sold_date_sk@1, customer_sk@2] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk@0 as sold_date_sk, cs_bill_customer_sk@3 as customer_sk, cs_item_sk@15 as item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk@0 as sold_date_sk, ws_bill_customer_sk@4 as customer_sk, ws_item_sk@3 as item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 = 12 AND d_year@1 = 1998, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 12 AND d_year@6 = 1998, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 12 AND 12 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@2 = Women AND i_class@1 = maternity, projection=[i_item_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=i_category@12 = Women AND i_class@10 = maternity, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 AND i_class_null_count@6 != row_count@3 AND i_class_min@4 <= maternity AND maternity <= i_class_max@5, required_guarantees=[i_category in (Women), i_class in (maternity)] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ RepartitionExec: partitioning=Hash([date_dim.d_month_seq + Int64(3)@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[date_dim.d_month_seq + Int64(3)@0 as date_dim.d_month_seq + Int64(3)], aggr=[] + │ ProjectionExec: expr=[CAST(d_month_seq@0 AS Int64) + 3 as date_dim.d_month_seq + Int64(3)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 12, projection=[d_month_seq@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_month_seq, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -5655,40 +5373,39 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ext_price@2 DESC, brand_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[ext_price@2 DESC, brand_id@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, sum(store_sales.ss_ext_sales_price)@2 as ext_price] │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1], 6), input_partitions=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_brand@2 as i_brand, i_brand_id@1 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_brand_id@3, i_brand@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ FilterExec: i_manager_id@3 = 28, projection=[i_item_sk@0, i_brand_id@1, i_brand@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_brand, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 28 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 28 AND 28 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (28)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_ext_sales_price@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_moy@2 = 11 AND d_year@1 = 1999, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] └────────────────────────────────────────────────── "); Ok(()) @@ -5699,146 +5416,155 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [total_sales@1 ASC, i_item_id@0 ASC], fetch=100 - │ SortExec: TopK(fetch=100), expr=[total_sales@1 ASC, i_item_id@0 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(tmp1.total_sales)@1 as total_sales] - │ AggregateExec: mode=SinglePartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(tmp1.total_sales)] - │ InterleaveExec - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(store_sales.ss_ext_sales_price)@1 as total_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_color@1 = slate OR i_color@1 = blanched OR i_color@1 = burnished, projection=[i_item_id@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_color], file_type=parquet, predicate=i_color@17 = slate OR i_color@17 = blanched OR i_color@17 = burnished, pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= slate AND slate <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= blanched AND blanched <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burnished AND burnished <= i_color_max@1, required_guarantees=[i_color in (blanched, burnished, slate)] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(catalog_sales.cs_ext_sales_price)@1 as total_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_color@1 = slate OR i_color@1 = blanched OR i_color@1 = burnished, projection=[i_item_id@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_color], file_type=parquet, predicate=i_color@17 = slate OR i_color@17 = blanched OR i_color@17 = burnished, pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= slate AND slate <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= blanched AND blanched <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burnished AND burnished <= i_color_max@1, required_guarantees=[i_color in (blanched, burnished, slate)] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(web_sales.ws_ext_sales_price)@1 as total_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_color@1 = slate OR i_color@1 = blanched OR i_color@1 = burnished, projection=[i_item_id@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_color], file_type=parquet, predicate=i_color@17 = slate OR i_color@17 = blanched OR i_color@17 = burnished, pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= slate AND slate <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= blanched AND blanched <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burnished AND burnished <= i_color_max@1, required_guarantees=[i_color in (blanched, burnished, slate)] + │ [Stage 16] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[total_sales@1 ASC, i_item_id@0 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(tmp1.total_sales)@1 as total_sales] + │ AggregateExec: mode=SinglePartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(tmp1.total_sales)] + │ InterleaveExec + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(store_sales.ss_ext_sales_price)@1 as total_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(catalog_sales.cs_ext_sales_price)@1 as total_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(web_sales.ws_ext_sales_price)@1 as total_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_color@1 = slate OR i_color@1 = blanched OR i_color@1 = burnished, projection=[i_item_id@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_color], file_type=parquet, predicate=i_color@17 = slate OR i_color@17 = blanched OR i_color@17 = burnished, pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= slate AND slate <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= blanched AND blanched <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burnished AND burnished <= i_color_max@1, required_guarantees=[i_color in (blanched, burnished, slate)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_addr_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 2, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_addr_sk@4, ss_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_color@1 = slate OR i_color@1 = blanched OR i_color@1 = burnished, projection=[i_item_id@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_color], file_type=parquet, predicate=i_color@17 = slate OR i_color@17 = blanched OR i_color@17 = burnished, pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= slate AND slate <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= blanched AND blanched <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burnished AND burnished <= i_color_max@1, required_guarantees=[i_color in (blanched, burnished, slate)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 2, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 15 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] + │ CoalescePartitionsExec + │ [Stage 14] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_color@1 = slate OR i_color@1 = blanched OR i_color@1 = burnished, projection=[i_item_id@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_color], file_type=parquet, predicate=i_color@17 = slate OR i_color@17 = blanched OR i_color@17 = burnished, pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= slate AND slate <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= blanched AND blanched <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burnished AND burnished <= i_color_max@1, required_guarantees=[i_color in (blanched, burnished, slate)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_bill_addr_sk@4, ws_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 2, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] - └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_bill_addr_sk@4, ws_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -5864,136 +5590,121 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2], 3), input_partitions=3 │ BoundedWindowAggExec: wdw=[avg(sum(catalog_sales.cs_sales_price)) PARTITION BY [item.i_category, item.i_brand, call_center.cc_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "avg(sum(catalog_sales.cs_sales_price)) PARTITION BY [item.i_category, item.i_brand, call_center.cc_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Decimal128(19, 6) }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, cc_name@2 ASC NULLS LAST, d_year@3 ASC NULLS LAST, d_moy@4 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2, d_year@3], 3), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, sum(catalog_sales.cs_sales_price)@5 as sum_sales, rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 AS Decimal128(20, 0)) + Some(1),20,0 as v1_lag.rn + Decimal128(Some(1),20,0)] │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, cc_name@2 ASC NULLS LAST, d_year@3 ASC NULLS LAST, d_moy@4 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2], 3), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, sum(catalog_sales.cs_sales_price)@5 as sum_sales, rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 AS Decimal128(20, 0)) - Some(1),20,0 as v1_lead.rn - Decimal128(Some(1),20,0)] │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, cc_name@2 ASC NULLS LAST, d_year@3 ASC NULLS LAST, d_moy@4 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2], 3), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2, d_year@3], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2, d_year@3, d_moy@4], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, cc_name@5 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] - │ ProjectionExec: expr=[i_brand@1 as i_brand, i_category@2 as i_category, cs_sales_price@3 as cs_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, cc_name@0 as cc_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@2, cs_call_center_sk@2)], projection=[cc_name@1, i_brand@3, i_category@4, cs_sales_price@6, d_year@7, d_moy@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, cs_call_center_sk@4 as cs_call_center_sk, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, cs_call_center_sk@7, cs_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2, d_year@3, d_moy@4], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, cc_name@5 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] + │ ProjectionExec: expr=[i_brand@1 as i_brand, i_category@2 as i_category, cs_sales_price@3 as cs_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, cc_name@0 as cc_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@2, cs_call_center_sk@2)], projection=[cc_name@1, i_brand@3, i_category@4, cs_sales_price@6, d_year@7, d_moy@8] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, cs_call_center_sk@4 as cs_call_center_sk, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, cs_call_center_sk@7, cs_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2, d_year@3, d_moy@4], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, cc_name@5 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] - │ ProjectionExec: expr=[i_brand@1 as i_brand, i_category@2 as i_category, cs_sales_price@3 as cs_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, cc_name@0 as cc_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@2, cs_call_center_sk@2)], projection=[cc_name@1, i_brand@3, i_category@4, cs_sales_price@6, d_year@7, d_moy@8] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, cs_call_center_sk@4 as cs_call_center_sk, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, cs_call_center_sk@7, cs_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2, d_year@3, d_moy@4], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, cc_name@5 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] - │ ProjectionExec: expr=[i_brand@1 as i_brand, i_category@2 as i_category, cs_sales_price@3 as cs_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, cc_name@0 as cc_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@2, cs_call_center_sk@2)], projection=[cc_name@1, i_brand@3, i_category@4, cs_sales_price@6, d_year@7, d_moy@8] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, cs_call_center_sk@4 as cs_call_center_sk, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, cs_call_center_sk@7, cs_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2, d_year@3, d_moy@4], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, cc_name@5 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] + │ ProjectionExec: expr=[i_brand@1 as i_brand, i_category@2 as i_category, cs_sales_price@3 as cs_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, cc_name@0 as cc_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@2, cs_call_center_sk@2)], projection=[cc_name@1, i_brand@3, i_category@4, cs_sales_price@6, d_year@7, d_moy@8] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, cs_call_center_sk@4 as cs_call_center_sk, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, cs_call_center_sk@7, cs_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2, d_year@3, d_moy@4], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, cc_name@5 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] + │ ProjectionExec: expr=[i_brand@1 as i_brand, i_category@2 as i_category, cs_sales_price@3 as cs_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, cc_name@0 as cc_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@2, cs_call_center_sk@2)], projection=[cc_name@1, i_brand@3, i_category@4, cs_sales_price@6, d_year@7, d_moy@8] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, cs_call_center_sk@4 as cs_call_center_sk, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, cs_call_center_sk@7, cs_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 1 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet - └────────────────────────────────────────────────── "#); Ok(()) } @@ -6003,143 +5714,155 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [item_id@0 ASC, ss_item_rev@1 ASC], fetch=100 - │ SortExec: TopK(fetch=100), expr=[item_id@0 ASC, ss_item_rev@1 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[item_id@1 as item_id, ss_item_rev@2 as ss_item_rev, ss_item_rev@2 / __common_expr_7@0 * Some(100),20,0 as ss_dev, cs_item_rev@3 as cs_item_rev, cs_item_rev@3 / __common_expr_7@0 * Some(100),20,0 as cs_dev, ws_item_rev@4 as ws_item_rev, ws_item_rev@4 / __common_expr_7@0 * Some(100),20,0 as ws_dev, __common_expr_7@0 as average] - │ ProjectionExec: expr=[(ss_item_rev@2 + cs_item_rev@3 + ws_item_rev@0) / Some(3),20,0 as __common_expr_7, item_id@1 as item_id, ss_item_rev@2 as ss_item_rev, cs_item_rev@3 as cs_item_rev, ws_item_rev@0 as ws_item_rev] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], filter=CAST(ss_item_rev@0 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(ss_item_rev@0 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)), projection=[ws_item_rev@1, item_id@2, ss_item_rev@3, cs_item_rev@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(web_sales.ws_ext_sales_price)@1 as ws_item_rev] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[ws_ext_sales_price@0, i_item_id@1] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ FilterExec: d_date@0 = 2000-01-03, projection=[d_week_seq@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[item_id@1 as item_id, ss_item_rev@2 as ss_item_rev, cs_item_rev@0 as cs_item_rev] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], filter=CAST(ss_item_rev@0 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)) AND CAST(ss_item_rev@0 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)), projection=[cs_item_rev@1, item_id@2, ss_item_rev@3] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(catalog_sales.cs_ext_sales_price)@1 as cs_item_rev] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[cs_ext_sales_price@0, i_item_id@1] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ FilterExec: d_date@0 = 2000-01-03, projection=[d_week_seq@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(store_sales.ss_ext_sales_price)@1 as ss_item_rev] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[ss_ext_sales_price@0, i_item_id@1] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ FilterExec: d_date@0 = 2000-01-03, projection=[d_week_seq@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 18] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ ProjectionExec: expr=[ws_ext_sales_price@1 as ws_ext_sales_price, i_item_id@2 as i_item_id, d_date@0 as d_date] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_ext_sales_price@4, i_item_id@5] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 9), input_partitions=2 - │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ext_sales_price@2 as ws_ext_sales_price, i_item_id@0 as i_item_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_id@1, ws_sold_date_sk@2, ws_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[item_id@0 ASC, ss_item_rev@1 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[item_id@1 as item_id, ss_item_rev@2 as ss_item_rev, ss_item_rev@2 / __common_expr_7@0 * Some(100),20,0 as ss_dev, cs_item_rev@3 as cs_item_rev, cs_item_rev@3 / __common_expr_7@0 * Some(100),20,0 as cs_dev, ws_item_rev@4 as ws_item_rev, ws_item_rev@4 / __common_expr_7@0 * Some(100),20,0 as ws_dev, __common_expr_7@0 as average] + │ ProjectionExec: expr=[(ss_item_rev@2 + cs_item_rev@3 + ws_item_rev@0) / Some(3),20,0 as __common_expr_7, item_id@1 as item_id, ss_item_rev@2 as ss_item_rev, cs_item_rev@3 as cs_item_rev, ws_item_rev@0 as ws_item_rev] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], filter=CAST(ss_item_rev@0 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(ss_item_rev@0 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)), projection=[ws_item_rev@1, item_id@2, ss_item_rev@3, cs_item_rev@4] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[item_id@1 as item_id, ss_item_rev@2 as ss_item_rev, cs_item_rev@0 as cs_item_rev] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], filter=CAST(ss_item_rev@0 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)) AND CAST(ss_item_rev@0 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)), projection=[cs_item_rev@1, item_id@2, ss_item_rev@3] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(store_sales.ss_ext_sales_price)@1 as ss_item_rev] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ [Stage 17] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(web_sales.ws_ext_sales_price)@1 as ws_item_rev] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[ws_ext_sales_price@0, i_item_id@1] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ FilterExec: d_date@0 = 2000-01-03, projection=[d_week_seq@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ ProjectionExec: expr=[cs_ext_sales_price@1 as cs_ext_sales_price, i_item_id@2 as i_item_id, d_date@0 as d_date] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_date@1, cs_ext_sales_price@4, i_item_id@5] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 9), input_partitions=2 - │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ext_sales_price@2 as cs_ext_sales_price, i_item_id@0 as i_item_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_id@1, cs_sold_date_sk@2, cs_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ ProjectionExec: expr=[ws_ext_sales_price@1 as ws_ext_sales_price, i_item_id@2 as i_item_id, d_date@0 as d_date] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_ext_sales_price@4, i_item_id@5] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 12), input_partitions=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 12), input_partitions=2 + │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ext_sales_price@2 as ws_ext_sales_price, i_item_id@0 as i_item_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_id@1, ws_sold_date_sk@2, ws_ext_sales_price@4] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(catalog_sales.cs_ext_sales_price)@1 as cs_item_rev] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[cs_ext_sales_price@0, i_item_id@1] + │ CoalescePartitionsExec + │ [Stage 10] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ FilterExec: d_date@0 = 2000-01-03, projection=[d_week_seq@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ ProjectionExec: expr=[ss_ext_sales_price@1 as ss_ext_sales_price, i_item_id@2 as i_item_id, d_date@0 as d_date] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_ext_sales_price@4, i_item_id@5] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 9), input_partitions=2 - │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_ext_sales_price@2 as ss_ext_sales_price, i_item_id@0 as i_item_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_item_id@1, ss_sold_date_sk@2, ss_ext_sales_price@4] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ ProjectionExec: expr=[cs_ext_sales_price@1 as cs_ext_sales_price, i_item_id@2 as i_item_id, d_date@0 as d_date] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_date@1, cs_ext_sales_price@4, i_item_id@5] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 12), input_partitions=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 12), input_partitions=2 + │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ext_sales_price@2 as cs_ext_sales_price, i_item_id@0 as i_item_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_id@1, cs_sold_date_sk@2, cs_ext_sales_price@4] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 17 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[ss_ext_sales_price@0, i_item_id@1] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 16] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ FilterExec: d_date@0 = 2000-01-03, projection=[d_week_seq@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ ProjectionExec: expr=[ss_ext_sales_price@1 as ss_ext_sales_price, i_item_id@2 as i_item_id, d_date@0 as d_date] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_ext_sales_price@4, i_item_id@5] + │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 12), input_partitions=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 15 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 12), input_partitions=2 + │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_ext_sales_price@2 as ss_ext_sales_price, i_item_id@0 as i_item_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_item_id@1, ss_sold_date_sk@2, ss_ext_sales_price@4] + │ CoalescePartitionsExec + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + └────────────────────────────────────────────────── "); Ok(()) } @@ -6149,97 +5872,93 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_store_name1@0 ASC, s_store_id1@1 ASC, d_week_seq1@2 ASC], fetch=100 - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] │ SortExec: TopK(fetch=100), expr=[s_store_name1@0 ASC, s_store_id1@1 ASC, d_week_seq1@2 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[s_store_name1@0 as s_store_name1, s_store_id1@2 as s_store_id1, d_week_seq1@1 as d_week_seq1, sun_sales1@3 / sun_sales2@10 as sun_sales_ratio, mon_sales1@4 / mon_sales2@11 as mon_sales_ratio, tue_sales1@5 / tue_sales2@12 as tue_sales_ratio, wed_sales1@6 / wed_sales2@13 as wed_sales_ratio, thu_sales1@7 / thu_sales2@14 as thu_sales_ratio, fri_sales1@8 / fri_sales2@15 as fri_sales_ratio, sat_sales1@9 / sat_sales2@16 as sat_sales_ratio] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_store_id1@2, s_store_id2@1), (CAST(y.d_week_seq1 AS Int64)@10, x.d_week_seq2 - Int64(52)@9)], projection=[s_store_name1@0, d_week_seq1@1, s_store_id1@2, sun_sales1@3, mon_sales1@4, tue_sales1@5, wed_sales1@6, thu_sales1@7, fri_sales1@8, sat_sales1@9, sun_sales2@13, mon_sales2@14, tue_sales2@15, wed_sales2@16, thu_sales2@17, fri_sales2@18, sat_sales2@19] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq2, s_store_id@8 as s_store_id2, sun_sales@1 as sun_sales2, mon_sales@2 as mon_sales2, tue_sales@3 as tue_sales2, wed_sales@4 as wed_sales2, thu_sales@5 as thu_sales2, fri_sales@6 as fri_sales2, sat_sales@7 as sat_sales2, CAST(d_week_seq@0 AS Int64) - 52 as x.d_week_seq2 - Int64(52)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@0)], projection=[d_week_seq@0, sun_sales@1, mon_sales@2, tue_sales@3, wed_sales@4, thu_sales@5, fri_sales@6, sat_sales@7, s_store_id@8] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ FilterExec: d_month_seq@0 >= 1224 AND d_month_seq@0 <= 1235, projection=[d_week_seq@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_month_seq, d_week_seq], file_type=parquet, predicate=d_month_seq@3 >= 1224 AND d_month_seq@3 <= 1235 AND DynamicFilter [ empty ], pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1224 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1235, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_name@9 as s_store_name1, d_week_seq@0 as d_week_seq1, s_store_id@8 as s_store_id1, sun_sales@1 as sun_sales1, mon_sales@2 as mon_sales1, tue_sales@3 as tue_sales1, wed_sales@4 as wed_sales1, thu_sales@5 as thu_sales1, fri_sales@6 as fri_sales1, sat_sales@7 as sat_sales1, CAST(d_week_seq@0 AS Int64) as CAST(y.d_week_seq1 AS Int64)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@0)], projection=[d_week_seq@0, sun_sales@1, mon_sales@2, tue_sales@3, wed_sales@4, thu_sales@5, fri_sales@6, sat_sales@7, s_store_id@8, s_store_name@9] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ FilterExec: d_month_seq@0 >= 1212 AND d_month_seq@0 <= 1223, projection=[d_week_seq@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_month_seq, d_week_seq], file_type=parquet, predicate=d_month_seq@3 >= 1212 AND d_month_seq@3 <= 1223 AND DynamicFilter [ empty ], pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1212 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1223, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_week_seq@2 as d_week_seq, sun_sales@3 as sun_sales, mon_sales@4 as mon_sales, tue_sales@5 as tue_sales, wed_sales@6 as wed_sales, thu_sales@7 as thu_sales, fri_sales@8 as fri_sales, sat_sales@9 as sat_sales, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, d_week_seq@4, sun_sales@6, mon_sales@7, tue_sales@8, wed_sales@9, thu_sales@10, fri_sales@11, sat_sales@12] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_week_seq@0, ss_store_sk@1], 6), input_partitions=2 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([d_week_seq@0, ss_store_sk@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[d_week_seq@2 as d_week_seq, ss_store_sk@0 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] │ ProjectionExec: expr=[ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_week_seq@0 as d_week_seq, d_day_name@1 as d_day_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, ss_store_sk@5, ss_sales_price@6] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_week_seq@1 as d_week_seq, sun_sales@2 as sun_sales, mon_sales@3 as mon_sales, tue_sales@4 as tue_sales, wed_sales@5 as wed_sales, thu_sales@6 as thu_sales, fri_sales@7 as fri_sales, sat_sales@8 as sat_sales, s_store_id@0 as s_store_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, d_week_seq@3, sun_sales@5, mon_sales@6, tue_sales@7, wed_sales@8, thu_sales@9, fri_sales@10, sat_sales@11] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_week_seq@0, ss_store_sk@1], 6), input_partitions=2 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([d_week_seq@0, ss_store_sk@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[d_week_seq@2 as d_week_seq, ss_store_sk@0 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] │ ProjectionExec: expr=[ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_week_seq@0 as d_week_seq, d_day_name@1 as d_day_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, ss_store_sk@5, ss_sales_price@6] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -6250,146 +5969,155 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST, total_sales@1 ASC NULLS LAST], fetch=100 - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, total_sales@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(tmp1.total_sales)@1 as total_sales] - │ AggregateExec: mode=SinglePartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(tmp1.total_sales)] - │ InterleaveExec - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(store_sales.ss_ext_sales_price)@1 as total_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_category@1 = Music, projection=[i_item_id@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_category], file_type=parquet, predicate=i_category@12 = Music, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1, required_guarantees=[i_category in (Music)] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(catalog_sales.cs_ext_sales_price)@1 as total_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_category@1 = Music, projection=[i_item_id@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_category], file_type=parquet, predicate=i_category@12 = Music, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1, required_guarantees=[i_category in (Music)] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(web_sales.ws_ext_sales_price)@1 as total_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 - │ FilterExec: i_category@1 = Music, projection=[i_item_id@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_category], file_type=parquet, predicate=i_category@12 = Music, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1, required_guarantees=[i_category in (Music)] + │ [Stage 16] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, total_sales@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(tmp1.total_sales)@1 as total_sales] + │ AggregateExec: mode=SinglePartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(tmp1.total_sales)] + │ InterleaveExec + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(store_sales.ss_ext_sales_price)@1 as total_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(catalog_sales.cs_ext_sales_price)@1 as total_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, sum(web_sales.ws_ext_sales_price)@1 as total_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_category@1 = Music, projection=[i_item_id@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_category], file_type=parquet, predicate=i_category@12 = Music, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1, required_guarantees=[i_category in (Music)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_addr_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 9, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 9 AND 9 <= d_moy_max@5, required_guarantees=[d_moy in (9), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_addr_sk@4, ss_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 9, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 9 AND 9 <= d_moy_max@5, required_guarantees=[d_moy in (9), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_category@1 = Music, projection=[i_item_id@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_category], file_type=parquet, predicate=i_category@12 = Music, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1, required_guarantees=[i_category in (Music)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 9, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 9 AND 9 <= d_moy_max@5, required_guarantees=[d_moy in (9), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 9, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 9 AND 9 <= d_moy_max@5, required_guarantees=[d_moy in (9), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 15 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] + │ CoalescePartitionsExec + │ [Stage 14] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ FilterExec: i_category@1 = Music, projection=[i_item_id@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_id, i_category], file_type=parquet, predicate=i_category@12 = Music, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1, required_guarantees=[i_category in (Music)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_bill_addr_sk@4, ws_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 9, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 9 AND 9 <= d_moy_max@5, required_guarantees=[d_moy in (9), d_year in (1998)] - └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_bill_addr_sk@4, ws_ext_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 9, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 9 AND 9 <= d_moy_max@5, required_guarantees=[d_moy in (9), d_year in (1998)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -6404,137 +6132,127 @@ mod tests { │ ProjectionExec: expr=[sum(store_sales.ss_ext_sales_price)@0 as promotions] │ AggregateExec: mode=Final, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] │ CoalescePartitionsExec - │ [Stage 7] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 + │ FilterExec: i_category@1 = Jewelry, projection=[i_item_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet, predicate=i_category@12 = Jewelry AND DynamicFilter [ empty ], pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1, required_guarantees=[i_category in (Jewelry)] │ ProjectionExec: expr=[sum(store_sales.ss_ext_sales_price)@0 as total] │ AggregateExec: mode=Final, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] │ CoalescePartitionsExec - │ [Stage 13] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 + │ FilterExec: i_category@1 = Jewelry, projection=[i_item_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet, predicate=i_category@12 = Jewelry AND DynamicFilter [ empty ], pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1, required_guarantees=[i_category in (Jewelry)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@2, ca_address_sk@0)], projection=[ss_item_sk@0, ss_ext_sales_price@1] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_category@1 = Jewelry, projection=[i_item_sk@0] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet, predicate=i_category@12 = Jewelry AND DynamicFilter [ empty ], pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1, required_guarantees=[i_category in (Jewelry)] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@2, ca_address_sk@0)], projection=[ss_item_sk@0, ss_ext_sales_price@1] + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_ext_sales_price@2, c_current_addr_sk@4] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_ext_sales_price@2, c_current_addr_sk@4] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] t3:[p18..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@3] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 11, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@3] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_promo_sk@3, CAST(promotion.p_promo_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@4] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 11, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_promo_sk@3, CAST(promotion.p_promo_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@4] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_dmail@1 = Y OR p_channel_email@2 = Y OR p_channel_tv@3 = Y, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_dmail, p_channel_email, p_channel_tv], file_type=parquet, predicate=p_channel_dmail@8 = Y OR p_channel_email@9 = Y OR p_channel_tv@11 = Y, pruning_predicate=p_channel_dmail_null_count@2 != row_count@3 AND p_channel_dmail_min@0 <= Y AND Y <= p_channel_dmail_max@1 OR p_channel_email_null_count@6 != row_count@3 AND p_channel_email_min@4 <= Y AND Y <= p_channel_email_max@5 OR p_channel_tv_null_count@9 != row_count@3 AND p_channel_tv_min@7 <= Y AND Y <= p_channel_tv_max@8, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_promo_sk@6, ss_ext_sales_price@7] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_dmail@1 = Y OR p_channel_email@2 = Y OR p_channel_tv@3 = Y, projection=[p_promo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_dmail, p_channel_email, p_channel_tv], file_type=parquet, predicate=p_channel_dmail@8 = Y OR p_channel_email@9 = Y OR p_channel_tv@11 = Y, pruning_predicate=p_channel_dmail_null_count@2 != row_count@3 AND p_channel_dmail_min@0 <= Y AND Y <= p_channel_dmail_max@1 OR p_channel_email_null_count@6 != row_count@3 AND p_channel_email_min@4 <= Y AND Y <= p_channel_email_max@5 OR p_channel_tv_null_count@9 != row_count@3 AND p_channel_tv_min@7 <= Y AND Y <= p_channel_tv_max@8, required_guarantees=[] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_promo_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_promo_sk@6, ss_ext_sales_price@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_promo_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_gmt_offset@1 = Some(-500),3,2, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_gmt_offset], file_type=parquet, predicate=s_gmt_offset@27 = Some(-500),3,2, pruning_predicate=s_gmt_offset_null_count@2 != row_count@3 AND s_gmt_offset_min@0 <= Some(-500),3,2 AND Some(-500),3,2 <= s_gmt_offset_max@1, required_guarantees=[s_gmt_offset in (Some(-500),3,2)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_gmt_offset@1 = Some(-500),3,2, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_gmt_offset], file_type=parquet, predicate=s_gmt_offset@27 = Some(-500),3,2, pruning_predicate=s_gmt_offset_null_count@2 != row_count@3 AND s_gmt_offset_min@0 <= Some(-500),3,2 AND Some(-500),3,2 <= s_gmt_offset_max@1, required_guarantees=[s_gmt_offset in (Some(-500),3,2)] - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@2, ca_address_sk@0)], projection=[ss_item_sk@0, ss_ext_sales_price@1] │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_category@1 = Jewelry, projection=[i_item_sk@0] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=4 + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet, predicate=i_category@12 = Jewelry AND DynamicFilter [ empty ], pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1, required_guarantees=[i_category in (Jewelry)] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@2, ca_address_sk@0)], projection=[ss_item_sk@0, ss_ext_sales_price@1] + ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] t3:[p18..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_ext_sales_price@2, c_current_addr_sk@4] │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_ext_sales_price@2, c_current_addr_sk@4] + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@3] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 11, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@3] + ┌───── Stage 8 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_ext_sales_price@6] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 11, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_ext_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_gmt_offset@1 = Some(-500),3,2, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_gmt_offset], file_type=parquet, predicate=s_gmt_offset@27 = Some(-500),3,2, pruning_predicate=s_gmt_offset_null_count@2 != row_count@3 AND s_gmt_offset_min@0 <= Some(-500),3,2 AND Some(-500),3,2 <= s_gmt_offset_max@1, required_guarantees=[s_gmt_offset in (Some(-500),3,2)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_gmt_offset@1 = Some(-500),3,2, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_gmt_offset], file_type=parquet, predicate=s_gmt_offset@27 = Some(-500),3,2, pruning_predicate=s_gmt_offset_null_count@2 != row_count@3 AND s_gmt_offset_min@0 <= Some(-500),3,2 AND Some(-500),3,2 <= s_gmt_offset_max@1, required_guarantees=[s_gmt_offset in (Some(-500),3,2)] - └────────────────────────────────────────────────── "); Ok(()) } @@ -6544,57 +6262,49 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [w_substr@0 ASC, sm_type@1 ASC, web_name@2 ASC], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[w_substr@0 ASC, sm_type@1 ASC, web_name@2 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[w_substr@0 as w_substr, sm_type@1 as sm_type, web_name@2 as web_name, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@3 as 30 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@4 as 31-60 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@5 as 61-90 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@6 as 91-120 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@7 as >120 days] + │ AggregateExec: mode=FinalPartitioned, gby=[w_substr@0 as w_substr, sm_type@1 as sm_type, web_name@2 as web_name], aggr=[sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] + │ RepartitionExec: partitioning=Hash([w_substr@0, sm_type@1, web_name@2], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[w_substr@1 as w_substr, sm_type@2 as sm_type, web_name@3 as web_name], aggr=[sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] + │ ProjectionExec: expr=[ws_ship_date_sk@1 - ws_sold_date_sk@0 as __common_expr_1, w_substr@2 as w_substr, sm_type@3 as sm_type, web_name@4 as web_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_ship_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, w_substr@2, sm_type@3, web_name@4] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[w_substr@0 ASC, sm_type@1 ASC, web_name@2 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[w_substr@0 as w_substr, sm_type@1 as sm_type, web_name@2 as web_name, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@3 as 30 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@4 as 31-60 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@5 as 61-90 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@6 as 91-120 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@7 as >120 days] - │ AggregateExec: mode=FinalPartitioned, gby=[w_substr@0 as w_substr, sm_type@1 as sm_type, web_name@2 as web_name], aggr=[sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([w_substr@0, sm_type@1, web_name@2], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[w_substr@1 as w_substr, sm_type@2 as sm_type, web_name@3 as web_name], aggr=[sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] - │ ProjectionExec: expr=[ws_ship_date_sk@1 - ws_sold_date_sk@0 as __common_expr_1, w_substr@2 as w_substr, sm_type@3 as sm_type, web_name@4 as web_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_ship_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, w_substr@2, sm_type@3, web_name@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_web_site_sk@2, CAST(web_site.web_site_sk AS Float64)@2)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, w_substr@3, sm_type@4, web_name@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_name, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_web_site_sk@2, CAST(web_site.web_site_sk AS Float64)@2)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, w_substr@3, sm_type@4, web_name@6] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_name, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_ship_mode_sk@3, CAST(ship_mode.sm_ship_mode_sk AS Float64)@2)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, ws_web_site_sk@2, w_substr@4, sm_type@6] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_type, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ship_date_sk@2 as ws_ship_date_sk, ws_web_site_sk@3 as ws_web_site_sk, ws_ship_mode_sk@4 as ws_ship_mode_sk, w_substr@0 as w_substr] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(sq1.w_warehouse_sk AS Float64)@2, ws_warehouse_sk@4)], projection=[w_substr@0, ws_sold_date_sk@3, ws_ship_date_sk@4, ws_web_site_sk@5, ws_ship_mode_sk@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_ship_date_sk, ws_web_site_sk, ws_ship_mode_sk, ws_warehouse_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_ship_mode_sk@3, CAST(ship_mode.sm_ship_mode_sk AS Float64)@2)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, ws_web_site_sk@2, w_substr@4, sm_type@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_type, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ship_date_sk@2 as ws_ship_date_sk, ws_web_site_sk@3 as ws_web_site_sk, ws_ship_mode_sk@4 as ws_ship_mode_sk, w_substr@0 as w_substr] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(sq1.w_warehouse_sk AS Float64)@2, ws_warehouse_sk@4)], projection=[w_substr@0, ws_sold_date_sk@3, ws_ship_date_sk@4, ws_web_site_sk@5, ws_ship_mode_sk@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_ship_date_sk, ws_web_site_sk, ws_ship_mode_sk, ws_warehouse_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[substr(w_warehouse_name@2, 1, 20) as w_substr, w_warehouse_sk, CAST(w_warehouse_sk@0 AS Float64) as CAST(sq1.w_warehouse_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[substr(w_warehouse_name@2, 1, 20) as w_substr, w_warehouse_sk, CAST(w_warehouse_sk@0 AS Float64) as CAST(sq1.w_warehouse_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── "); Ok(()) } @@ -6609,50 +6319,44 @@ mod tests { │ FilterExec: CASE WHEN avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manager_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 > Some(0),19,6 THEN abs(sum(store_sales.ss_sales_price)@1 - avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manager_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2) / avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manager_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 END > Some(1000000000),30,10 │ WindowAggExec: wdw=[avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manager_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_manager_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(19, 6), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] │ SortExec: expr=[i_manager_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_manager_id@0], 3), input_partitions=3 + │ ProjectionExec: expr=[i_manager_id@0 as i_manager_id, sum(store_sales.ss_sales_price)@2 as sum(store_sales.ss_sales_price)] + │ AggregateExec: mode=FinalPartitioned, gby=[i_manager_id@0 as i_manager_id, d_moy@1 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_manager_id@0], 3), input_partitions=3 - │ ProjectionExec: expr=[i_manager_id@0 as i_manager_id, sum(store_sales.ss_sales_price)@2 as sum(store_sales.ss_sales_price)] - │ AggregateExec: mode=FinalPartitioned, gby=[i_manager_id@0 as i_manager_id, d_moy@1 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_manager_id@0, d_moy@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_manager_id@0 as i_manager_id, d_moy@2 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[i_manager_id@2, ss_sales_price@4, d_moy@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_manager_id@1 as i_manager_id, ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_moy@0 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@1)], projection=[d_moy@1, i_manager_id@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_manager_id@1, ss_sold_date_sk@2, ss_store_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_manager_id@0, d_moy@1], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_manager_id@0 as i_manager_id, d_moy@2 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[i_manager_id@2, ss_sales_price@4, d_moy@5] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[i_manager_id@1 as i_manager_id, ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_moy@0 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@1)], projection=[d_moy@1, i_manager_id@3, ss_store_sk@5, ss_sales_price@6] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_manager_id@1, ss_sold_date_sk@2, ss_store_sk@4, ss_sales_price@5] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_moy@1 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@1 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), projection=[d_date_sk@0, d_moy@2] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq, d_moy], file_type=parquet, predicate=d_month_seq@3 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), pruning_predicate=d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1200 AND 1200 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1201 AND 1201 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1202 AND 1202 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1203 AND 1203 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1204 AND 1204 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1205 AND 1205 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1206 AND 1206 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1207 AND 1207 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1208 AND 1208 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1209 AND 1209 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1210 AND 1210 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1211 AND 1211 <= d_month_seq_max@1, required_guarantees=[d_month_seq in (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: (i_category@3 = Books OR i_category@3 = Children OR i_category@3 = Electronics) AND i_class@2 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@1 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@3 = Women OR i_category@3 = Music OR i_category@3 = Men) AND i_class@2 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@1 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), projection=[i_item_sk@0, i_manager_id@4] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_manager_id], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Children OR i_category@12 = Electronics) AND i_class@10 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@8 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@12 = Women OR i_category@12 = Music OR i_category@12 = Men) AND i_class@10 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@8 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Children AND Children <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= personal AND personal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= portable AND portable <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= reference AND reference <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= self-help AND self-help <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #14 AND scholaramalgamalg #14 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #7 AND scholaramalgamalg #7 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiunivamalg #9 AND exportiunivamalg #9 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #9 AND scholaramalgamalg #9 <= i_brand_max@8) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= accessories AND accessories <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= classical AND classical <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= fragrances AND fragrances <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= pants AND pants <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= amalgimporto #1 AND amalgimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= edu packscholar #1 AND edu packscholar #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiimporto #1 AND exportiimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= importoamalg #1 AND importoamalg #1 <= i_brand_max@8), required_guarantees=[i_brand in (amalgimporto #1, edu packscholar #1, exportiimporto #1, exportiunivamalg #9, importoamalg #1, scholaramalgamalg #14, scholaramalgamalg #7, scholaramalgamalg #9), i_class in (accessories, classical, fragrances, pants, personal, portable, reference, self-help)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_moy@1 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@1 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), projection=[d_date_sk@0, d_moy@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq, d_moy], file_type=parquet, predicate=d_month_seq@3 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), pruning_predicate=d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1200 AND 1200 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1201 AND 1201 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1202 AND 1202 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1203 AND 1203 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1204 AND 1204 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1205 AND 1205 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1206 AND 1206 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1207 AND 1207 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1208 AND 1208 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1209 AND 1209 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1210 AND 1210 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1211 AND 1211 <= d_month_seq_max@1, required_guarantees=[d_month_seq in (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: (i_category@3 = Books OR i_category@3 = Children OR i_category@3 = Electronics) AND i_class@2 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@1 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@3 = Women OR i_category@3 = Music OR i_category@3 = Men) AND i_class@2 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@1 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), projection=[i_item_sk@0, i_manager_id@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_manager_id], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Children OR i_category@12 = Electronics) AND i_class@10 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@8 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@12 = Women OR i_category@12 = Music OR i_category@12 = Men) AND i_class@10 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@8 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Children AND Children <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= personal AND personal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= portable AND portable <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= reference AND reference <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= self-help AND self-help <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #14 AND scholaramalgamalg #14 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #7 AND scholaramalgamalg #7 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiunivamalg #9 AND exportiunivamalg #9 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #9 AND scholaramalgamalg #9 <= i_brand_max@8) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= accessories AND accessories <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= classical AND classical <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= fragrances AND fragrances <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= pants AND pants <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= amalgimporto #1 AND amalgimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= edu packscholar #1 AND edu packscholar #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiimporto #1 AND exportiimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= importoamalg #1 AND importoamalg #1 <= i_brand_max@8), required_guarantees=[i_brand in (amalgimporto #1, edu packscholar #1, exportiimporto #1, exportiunivamalg #9, importoamalg #1, scholaramalgamalg #14, scholaramalgamalg #7, scholaramalgamalg #9), i_class in (accessories, classical, fragrances, pants, personal, portable, reference, self-help)] - └────────────────────────────────────────────────── "#); Ok(()) } @@ -6663,386 +6367,360 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[product_name@0 as product_name, store_name@1 as store_name, store_zip@2 as store_zip, b_street_number@3 as b_street_number, b_street_name@4 as b_street_name, b_city@5 as b_city, b_zip@6 as b_zip, c_street_number@7 as c_street_number, c_street_name@8 as c_street_name, c_city@9 as c_city, c_zip@10 as c_zip, cs1syear@11 as cs1syear, cs1cnt@12 as cs1cnt, s11@13 as s11, s21@14 as s21, s31@15 as s31, s12@16 as s12, s22@17 as s22, s32@18 as s32, syear@19 as syear, cnt@20 as cnt] │ SortPreservingMergeExec: [product_name@0 ASC NULLS LAST, store_name@1 ASC NULLS LAST, cnt@20 ASC NULLS LAST, s1@21 ASC NULLS LAST, s1@22 ASC NULLS LAST] - │ [Stage 46] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: expr=[product_name@0 ASC NULLS LAST, store_name@1 ASC NULLS LAST, cnt@20 ASC NULLS LAST, s11@13 ASC NULLS LAST, s12@16 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[product_name@0 as product_name, store_name@1 as store_name, store_zip@2 as store_zip, b_street_number@3 as b_street_number, b_street_name@4 as b_street_name, b_city@5 as b_city, b_zip@6 as b_zip, c_street_number@7 as c_street_number, c_street_name@8 as c_street_name, c_city@9 as c_city, c_zip@10 as c_zip, syear@11 as cs1syear, cnt@12 as cs1cnt, s1@13 as s11, s2@14 as s21, s3@15 as s31, s1@18 as s12, s2@19 as s22, s3@20 as s32, syear@16 as syear, cnt@17 as cnt, s1@13 as s1, s1@18 as s1] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_sk@1, item_sk@0), (store_name@2, store_name@1), (store_zip@3, store_zip@2)], filter=cnt@1 <= cnt@0, projection=[product_name@0, store_name@2, store_zip@3, b_street_number@4, b_street_name@5, b_city@6, b_zip@7, c_street_number@8, c_street_name@9, c_city@10, c_zip@11, syear@12, cnt@13, s1@14, s2@15, s3@16, syear@20, cnt@21, s1@22, s2@23, s3@24] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ ProjectionExec: expr=[i_product_name@0 as product_name, i_item_sk@1 as item_sk, s_store_name@2 as store_name, s_zip@3 as store_zip, ca_street_number@4 as b_street_number, ca_street_name@5 as b_street_name, ca_city@6 as b_city, ca_zip@7 as b_zip, ca_street_number@8 as c_street_number, ca_street_name@9 as c_street_name, ca_city@10 as c_city, ca_zip@11 as c_zip, d_year@12 as syear, count(Int64(1))@15 as cnt, sum(store_sales.ss_wholesale_cost)@16 as s1, sum(store_sales.ss_list_price)@17 as s2, sum(store_sales.ss_coupon_amt)@18 as s3] + │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name, i_item_sk@1 as i_item_sk, s_store_name@2 as s_store_name, s_zip@3 as s_zip, ca_street_number@4 as ca_street_number, ca_street_name@5 as ca_street_name, ca_city@6 as ca_city, ca_zip@7 as ca_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, d_year@12 as d_year, d_year@13 as d_year, d_year@14 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) + │ [Stage 21] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[i_item_sk@1 as item_sk, s_store_name@2 as store_name, s_zip@3 as store_zip, d_year@12 as syear, count(Int64(1))@15 as cnt, sum(store_sales.ss_wholesale_cost)@16 as s1, sum(store_sales.ss_list_price)@17 as s2, sum(store_sales.ss_coupon_amt)@18 as s3] + │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name, i_item_sk@1 as i_item_sk, s_store_name@2 as s_store_name, s_zip@3 as s_zip, ca_street_number@4 as ca_street_number, ca_street_name@5 as ca_street_name, ca_city@6 as ca_city, ca_zip@7 as ca_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, d_year@12 as d_year, d_year@13 as d_year, d_year@14 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) + │ [Stage 42] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 46 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[product_name@0 ASC NULLS LAST, store_name@1 ASC NULLS LAST, cnt@20 ASC NULLS LAST, s11@13 ASC NULLS LAST, s12@16 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[product_name@0 as product_name, store_name@1 as store_name, store_zip@2 as store_zip, b_street_number@3 as b_street_number, b_street_name@4 as b_street_name, b_city@5 as b_city, b_zip@6 as b_zip, c_street_number@7 as c_street_number, c_street_name@8 as c_street_name, c_city@9 as c_city, c_zip@10 as c_zip, syear@11 as cs1syear, cnt@12 as cs1cnt, s1@13 as s11, s2@14 as s21, s3@15 as s31, s1@18 as s12, s2@19 as s22, s3@20 as s32, syear@16 as syear, cnt@17 as cnt, s1@13 as s1, s1@18 as s1] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_sk@1, item_sk@0), (store_name@2, store_name@1), (store_zip@3, store_zip@2)], filter=cnt@1 <= cnt@0, projection=[product_name@0, store_name@2, store_zip@3, b_street_number@4, b_street_name@5, b_city@6, b_zip@7, c_street_number@8, c_street_name@9, c_city@10, c_zip@11, syear@12, cnt@13, s1@14, s2@15, s3@16, syear@20, cnt@21, s1@22, s2@23, s3@24] - │ CoalescePartitionsExec - │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_item_sk@1 as item_sk, s_store_name@2 as store_name, s_zip@3 as store_zip, d_year@12 as syear, count(Int64(1))@15 as cnt, sum(store_sales.ss_wholesale_cost)@16 as s1, sum(store_sales.ss_list_price)@17 as s2, sum(store_sales.ss_coupon_amt)@18 as s3] - │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name, i_item_sk@1 as i_item_sk, s_store_name@2 as s_store_name, s_zip@3 as s_zip, ca_street_number@4 as ca_street_number, ca_street_name@5 as ca_street_name, ca_city@6 as ca_city, ca_zip@7 as ca_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, d_year@12 as d_year, d_year@13 as d_year, d_year@14 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) - │ [Stage 45] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 21 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_product_name@0, i_item_sk@1, s_store_name@2, s_zip@3, ca_street_number@4, ca_street_name@5, ca_city@6, ca_zip@7, ca_street_number@8, ca_street_name@9, ca_city@10, ca_zip@11, d_year@12, d_year@13, d_year@14], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_product_name@17 as i_product_name, i_item_sk@16 as i_item_sk, s_store_name@6 as s_store_name, s_zip@7 as s_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, d_year@3 as d_year, d_year@4 as d_year, d_year@5 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) + │ ProjectionExec: expr=[ss_wholesale_cost@0 as ss_wholesale_cost, ss_list_price@1 as ss_list_price, ss_coupon_amt@2 as ss_coupon_amt, d_year@3 as d_year, d_year@6 as d_year, d_year@7 as d_year, s_store_name@4 as s_store_name, s_zip@5 as s_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, i_item_sk@16 as i_item_sk, i_product_name@17 as i_product_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@9, ca_street_name@10, ca_city@11, ca_zip@12, ca_street_number@13, ca_street_name@14, ca_city@15, ca_zip@16, i_item_sk@17, i_product_name@18] + │ CoalescePartitionsExec + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 + │ FilterExec: i_color@2 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@1 >= Some(6500),4,2 AND i_current_price@1 <= Some(7400),4,2, projection=[i_item_sk@0, i_product_name@3] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_color, i_product_name], file_type=parquet, predicate=i_color@17 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@5 >= Some(6500),4,2 AND i_current_price@5 <= Some(7400),4,2 AND DynamicFilter [ empty ], pruning_predicate=(i_color_null_count@2 != row_count@3 AND i_color_min@0 <= purple AND purple <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burlywood AND burlywood <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= indian AND indian <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= spring AND spring <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= floral AND floral <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= medium AND medium <= i_color_max@1) AND i_current_price_null_count@5 != row_count@3 AND i_current_price_max@4 >= Some(6500),4,2 AND i_current_price_null_count@5 != row_count@3 AND i_current_price_min@6 <= Some(7400),4,2, required_guarantees=[i_color in (burlywood, floral, indian, medium, purple, spring)] └────────────────────────────────────────────────── - ┌───── Stage 23 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[i_product_name@0 as product_name, i_item_sk@1 as item_sk, s_store_name@2 as store_name, s_zip@3 as store_zip, ca_street_number@4 as b_street_number, ca_street_name@5 as b_street_name, ca_city@6 as b_city, ca_zip@7 as b_zip, ca_street_number@8 as c_street_number, ca_street_name@9 as c_street_name, ca_city@10 as c_city, ca_zip@11 as c_zip, d_year@12 as syear, count(Int64(1))@15 as cnt, sum(store_sales.ss_wholesale_cost)@16 as s1, sum(store_sales.ss_list_price)@17 as s2, sum(store_sales.ss_coupon_amt)@18 as s3] - │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name, i_item_sk@1 as i_item_sk, s_store_name@2 as s_store_name, s_zip@3 as s_zip, ca_street_number@4 as ca_street_number, ca_street_name@5 as ca_street_name, ca_city@6 as ca_city, ca_zip@7 as ca_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, d_year@12 as d_year, d_year@13 as d_year, d_year@14 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) - │ [Stage 22] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_product_name@0, i_item_sk@1, s_store_name@2, s_zip@3, ca_street_number@4, ca_street_name@5, ca_city@6, ca_zip@7, ca_street_number@8, ca_street_name@9, ca_city@10, ca_zip@11, d_year@12, d_year@13, d_year@14], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_product_name@17 as i_product_name, i_item_sk@16 as i_item_sk, s_store_name@6 as s_store_name, s_zip@7 as s_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, d_year@3 as d_year, d_year@4 as d_year, d_year@5 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) - │ ProjectionExec: expr=[ss_wholesale_cost@0 as ss_wholesale_cost, ss_list_price@1 as ss_list_price, ss_coupon_amt@2 as ss_coupon_amt, d_year@3 as d_year, d_year@6 as d_year, d_year@7 as d_year, s_store_name@4 as s_store_name, s_zip@5 as s_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, i_item_sk@16 as i_item_sk, i_product_name@17 as i_product_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@9, ca_street_name@10, ca_city@11, ca_zip@12, ca_street_number@13, ca_street_name@14, ca_city@15, ca_zip@16, i_item_sk@17, i_product_name@18] - │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ FilterExec: i_color@2 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@1 >= Some(6500),4,2 AND i_current_price@1 <= Some(7400),4,2, projection=[i_item_sk@0, i_product_name@3] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_color, i_product_name], file_type=parquet, predicate=i_color@17 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@5 >= Some(6500),4,2 AND i_current_price@5 <= Some(7400),4,2 AND DynamicFilter [ empty ], pruning_predicate=(i_color_null_count@2 != row_count@3 AND i_color_min@0 <= purple AND purple <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burlywood AND burlywood <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= indian AND indian <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= spring AND spring <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= floral AND floral <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= medium AND medium <= i_color_max@1) AND i_current_price_null_count@5 != row_count@3 AND i_current_price_max@4 >= Some(6500),4,2 AND i_current_price_null_count@5 != row_count@3 AND i_current_price_min@6 <= Some(7400),4,2, required_guarantees=[i_color in (burlywood, floral, indian, medium, purple, spring)] + ┌───── Stage 20 ── Tasks: t0:[p0..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(hd_income_band_sk@9, ib_income_band_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@10, ca_street_name@11, ca_city@12, ca_zip@13, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + │ CoalescePartitionsExec + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 19 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@9)], projection=[ss_item_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, d_year@8, d_year@9, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@16, ca_street_name@17, ca_city@18, ca_zip@19] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@7, ca_address_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@8, d_year@9, hd_income_band_sk@10, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@17, ca_street_name@18, ca_city@19, ca_zip@20] + │ CoalescePartitionsExec + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(hd_income_band_sk@9, ib_income_band_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@10, ca_street_name@11, ca_city@12, ca_zip@13, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 18 ── Tasks: t0:[p0..p5] t1:[p6..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(ad1.ca_address_sk AS Float64)@5)], projection=[ss_item_sk@0, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@8, d_year@9, d_year@10, hd_income_band_sk@11, hd_income_band_sk@12, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_city@3 as ca_city, ca_zip@4 as ca_zip, CAST(ca_address_sk@0 AS Float64) as CAST(ad1.ca_address_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@9)], projection=[ss_item_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, d_year@8, d_year@9, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@16, ca_street_name@17, ca_city@18, ca_zip@19] + ┌───── Stage 17 ── Tasks: t0:[p0..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@8, CAST(hd2.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@9, d_year@10, d_year@11, hd_income_band_sk@12, hd_income_band_sk@14] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@7, ca_address_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@8, d_year@9, hd_income_band_sk@10, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@17, ca_street_name@18, ca_city@19, ca_zip@20] - │ CoalescePartitionsExec - │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] + │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(hd1.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@2, ss_wholesale_cost@3, ss_list_price@4, ss_coupon_amt@5, d_year@6, s_store_name@7, s_zip@8, c_current_hdemo_sk@9, c_current_addr_sk@10, d_year@11, d_year@12, hd_income_band_sk@14] + │ CoalescePartitionsExec + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=4 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd1.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd2.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(ad1.ca_address_sk AS Float64)@5)], projection=[ss_item_sk@0, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@8, d_year@9, d_year@10, hd_income_band_sk@11, hd_income_band_sk@12, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + ┌───── Stage 16 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@3)], projection=[ss_item_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15] │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_city@3 as ca_city, ca_zip@4 as ca_zip, CAST(ca_address_sk@0 AS Float64) as CAST(ad1.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@10, CAST(cd2.cd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 != cd_marital_status@0, projection=[ss_item_sk@0, ss_hdemo_sk@1, ss_addr_sk@2, ss_promo_sk@3, ss_wholesale_cost@4, ss_list_price@5, ss_coupon_amt@6, d_year@7, s_store_name@8, s_zip@9, c_current_hdemo_sk@11, c_current_addr_sk@12, d_year@13, d_year@14] + │ CoalescePartitionsExec + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@8, CAST(hd2.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@9, d_year@10, d_year@11, hd_income_band_sk@12, hd_income_band_sk@14] + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 15 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(cd1.cd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15, cd_marital_status@17] │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd2.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(hd1.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@2, ss_wholesale_cost@3, ss_list_price@4, ss_coupon_amt@5, d_year@6, s_store_name@7, s_zip@8, c_current_hdemo_sk@9, c_current_addr_sk@10, d_year@11, d_year@12, hd_income_band_sk@14] + ┌───── Stage 14 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_shipto_date_sk@14, CAST(d3.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@15, d_year@17] │ CoalescePartitionsExec - │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd1.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@3)], projection=[ss_item_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15] + ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_sales_date_sk@15, CAST(d2.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, c_first_shipto_date_sk@14, d_year@17] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@10, CAST(cd2.cd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 != cd_marital_status@0, projection=[ss_item_sk@0, ss_hdemo_sk@1, ss_addr_sk@2, ss_promo_sk@3, ss_wholesale_cost@4, ss_list_price@5, ss_coupon_amt@6, d_year@7, s_store_name@8, s_zip@9, c_current_hdemo_sk@11, c_current_addr_sk@12, d_year@13, d_year@14] - │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(cd1.cd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15, cd_marital_status@17] + ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] t3:[p27..p35] + │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ss_item_sk@0, ss_cdemo_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_promo_sk@5, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_cdemo_sk@13, c_current_hdemo_sk@14, c_current_addr_sk@15, c_first_shipto_date_sk@16, c_first_sales_date_sk@17] │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_shipto_date_sk@14, CAST(d3.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@15, d_year@17] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet + ┌───── Stage 11 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_cdemo_sk@4 as ss_cdemo_sk, ss_hdemo_sk@5 as ss_hdemo_sk, ss_addr_sk@6 as ss_addr_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@11 as d_year, s_store_name@0 as s_store_name, s_zip@1 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@5)], projection=[s_store_name@1, s_zip@2, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13, d_year@14] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_cdemo_sk@3 as ss_cdemo_sk, ss_hdemo_sk@4 as ss_hdemo_sk, ss_addr_sk@5 as ss_addr_sk, ss_store_sk@6 as ss_store_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_store_sk@9, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_sales_date_sk@15, CAST(d2.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, c_first_shipto_date_sk@14, d_year@17] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + ┌───── Stage 3 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ss_item_sk@0, ss_cdemo_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_promo_sk@5, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_cdemo_sk@13, c_current_hdemo_sk@14, c_current_addr_sk@15, c_first_shipto_date_sk@16, c_first_sales_date_sk@17] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ ProjectionExec: expr=[cs_item_sk@3 as cs_item_sk, cs_ext_list_price@4 as cs_ext_list_price, cr_refunded_cash@0 as cr_refunded_cash, cr_reversed_charge@1 as cr_reversed_charge, cr_store_credit@2 as cr_store_credit] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cr_item_sk@0, cs_item_sk@0), (cr_order_number@1, cs_order_number@1)], projection=[cr_refunded_cash@2, cr_reversed_charge@3, cr_store_credit@4, cs_item_sk@5, cs_ext_list_price@7] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_cdemo_sk@4 as ss_cdemo_sk, ss_hdemo_sk@5 as ss_hdemo_sk, ss_addr_sk@6 as ss_addr_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@11 as d_year, s_store_name@0 as s_store_name, s_zip@1 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@5)], projection=[s_store_name@1, s_zip@2, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13, d_year@14] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_cdemo_sk@3 as ss_cdemo_sk, ss_hdemo_sk@4 as ss_hdemo_sk, ss_addr_sk@5 as ss_addr_sk, ss_store_sk@6 as ss_store_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_store_sk@9, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash, cr_reversed_charge, cr_store_credit], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ ProjectionExec: expr=[cs_item_sk@3 as cs_item_sk, cs_ext_list_price@4 as cs_ext_list_price, cr_refunded_cash@0 as cr_refunded_cash, cr_reversed_charge@1 as cr_reversed_charge, cr_store_credit@2 as cr_store_credit] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cr_item_sk@0, cs_item_sk@0), (cr_order_number@1, cs_order_number@1)], projection=[cr_refunded_cash@2, cr_reversed_charge@3, cr_store_credit@4, cs_item_sk@5, cs_ext_list_price@7] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash, cr_reversed_charge, cr_store_credit], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_item_sk@0, cs_order_number@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk, cs_order_number, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@8], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_promo_sk, ss_ticket_number, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 45 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_product_name@0, i_item_sk@1, s_store_name@2, s_zip@3, ca_street_number@4, ca_street_name@5, ca_city@6, ca_zip@7, ca_street_number@8, ca_street_name@9, ca_city@10, ca_zip@11, d_year@12, d_year@13, d_year@14], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[i_product_name@17 as i_product_name, i_item_sk@16 as i_item_sk, s_store_name@6 as s_store_name, s_zip@7 as s_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, d_year@3 as d_year, d_year@4 as d_year, d_year@5 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) - │ ProjectionExec: expr=[ss_wholesale_cost@0 as ss_wholesale_cost, ss_list_price@1 as ss_list_price, ss_coupon_amt@2 as ss_coupon_amt, d_year@3 as d_year, d_year@6 as d_year, d_year@7 as d_year, s_store_name@4 as s_store_name, s_zip@5 as s_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, i_item_sk@16 as i_item_sk, i_product_name@17 as i_product_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@9, ca_street_name@10, ca_city@11, ca_zip@12, ca_street_number@13, ca_street_name@14, ca_city@15, ca_zip@16, i_item_sk@17, i_product_name@18] - │ CoalescePartitionsExec - │ [Stage 44] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ FilterExec: i_color@2 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@1 >= Some(6500),4,2 AND i_current_price@1 <= Some(7400),4,2, projection=[i_item_sk@0, i_product_name@3] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_color, i_product_name], file_type=parquet, predicate=i_color@17 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@5 >= Some(6500),4,2 AND i_current_price@5 <= Some(7400),4,2 AND DynamicFilter [ empty ], pruning_predicate=(i_color_null_count@2 != row_count@3 AND i_color_min@0 <= purple AND purple <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burlywood AND burlywood <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= indian AND indian <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= spring AND spring <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= floral AND floral <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= medium AND medium <= i_color_max@1) AND i_current_price_null_count@5 != row_count@3 AND i_current_price_max@4 >= Some(6500),4,2 AND i_current_price_null_count@5 != row_count@3 AND i_current_price_min@6 <= Some(7400),4,2, required_guarantees=[i_color in (burlywood, floral, indian, medium, purple, spring)] + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_item_sk@0, cs_order_number@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk, cs_order_number, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 12), input_partitions=3 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@8], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_promo_sk, ss_ticket_number, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 42 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_product_name@0, i_item_sk@1, s_store_name@2, s_zip@3, ca_street_number@4, ca_street_name@5, ca_city@6, ca_zip@7, ca_street_number@8, ca_street_name@9, ca_city@10, ca_zip@11, d_year@12, d_year@13, d_year@14], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[i_product_name@17 as i_product_name, i_item_sk@16 as i_item_sk, s_store_name@6 as s_store_name, s_zip@7 as s_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, d_year@3 as d_year, d_year@4 as d_year, d_year@5 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) + │ ProjectionExec: expr=[ss_wholesale_cost@0 as ss_wholesale_cost, ss_list_price@1 as ss_list_price, ss_coupon_amt@2 as ss_coupon_amt, d_year@3 as d_year, d_year@6 as d_year, d_year@7 as d_year, s_store_name@4 as s_store_name, s_zip@5 as s_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, i_item_sk@16 as i_item_sk, i_product_name@17 as i_product_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@9, ca_street_name@10, ca_city@11, ca_zip@12, ca_street_number@13, ca_street_name@14, ca_city@15, ca_zip@16, i_item_sk@17, i_product_name@18] + │ CoalescePartitionsExec + │ [Stage 41] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 + │ FilterExec: i_color@2 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@1 >= Some(6500),4,2 AND i_current_price@1 <= Some(7400),4,2, projection=[i_item_sk@0, i_product_name@3] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_color, i_product_name], file_type=parquet, predicate=i_color@17 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@5 >= Some(6500),4,2 AND i_current_price@5 <= Some(7400),4,2 AND DynamicFilter [ empty ], pruning_predicate=(i_color_null_count@2 != row_count@3 AND i_color_min@0 <= purple AND purple <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burlywood AND burlywood <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= indian AND indian <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= spring AND spring <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= floral AND floral <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= medium AND medium <= i_color_max@1) AND i_current_price_null_count@5 != row_count@3 AND i_current_price_max@4 >= Some(6500),4,2 AND i_current_price_null_count@5 != row_count@3 AND i_current_price_min@6 <= Some(7400),4,2, required_guarantees=[i_color in (burlywood, floral, indian, medium, purple, spring)] + └────────────────────────────────────────────────── + ┌───── Stage 41 ── Tasks: t0:[p0..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(hd_income_band_sk@9, ib_income_band_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@10, ca_street_name@11, ca_city@12, ca_zip@13, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + │ CoalescePartitionsExec + │ [Stage 40] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 44 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(hd_income_band_sk@9, ib_income_band_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@10, ca_street_name@11, ca_city@12, ca_zip@13, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + ┌───── Stage 40 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@9)], projection=[ss_item_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, d_year@8, d_year@9, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@16, ca_street_name@17, ca_city@18, ca_zip@19] │ CoalescePartitionsExec - │ [Stage 43] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@7, ca_address_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@8, d_year@9, hd_income_band_sk@10, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@17, ca_street_name@18, ca_city@19, ca_zip@20] + │ CoalescePartitionsExec + │ [Stage 39] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 43 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@9)], projection=[ss_item_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, d_year@8, d_year@9, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@16, ca_street_name@17, ca_city@18, ca_zip@19] + ┌───── Stage 22 ── Tasks: t0:[p0..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 39 ── Tasks: t0:[p0..p5] t1:[p6..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(ad1.ca_address_sk AS Float64)@5)], projection=[ss_item_sk@0, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@8, d_year@9, d_year@10, hd_income_band_sk@11, hd_income_band_sk@12, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] │ CoalescePartitionsExec - │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@7, ca_address_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@8, d_year@9, hd_income_band_sk@10, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@17, ca_street_name@18, ca_city@19, ca_zip@20] - │ CoalescePartitionsExec - │ [Stage 42] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 38] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_city@3 as ca_city, ca_zip@4 as ca_zip, CAST(ca_address_sk@0 AS Float64) as CAST(ad1.ca_address_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 42 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(ad1.ca_address_sk AS Float64)@5)], projection=[ss_item_sk@0, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@8, d_year@9, d_year@10, hd_income_band_sk@11, hd_income_band_sk@12, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + ┌───── Stage 38 ── Tasks: t0:[p0..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@8, CAST(hd2.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@9, d_year@10, d_year@11, hd_income_band_sk@12, hd_income_band_sk@14] │ CoalescePartitionsExec - │ [Stage 41] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_city@3 as ca_city, ca_zip@4 as ca_zip, CAST(ca_address_sk@0 AS Float64) as CAST(ad1.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet + │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(hd1.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@2, ss_wholesale_cost@3, ss_list_price@4, ss_coupon_amt@5, d_year@6, s_store_name@7, s_zip@8, c_current_hdemo_sk@9, c_current_addr_sk@10, d_year@11, d_year@12, hd_income_band_sk@14] + │ CoalescePartitionsExec + │ [Stage 37] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=4 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd1.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd2.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 41 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@8, CAST(hd2.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@9, d_year@10, d_year@11, hd_income_band_sk@12, hd_income_band_sk@14] + ┌───── Stage 37 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@3)], projection=[ss_item_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15] │ CoalescePartitionsExec - │ [Stage 40] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd2.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@10, CAST(cd2.cd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 != cd_marital_status@0, projection=[ss_item_sk@0, ss_hdemo_sk@1, ss_addr_sk@2, ss_promo_sk@3, ss_wholesale_cost@4, ss_list_price@5, ss_coupon_amt@6, d_year@7, s_store_name@8, s_zip@9, c_current_hdemo_sk@11, c_current_addr_sk@12, d_year@13, d_year@14] + │ CoalescePartitionsExec + │ [Stage 36] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 40 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(hd1.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@2, ss_wholesale_cost@3, ss_list_price@4, ss_coupon_amt@5, d_year@6, s_store_name@7, s_zip@8, c_current_hdemo_sk@9, c_current_addr_sk@10, d_year@11, d_year@12, hd_income_band_sk@14] + ┌───── Stage 23 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 36 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(cd1.cd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15, cd_marital_status@17] │ CoalescePartitionsExec - │ [Stage 39] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd1.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 35] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 39 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@3)], projection=[ss_item_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15] + ┌───── Stage 35 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_shipto_date_sk@14, CAST(d3.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@15, d_year@17] │ CoalescePartitionsExec - │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@10, CAST(cd2.cd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 != cd_marital_status@0, projection=[ss_item_sk@0, ss_hdemo_sk@1, ss_addr_sk@2, ss_promo_sk@3, ss_wholesale_cost@4, ss_list_price@5, ss_coupon_amt@6, d_year@7, s_store_name@8, s_zip@9, c_current_hdemo_sk@11, c_current_addr_sk@12, d_year@13, d_year@14] - │ CoalescePartitionsExec - │ [Stage 38] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 34] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 25 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 38 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(cd1.cd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15, cd_marital_status@17] + ┌───── Stage 34 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_sales_date_sk@15, CAST(d2.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, c_first_shipto_date_sk@14, d_year@17] │ CoalescePartitionsExec - │ [Stage 37] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 33] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 37 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] + ┌───── Stage 33 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] t3:[p27..p35] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_shipto_date_sk@14, CAST(d3.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@15, d_year@17] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ss_item_sk@0, ss_cdemo_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_promo_sk@5, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_cdemo_sk@13, c_current_hdemo_sk@14, c_current_addr_sk@15, c_first_shipto_date_sk@16, c_first_sales_date_sk@17] │ CoalescePartitionsExec - │ [Stage 36] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet + │ [Stage 32] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 36 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_sales_date_sk@15, CAST(d2.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, c_first_shipto_date_sk@14, d_year@17] - │ CoalescePartitionsExec - │ [Stage 35] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet + ┌───── Stage 32 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_cdemo_sk@4 as ss_cdemo_sk, ss_hdemo_sk@5 as ss_hdemo_sk, ss_addr_sk@6 as ss_addr_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@11 as d_year, s_store_name@0 as s_store_name, s_zip@1 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@5)], projection=[s_store_name@1, s_zip@2, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13, d_year@14] + │ CoalescePartitionsExec + │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_cdemo_sk@3 as ss_cdemo_sk, ss_hdemo_sk@4 as ss_hdemo_sk, ss_addr_sk@5 as ss_addr_sk, ss_store_sk@6 as ss_store_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_store_sk@9, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] + │ CoalescePartitionsExec + │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] + │ CoalescePartitionsExec + │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] + │ [Stage 30] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 31] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 35 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ss_item_sk@0, ss_cdemo_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_promo_sk@5, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_cdemo_sk@13, c_current_hdemo_sk@14, c_current_addr_sk@15, c_first_shipto_date_sk@16, c_first_sales_date_sk@17] - │ CoalescePartitionsExec - │ [Stage 34] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + ┌───── Stage 24 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 25 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 34 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_cdemo_sk@4 as ss_cdemo_sk, ss_hdemo_sk@5 as ss_hdemo_sk, ss_addr_sk@6 as ss_addr_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@11 as d_year, s_store_name@0 as s_store_name, s_zip@1 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@5)], projection=[s_store_name@1, s_zip@2, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13, d_year@14] - │ CoalescePartitionsExec - │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_cdemo_sk@3 as ss_cdemo_sk, ss_hdemo_sk@4 as ss_hdemo_sk, ss_addr_sk@5 as ss_addr_sk, ss_store_sk@6 as ss_store_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_store_sk@9, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ CoalescePartitionsExec - │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] - │ CoalescePartitionsExec - │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ [Stage 32] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 33] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 29 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ [Stage 28] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 28 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 12), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ ProjectionExec: expr=[cs_item_sk@3 as cs_item_sk, cs_ext_list_price@4 as cs_ext_list_price, cr_refunded_cash@0 as cr_refunded_cash, cr_reversed_charge@1 as cr_reversed_charge, cr_store_credit@2 as cr_store_credit] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cr_item_sk@0, cs_item_sk@0), (cr_order_number@1, cs_order_number@1)], projection=[cr_refunded_cash@2, cr_reversed_charge@3, cr_store_credit@4, cs_item_sk@5, cs_ext_list_price@7] + │ [Stage 26] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 27] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 26 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 27 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] - └────────────────────────────────────────────────── - ┌───── Stage 31 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ [Stage 30] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 30 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ ProjectionExec: expr=[cs_item_sk@3 as cs_item_sk, cs_ext_list_price@4 as cs_ext_list_price, cr_refunded_cash@0 as cr_refunded_cash, cr_reversed_charge@1 as cr_reversed_charge, cr_store_credit@2 as cr_store_credit] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cr_item_sk@0, cs_item_sk@0), (cr_order_number@1, cs_order_number@1)], projection=[cr_refunded_cash@2, cr_reversed_charge@3, cr_store_credit@4, cs_item_sk@5, cs_ext_list_price@7] - │ [Stage 28] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 29] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 28 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash, cr_reversed_charge, cr_store_credit], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 29 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_item_sk@0, cs_order_number@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk, cs_order_number, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 32 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet + ┌───── Stage 26 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash, cr_reversed_charge, cr_store_credit], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 33 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@8], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_promo_sk, ss_ticket_number, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 27 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_item_sk@0, cs_order_number@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk, cs_order_number, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 30 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 12), input_partitions=3 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 31 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@8], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_promo_sk, ss_ticket_number, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── "); Ok(()) } @@ -7052,81 +6730,78 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_store_name@0 ASC, i_item_desc@1 ASC], fetch=100 - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC, i_item_desc@1 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[s_store_name@0 as s_store_name, i_item_desc@2 as i_item_desc, revenue@1 as revenue, i_current_price@3 as i_current_price, i_wholesale_cost@4 as i_wholesale_cost, i_brand@5 as i_brand] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_store_sk@1, ss_store_sk@0)], filter=CAST(revenue@0 AS Decimal128(30, 15)) <= CAST(0.1 * CAST(ave@1 AS Float64) AS Decimal128(30, 15)), projection=[s_store_name@0, revenue@2, i_item_desc@3, i_current_price@4, i_wholesale_cost@5, i_brand@6] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=3 │ ProjectionExec: expr=[ss_store_sk@0 as ss_store_sk, avg(sa.revenue)@1 as ave] │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(sa.revenue)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@2, i_item_sk@0)], projection=[s_store_name@0, ss_store_sk@1, revenue@3, i_item_desc@5, i_current_price@6, i_wholesale_cost@7, i_brand@8] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=4 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc, i_current_price, i_wholesale_cost, i_brand], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] t3:[p27..p35] + │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@0)], projection=[s_store_name@1, ss_store_sk@3, ss_item_sk@4, revenue@5] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ ProjectionExec: expr=[ss_store_sk@0 as ss_store_sk, ss_item_sk@1 as ss_item_sk, sum(store_sales.ss_sales_price)@2 as revenue] │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk, ss_item_sk@1 as ss_item_sk], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_store_sk@0, ss_item_sk@1], 6), input_partitions=2 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_store_sk@0, ss_item_sk@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_store_sk@1 as ss_store_sk, ss_item_sk@0 as ss_item_sk], aggr=[sum(store_sales.ss_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_sales_price@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@1 >= 1176 AND d_month_seq@1 <= 1187, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1176 AND d_month_seq@3 <= 1187, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1176 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1187, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1176 AND d_month_seq@3 <= 1187, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1176 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1187, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_store_sk@0], 6), input_partitions=3 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_store_sk@0], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(sa.revenue)] │ ProjectionExec: expr=[ss_store_sk@0 as ss_store_sk, sum(store_sales.ss_sales_price)@2 as revenue] │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk, ss_item_sk@1 as ss_item_sk], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_store_sk@0, ss_item_sk@1], 6), input_partitions=2 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_store_sk@0, ss_item_sk@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_store_sk@1 as ss_store_sk, ss_item_sk@0 as ss_item_sk], aggr=[sum(store_sales.ss_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_sales_price@5] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@1 >= 1176 AND d_month_seq@1 <= 1187, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1176 AND d_month_seq@3 <= 1187, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1176 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1187, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1176 AND d_month_seq@3 <= 1187, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1176 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1187, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -7140,118 +6815,111 @@ mod tests { │ SortExec: TopK(fetch=100), expr=[w_warehouse_name@0 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, ship_carriers@6 as ship_carriers, year_@7 as year_, sum(x.jan_sales)@8 as jan_sales, sum(x.feb_sales)@9 as feb_sales, sum(x.mar_sales)@10 as mar_sales, sum(x.apr_sales)@11 as apr_sales, sum(x.may_sales)@12 as may_sales, sum(x.jun_sales)@13 as jun_sales, sum(x.jul_sales)@14 as jul_sales, sum(x.aug_sales)@15 as aug_sales, sum(x.sep_sales)@16 as sep_sales, sum(x.oct_sales)@17 as oct_sales, sum(x.nov_sales)@18 as nov_sales, sum(x.dec_sales)@19 as dec_sales, sum(x.jan_sales / x.w_warehouse_sq_ft)@20 as jan_sales_per_sq_foot, sum(x.feb_sales / x.w_warehouse_sq_ft)@21 as feb_sales_per_sq_foot, sum(x.mar_sales / x.w_warehouse_sq_ft)@22 as mar_sales_per_sq_foot, sum(x.apr_sales / x.w_warehouse_sq_ft)@23 as apr_sales_per_sq_foot, sum(x.may_sales / x.w_warehouse_sq_ft)@24 as may_sales_per_sq_foot, sum(x.jun_sales / x.w_warehouse_sq_ft)@25 as jun_sales_per_sq_foot, sum(x.jul_sales / x.w_warehouse_sq_ft)@26 as jul_sales_per_sq_foot, sum(x.aug_sales / x.w_warehouse_sq_ft)@27 as aug_sales_per_sq_foot, sum(x.sep_sales / x.w_warehouse_sq_ft)@28 as sep_sales_per_sq_foot, sum(x.oct_sales / x.w_warehouse_sq_ft)@29 as oct_sales_per_sq_foot, sum(x.nov_sales / x.w_warehouse_sq_ft)@30 as nov_sales_per_sq_foot, sum(x.dec_sales / x.w_warehouse_sq_ft)@31 as dec_sales_per_sq_foot, sum(x.jan_net)@32 as jan_net, sum(x.feb_net)@33 as feb_net, sum(x.mar_net)@34 as mar_net, sum(x.apr_net)@35 as apr_net, sum(x.may_net)@36 as may_net, sum(x.jun_net)@37 as jun_net, sum(x.jul_net)@38 as jul_net, sum(x.aug_net)@39 as aug_net, sum(x.sep_net)@40 as sep_net, sum(x.oct_net)@41 as oct_net, sum(x.nov_net)@42 as nov_net, sum(x.dec_net)@43 as dec_net] │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, ship_carriers@6 as ship_carriers, year_@7 as year_], aggr=[sum(x.jan_sales), sum(x.feb_sales), sum(x.mar_sales), sum(x.apr_sales), sum(x.may_sales), sum(x.jun_sales), sum(x.jul_sales), sum(x.aug_sales), sum(x.sep_sales), sum(x.oct_sales), sum(x.nov_sales), sum(x.dec_sales), sum(x.jan_sales / x.w_warehouse_sq_ft), sum(x.feb_sales / x.w_warehouse_sq_ft), sum(x.mar_sales / x.w_warehouse_sq_ft), sum(x.apr_sales / x.w_warehouse_sq_ft), sum(x.may_sales / x.w_warehouse_sq_ft), sum(x.jun_sales / x.w_warehouse_sq_ft), sum(x.jul_sales / x.w_warehouse_sq_ft), sum(x.aug_sales / x.w_warehouse_sq_ft), sum(x.sep_sales / x.w_warehouse_sq_ft), sum(x.oct_sales / x.w_warehouse_sq_ft), sum(x.nov_sales / x.w_warehouse_sq_ft), sum(x.dec_sales / x.w_warehouse_sq_ft), sum(x.jan_net), sum(x.feb_net), sum(x.mar_net), sum(x.apr_net), sum(x.may_net), sum(x.jun_net), sum(x.jul_net), sum(x.aug_net), sum(x.sep_net), sum(x.oct_net), sum(x.nov_net), sum(x.dec_net)] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sq_ft@1, w_city@2, w_county@3, w_state@4, w_country@5, ship_carriers@6, year_@7], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[w_warehouse_name@1 as w_warehouse_name, w_warehouse_sq_ft@2 as w_warehouse_sq_ft, w_city@3 as w_city, w_county@4 as w_county, w_state@5 as w_state, w_country@6 as w_country, ship_carriers@7 as ship_carriers, year_@8 as year_], aggr=[sum(x.jan_sales), sum(x.feb_sales), sum(x.mar_sales), sum(x.apr_sales), sum(x.may_sales), sum(x.jun_sales), sum(x.jul_sales), sum(x.aug_sales), sum(x.sep_sales), sum(x.oct_sales), sum(x.nov_sales), sum(x.dec_sales), sum(x.jan_sales / x.w_warehouse_sq_ft), sum(x.feb_sales / x.w_warehouse_sq_ft), sum(x.mar_sales / x.w_warehouse_sq_ft), sum(x.apr_sales / x.w_warehouse_sq_ft), sum(x.may_sales / x.w_warehouse_sq_ft), sum(x.jun_sales / x.w_warehouse_sq_ft), sum(x.jul_sales / x.w_warehouse_sq_ft), sum(x.aug_sales / x.w_warehouse_sq_ft), sum(x.sep_sales / x.w_warehouse_sq_ft), sum(x.oct_sales / x.w_warehouse_sq_ft), sum(x.nov_sales / x.w_warehouse_sq_ft), sum(x.dec_sales / x.w_warehouse_sq_ft), sum(x.jan_net), sum(x.feb_net), sum(x.mar_net), sum(x.apr_net), sum(x.may_net), sum(x.jun_net), sum(x.jul_net), sum(x.aug_net), sum(x.sep_net), sum(x.oct_net), sum(x.nov_net), sum(x.dec_net)] + │ ProjectionExec: expr=[CAST(w_warehouse_sq_ft@1 AS Float64) as __common_expr_1, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, ship_carriers@6 as ship_carriers, year_@7 as year_, jan_sales@8 as jan_sales, feb_sales@9 as feb_sales, mar_sales@10 as mar_sales, apr_sales@11 as apr_sales, may_sales@12 as may_sales, jun_sales@13 as jun_sales, jul_sales@14 as jul_sales, aug_sales@15 as aug_sales, sep_sales@16 as sep_sales, oct_sales@17 as oct_sales, nov_sales@18 as nov_sales, dec_sales@19 as dec_sales, jan_net@20 as jan_net, feb_net@21 as feb_net, mar_net@22 as mar_net, apr_net@23 as apr_net, may_net@24 as may_net, jun_net@25 as jun_net, jul_net@26 as jul_net, aug_net@27 as aug_net, sep_net@28 as sep_net, oct_net@29 as oct_net, nov_net@30 as nov_net, dec_net@31 as dec_net] + │ InterleaveExec + │ ProjectionExec: expr=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, DHL,BARIAN as ship_carriers, d_year@6 as year_, sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@7 as jan_sales, sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@8 as feb_sales, sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@9 as mar_sales, sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@10 as apr_sales, sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@11 as may_sales, sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@12 as jun_sales, sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@13 as jul_sales, sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@14 as aug_sales, sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@15 as sep_sales, sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@16 as oct_sales, sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@17 as nov_sales, sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@18 as dec_sales, sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@19 as jan_net, sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@20 as feb_net, sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@21 as mar_net, sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@22 as apr_net, sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@23 as may_net, sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@24 as jun_net, sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@25 as jul_net, sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@26 as aug_net, sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@27 as sep_net, sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@28 as oct_net, sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@29 as nov_net, sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@30 as dec_net] + │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, d_year@6 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, DHL,BARIAN as ship_carriers, d_year@6 as year_, sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@7 as jan_sales, sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@8 as feb_sales, sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@9 as mar_sales, sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@10 as apr_sales, sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@11 as may_sales, sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@12 as jun_sales, sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@13 as jul_sales, sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@14 as aug_sales, sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@15 as sep_sales, sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@16 as oct_sales, sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@17 as nov_sales, sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@18 as dec_sales, sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@19 as jan_net, sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@20 as feb_net, sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@21 as mar_net, sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@22 as apr_net, sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@23 as may_net, sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@24 as jun_net, sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@25 as jul_net, sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@26 as aug_net, sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@27 as sep_net, sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@28 as oct_net, sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@29 as nov_net, sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@30 as dec_net] + │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, d_year@6 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sq_ft@1, w_city@2, w_county@3, w_state@4, w_country@5, ship_carriers@6, year_@7], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[w_warehouse_name@1 as w_warehouse_name, w_warehouse_sq_ft@2 as w_warehouse_sq_ft, w_city@3 as w_city, w_county@4 as w_county, w_state@5 as w_state, w_country@6 as w_country, ship_carriers@7 as ship_carriers, year_@8 as year_], aggr=[sum(x.jan_sales), sum(x.feb_sales), sum(x.mar_sales), sum(x.apr_sales), sum(x.may_sales), sum(x.jun_sales), sum(x.jul_sales), sum(x.aug_sales), sum(x.sep_sales), sum(x.oct_sales), sum(x.nov_sales), sum(x.dec_sales), sum(x.jan_sales / x.w_warehouse_sq_ft), sum(x.feb_sales / x.w_warehouse_sq_ft), sum(x.mar_sales / x.w_warehouse_sq_ft), sum(x.apr_sales / x.w_warehouse_sq_ft), sum(x.may_sales / x.w_warehouse_sq_ft), sum(x.jun_sales / x.w_warehouse_sq_ft), sum(x.jul_sales / x.w_warehouse_sq_ft), sum(x.aug_sales / x.w_warehouse_sq_ft), sum(x.sep_sales / x.w_warehouse_sq_ft), sum(x.oct_sales / x.w_warehouse_sq_ft), sum(x.nov_sales / x.w_warehouse_sq_ft), sum(x.dec_sales / x.w_warehouse_sq_ft), sum(x.jan_net), sum(x.feb_net), sum(x.mar_net), sum(x.apr_net), sum(x.may_net), sum(x.jun_net), sum(x.jul_net), sum(x.aug_net), sum(x.sep_net), sum(x.oct_net), sum(x.nov_net), sum(x.dec_net)] - │ ProjectionExec: expr=[CAST(w_warehouse_sq_ft@1 AS Float64) as __common_expr_1, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, ship_carriers@6 as ship_carriers, year_@7 as year_, jan_sales@8 as jan_sales, feb_sales@9 as feb_sales, mar_sales@10 as mar_sales, apr_sales@11 as apr_sales, may_sales@12 as may_sales, jun_sales@13 as jun_sales, jul_sales@14 as jul_sales, aug_sales@15 as aug_sales, sep_sales@16 as sep_sales, oct_sales@17 as oct_sales, nov_sales@18 as nov_sales, dec_sales@19 as dec_sales, jan_net@20 as jan_net, feb_net@21 as feb_net, mar_net@22 as mar_net, apr_net@23 as apr_net, may_net@24 as may_net, jun_net@25 as jun_net, jul_net@26 as jul_net, aug_net@27 as aug_net, sep_net@28 as sep_net, oct_net@29 as oct_net, nov_net@30 as nov_net, dec_net@31 as dec_net] - │ InterleaveExec - │ ProjectionExec: expr=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, DHL,BARIAN as ship_carriers, d_year@6 as year_, sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@7 as jan_sales, sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@8 as feb_sales, sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@9 as mar_sales, sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@10 as apr_sales, sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@11 as may_sales, sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@12 as jun_sales, sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@13 as jul_sales, sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@14 as aug_sales, sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@15 as sep_sales, sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@16 as oct_sales, sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@17 as nov_sales, sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END)@18 as dec_sales, sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@19 as jan_net, sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@20 as feb_net, sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@21 as mar_net, sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@22 as apr_net, sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@23 as may_net, sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@24 as jun_net, sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@25 as jul_net, sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@26 as aug_net, sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@27 as sep_net, sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@28 as oct_net, sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@29 as nov_net, sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)@30 as dec_net] - │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, d_year@6 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ ProjectionExec: expr=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, DHL,BARIAN as ship_carriers, d_year@6 as year_, sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@7 as jan_sales, sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@8 as feb_sales, sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@9 as mar_sales, sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@10 as apr_sales, sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@11 as may_sales, sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@12 as jun_sales, sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@13 as jul_sales, sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@14 as aug_sales, sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@15 as sep_sales, sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@16 as oct_sales, sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@17 as nov_sales, sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END)@18 as dec_sales, sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@19 as jan_net, sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@20 as feb_net, sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@21 as mar_net, sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@22 as apr_net, sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@23 as may_net, sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@24 as jun_net, sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@25 as jul_net, sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@26 as aug_net, sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@27 as sep_net, sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@28 as oct_net, sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@29 as nov_net, sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)@30 as dec_net] - │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country, d_year@6 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sq_ft@1, w_city@2, w_county@3, w_state@4, w_country@5, d_year@6], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[w_warehouse_name@15 as w_warehouse_name, w_warehouse_sq_ft@16 as w_warehouse_sq_ft, w_city@17 as w_city, w_county@18 as w_county, w_state@19 as w_state, w_country@20 as w_country, d_year@21 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) - │ ProjectionExec: expr=[d_moy@10 = 1 as __common_expr_2, d_moy@10 = 2 as __common_expr_3, d_moy@10 = 3 as __common_expr_4, d_moy@10 = 4 as __common_expr_5, d_moy@10 = 5 as __common_expr_6, d_moy@10 = 6 as __common_expr_7, d_moy@10 = 7 as __common_expr_8, d_moy@10 = 8 as __common_expr_9, d_moy@10 = 9 as __common_expr_10, d_moy@10 = 10 as __common_expr_11, d_moy@10 = 11 as __common_expr_12, d_moy@10 = 12 as __common_expr_13, ws_quantity@0 as ws_quantity, ws_ext_sales_price@1 as ws_ext_sales_price, ws_net_paid@2 as ws_net_paid, w_warehouse_name@3 as w_warehouse_name, w_warehouse_sq_ft@4 as w_warehouse_sq_ft, w_city@5 as w_city, w_county@6 as w_county, w_state@7 as w_state, w_country@8 as w_country, d_year@9 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(ship_mode.sm_ship_mode_sk AS Float64)@1, ws_ship_mode_sk@0)], projection=[ws_quantity@3, ws_ext_sales_price@4, ws_net_paid@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@12, d_moy@13] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_time_sk@0, CAST(time_dim.t_time_sk AS Float64)@1)], projection=[ws_ship_mode_sk@1, ws_quantity@2, ws_ext_sales_price@3, ws_net_paid@4, w_warehouse_name@5, w_warehouse_sq_ft@6, w_city@7, w_county@8, w_state@9, w_country@10, d_year@11, d_moy@12] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_time@1 >= 30838 AND t_time@1 <= 59638, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_time], file_type=parquet, predicate=t_time@2 >= 30838 AND t_time@2 <= 59638, pruning_predicate=t_time_null_count@1 != row_count@2 AND t_time_max@0 >= 30838 AND t_time_null_count@1 != row_count@2 AND t_time_min@3 <= 59638, required_guarantees=[] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sq_ft@1, w_city@2, w_county@3, w_state@4, w_country@5, d_year@6], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[w_warehouse_name@15 as w_warehouse_name, w_warehouse_sq_ft@16 as w_warehouse_sq_ft, w_city@17 as w_city, w_county@18 as w_county, w_state@19 as w_state, w_country@20 as w_country, d_year@21 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) + │ ProjectionExec: expr=[d_moy@10 = 1 as __common_expr_2, d_moy@10 = 2 as __common_expr_3, d_moy@10 = 3 as __common_expr_4, d_moy@10 = 4 as __common_expr_5, d_moy@10 = 5 as __common_expr_6, d_moy@10 = 6 as __common_expr_7, d_moy@10 = 7 as __common_expr_8, d_moy@10 = 8 as __common_expr_9, d_moy@10 = 9 as __common_expr_10, d_moy@10 = 10 as __common_expr_11, d_moy@10 = 11 as __common_expr_12, d_moy@10 = 12 as __common_expr_13, ws_quantity@0 as ws_quantity, ws_ext_sales_price@1 as ws_ext_sales_price, ws_net_paid@2 as ws_net_paid, w_warehouse_name@3 as w_warehouse_name, w_warehouse_sq_ft@4 as w_warehouse_sq_ft, w_city@5 as w_city, w_county@6 as w_county, w_state@7 as w_state, w_country@8 as w_country, d_year@9 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(ship_mode.sm_ship_mode_sk AS Float64)@1, ws_ship_mode_sk@0)], projection=[ws_quantity@3, ws_ext_sales_price@4, ws_net_paid@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@12, d_moy@13] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_time_sk@0, CAST(time_dim.t_time_sk AS Float64)@1)], projection=[ws_ship_mode_sk@1, ws_quantity@2, ws_ext_sales_price@3, ws_net_paid@4, w_warehouse_name@5, w_warehouse_sq_ft@6, w_city@7, w_county@8, w_state@9, w_country@10, d_year@11, d_moy@12] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_time@1 >= 30838 AND t_time@1 <= 59638, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_time], file_type=parquet, predicate=t_time@2 >= 30838 AND t_time@2 <= 59638, pruning_predicate=t_time_null_count@1 != row_count@2 AND t_time_max@0 >= 30838 AND t_time_null_count@1 != row_count@2 AND t_time_min@3 <= 59638, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[sm_ship_mode_sk@0 as sm_ship_mode_sk, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)] + │ FilterExec: sm_carrier@1 = DHL OR sm_carrier@1 = BARIAN, projection=[sm_ship_mode_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_carrier], file_type=parquet, predicate=sm_carrier@4 = DHL OR sm_carrier@4 = BARIAN, pruning_predicate=sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= DHL AND DHL <= sm_carrier_max@1 OR sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= BARIAN AND BARIAN <= sm_carrier_max@1, required_guarantees=[sm_carrier in (BARIAN, DHL)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[sm_ship_mode_sk@0 as sm_ship_mode_sk, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)] - │ FilterExec: sm_carrier@1 = DHL OR sm_carrier@1 = BARIAN, projection=[sm_ship_mode_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_carrier], file_type=parquet, predicate=sm_carrier@4 = DHL OR sm_carrier@4 = BARIAN, pruning_predicate=sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= DHL AND DHL <= sm_carrier_max@1 OR sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= BARIAN AND BARIAN <= sm_carrier_max@1, required_guarantees=[sm_carrier in (BARIAN, DHL)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@3)], projection=[ws_sold_time_sk@1, ws_ship_mode_sk@2, ws_quantity@3, ws_ext_sales_price@4, ws_net_paid@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@13, d_moy@14] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ws_sold_date_sk@6 as ws_sold_date_sk, ws_sold_time_sk@7 as ws_sold_time_sk, ws_ship_mode_sk@8 as ws_ship_mode_sk, ws_quantity@9 as ws_quantity, ws_ext_sales_price@10 as ws_ext_sales_price, ws_net_paid@11 as ws_net_paid, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@7, ws_warehouse_sk@3)], projection=[w_warehouse_name@1, w_warehouse_sq_ft@2, w_city@3, w_county@4, w_state@5, w_country@6, ws_sold_date_sk@8, ws_sold_time_sk@9, ws_ship_mode_sk@10, ws_quantity@12, ws_ext_sales_price@13, ws_net_paid@14] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_sold_time_sk, ws_ship_mode_sk, ws_warehouse_sk, ws_quantity, ws_ext_sales_price, ws_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sq_ft@1, w_city@2, w_county@3, w_state@4, w_country@5, d_year@6], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[w_warehouse_name@15 as w_warehouse_name, w_warehouse_sq_ft@16 as w_warehouse_sq_ft, w_city@17 as w_city, w_county@18 as w_county, w_state@19 as w_state, w_country@20 as w_country, d_year@21 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) - │ ProjectionExec: expr=[d_moy@10 = 1 as __common_expr_14, d_moy@10 = 2 as __common_expr_15, d_moy@10 = 3 as __common_expr_16, d_moy@10 = 4 as __common_expr_17, d_moy@10 = 5 as __common_expr_18, d_moy@10 = 6 as __common_expr_19, d_moy@10 = 7 as __common_expr_20, d_moy@10 = 8 as __common_expr_21, d_moy@10 = 9 as __common_expr_22, d_moy@10 = 10 as __common_expr_23, d_moy@10 = 11 as __common_expr_24, d_moy@10 = 12 as __common_expr_25, cs_quantity@0 as cs_quantity, cs_sales_price@1 as cs_sales_price, cs_net_paid_inc_tax@2 as cs_net_paid_inc_tax, w_warehouse_name@3 as w_warehouse_name, w_warehouse_sq_ft@4 as w_warehouse_sq_ft, w_city@5 as w_city, w_county@6 as w_county, w_state@7 as w_state, w_country@8 as w_country, d_year@9 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(ship_mode.sm_ship_mode_sk AS Float64)@1, cs_ship_mode_sk@0)], projection=[cs_quantity@3, cs_sales_price@4, cs_net_paid_inc_tax@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@12, d_moy@13] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_time_sk@0, CAST(time_dim.t_time_sk AS Float64)@1)], projection=[cs_ship_mode_sk@1, cs_quantity@2, cs_sales_price@3, cs_net_paid_inc_tax@4, w_warehouse_name@5, w_warehouse_sq_ft@6, w_city@7, w_county@8, w_state@9, w_country@10, d_year@11, d_moy@12] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_time@1 >= 30838 AND t_time@1 <= 59638, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_time], file_type=parquet, predicate=t_time@2 >= 30838 AND t_time@2 <= 59638, pruning_predicate=t_time_null_count@1 != row_count@2 AND t_time_max@0 >= 30838 AND t_time_null_count@1 != row_count@2 AND t_time_min@3 <= 59638, required_guarantees=[] + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@3)], projection=[ws_sold_time_sk@1, ws_ship_mode_sk@2, ws_quantity@3, ws_ext_sales_price@4, ws_net_paid@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@13, d_moy@14] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ ProjectionExec: expr=[ws_sold_date_sk@6 as ws_sold_date_sk, ws_sold_time_sk@7 as ws_sold_time_sk, ws_ship_mode_sk@8 as ws_ship_mode_sk, ws_quantity@9 as ws_quantity, ws_ext_sales_price@10 as ws_ext_sales_price, ws_net_paid@11 as ws_net_paid, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@7, ws_warehouse_sk@3)], projection=[w_warehouse_name@1, w_warehouse_sq_ft@2, w_city@3, w_county@4, w_state@5, w_country@6, ws_sold_date_sk@8, ws_sold_time_sk@9, ws_ship_mode_sk@10, ws_quantity@12, ws_ext_sales_price@13, ws_net_paid@14] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_sold_time_sk, ws_ship_mode_sk, ws_warehouse_sk, ws_quantity, ws_ext_sales_price, ws_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sq_ft@1, w_city@2, w_county@3, w_state@4, w_country@5, d_year@6], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[w_warehouse_name@15 as w_warehouse_name, w_warehouse_sq_ft@16 as w_warehouse_sq_ft, w_city@17 as w_city, w_county@18 as w_county, w_state@19 as w_state, w_country@20 as w_country, d_year@21 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) + │ ProjectionExec: expr=[d_moy@10 = 1 as __common_expr_14, d_moy@10 = 2 as __common_expr_15, d_moy@10 = 3 as __common_expr_16, d_moy@10 = 4 as __common_expr_17, d_moy@10 = 5 as __common_expr_18, d_moy@10 = 6 as __common_expr_19, d_moy@10 = 7 as __common_expr_20, d_moy@10 = 8 as __common_expr_21, d_moy@10 = 9 as __common_expr_22, d_moy@10 = 10 as __common_expr_23, d_moy@10 = 11 as __common_expr_24, d_moy@10 = 12 as __common_expr_25, cs_quantity@0 as cs_quantity, cs_sales_price@1 as cs_sales_price, cs_net_paid_inc_tax@2 as cs_net_paid_inc_tax, w_warehouse_name@3 as w_warehouse_name, w_warehouse_sq_ft@4 as w_warehouse_sq_ft, w_city@5 as w_city, w_county@6 as w_county, w_state@7 as w_state, w_country@8 as w_country, d_year@9 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(ship_mode.sm_ship_mode_sk AS Float64)@1, cs_ship_mode_sk@0)], projection=[cs_quantity@3, cs_sales_price@4, cs_net_paid_inc_tax@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@12, d_moy@13] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_time_sk@0, CAST(time_dim.t_time_sk AS Float64)@1)], projection=[cs_ship_mode_sk@1, cs_quantity@2, cs_sales_price@3, cs_net_paid_inc_tax@4, w_warehouse_name@5, w_warehouse_sq_ft@6, w_city@7, w_county@8, w_state@9, w_country@10, d_year@11, d_moy@12] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_time@1 >= 30838 AND t_time@1 <= 59638, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_time], file_type=parquet, predicate=t_time@2 >= 30838 AND t_time@2 <= 59638, pruning_predicate=t_time_null_count@1 != row_count@2 AND t_time_max@0 >= 30838 AND t_time_null_count@1 != row_count@2 AND t_time_min@3 <= 59638, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[sm_ship_mode_sk@0 as sm_ship_mode_sk, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)] + │ FilterExec: sm_carrier@1 = DHL OR sm_carrier@1 = BARIAN, projection=[sm_ship_mode_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_carrier], file_type=parquet, predicate=sm_carrier@4 = DHL OR sm_carrier@4 = BARIAN, pruning_predicate=sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= DHL AND DHL <= sm_carrier_max@1 OR sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= BARIAN AND BARIAN <= sm_carrier_max@1, required_guarantees=[sm_carrier in (BARIAN, DHL)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[sm_ship_mode_sk@0 as sm_ship_mode_sk, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)] - │ FilterExec: sm_carrier@1 = DHL OR sm_carrier@1 = BARIAN, projection=[sm_ship_mode_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_carrier], file_type=parquet, predicate=sm_carrier@4 = DHL OR sm_carrier@4 = BARIAN, pruning_predicate=sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= DHL AND DHL <= sm_carrier_max@1 OR sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= BARIAN AND BARIAN <= sm_carrier_max@1, required_guarantees=[sm_carrier in (BARIAN, DHL)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@3)], projection=[cs_sold_time_sk@1, cs_ship_mode_sk@2, cs_quantity@3, cs_sales_price@4, cs_net_paid_inc_tax@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@13, d_moy@14] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@3)], projection=[cs_sold_time_sk@1, cs_ship_mode_sk@2, cs_quantity@3, cs_sales_price@4, cs_net_paid_inc_tax@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@13, d_moy@14] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ ProjectionExec: expr=[cs_sold_date_sk@6 as cs_sold_date_sk, cs_sold_time_sk@7 as cs_sold_time_sk, cs_ship_mode_sk@8 as cs_ship_mode_sk, cs_quantity@9 as cs_quantity, cs_sales_price@10 as cs_sales_price, cs_net_paid_inc_tax@11 as cs_net_paid_inc_tax, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@7, cs_warehouse_sk@3)], projection=[w_warehouse_name@1, w_warehouse_sq_ft@2, w_city@3, w_county@4, w_state@5, w_country@6, cs_sold_date_sk@8, cs_sold_time_sk@9, cs_ship_mode_sk@10, cs_quantity@12, cs_sales_price@13, cs_net_paid_inc_tax@14] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_sold_time_sk, cs_ship_mode_sk, cs_warehouse_sk, cs_quantity, cs_sales_price, cs_net_paid_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[cs_sold_date_sk@6 as cs_sold_date_sk, cs_sold_time_sk@7 as cs_sold_time_sk, cs_ship_mode_sk@8 as cs_ship_mode_sk, cs_quantity@9 as cs_quantity, cs_sales_price@10 as cs_sales_price, cs_net_paid_inc_tax@11 as cs_net_paid_inc_tax, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@7, cs_warehouse_sk@3)], projection=[w_warehouse_name@1, w_warehouse_sq_ft@2, w_city@3, w_county@4, w_state@5, w_country@6, cs_sold_date_sk@8, cs_sold_time_sk@9, cs_ship_mode_sk@10, cs_quantity@12, cs_sales_price@13, cs_net_paid_inc_tax@14] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_sold_time_sk, cs_ship_mode_sk, cs_warehouse_sk, cs_quantity, cs_sales_price, cs_net_paid_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── "); Ok(()) } @@ -7266,51 +6934,47 @@ mod tests { │ FilterExec: rank() PARTITION BY [dw1.i_category] ORDER BY [dw1.sumsales DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@9 <= 100 │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [dw1.i_category] ORDER BY [dw1.sumsales DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [dw1.i_category] ORDER BY [dw1.sumsales DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, sumsales@8 DESC], preserve_partitioning=[true] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_category@0], 3), input_partitions=3 + │ ProjectionExec: expr=[i_category@0 as i_category, i_class@1 as i_class, i_brand@2 as i_brand, i_product_name@3 as i_product_name, d_year@4 as d_year, d_qoy@5 as d_qoy, d_moy@6 as d_moy, s_store_id@7 as s_store_id, sum(coalesce(store_sales.ss_sales_price * store_sales.ss_quantity,Int64(0)))@9 as sumsales] + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class, i_brand@2 as i_brand, i_product_name@3 as i_product_name, d_year@4 as d_year, d_qoy@5 as d_qoy, d_moy@6 as d_moy, s_store_id@7 as s_store_id, __grouping_id@8 as __grouping_id], aggr=[sum(coalesce(store_sales.ss_sales_price * store_sales.ss_quantity,Int64(0)))] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0], 3), input_partitions=3 - │ ProjectionExec: expr=[i_category@0 as i_category, i_class@1 as i_class, i_brand@2 as i_brand, i_product_name@3 as i_product_name, d_year@4 as d_year, d_qoy@5 as d_qoy, d_moy@6 as d_moy, s_store_id@7 as s_store_id, sum(coalesce(store_sales.ss_sales_price * store_sales.ss_quantity,Int64(0)))@9 as sumsales] - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class, i_brand@2 as i_brand, i_product_name@3 as i_product_name, d_year@4 as d_year, d_qoy@5 as d_qoy, d_moy@6 as d_moy, s_store_id@7 as s_store_id, __grouping_id@8 as __grouping_id], aggr=[sum(coalesce(store_sales.ss_sales_price * store_sales.ss_quantity,Int64(0)))] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1, i_brand@2, i_product_name@3, d_year@4, d_qoy@5, d_moy@6, s_store_id@7, __grouping_id@8], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[(NULL as i_category, NULL as i_class, NULL as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, NULL as i_class, NULL as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, NULL as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, d_qoy@3 as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, d_qoy@3 as d_qoy, d_moy@2 as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, d_qoy@3 as d_qoy, d_moy@2 as d_moy, s_store_id@4 as s_store_id)], aggr=[sum(coalesce(store_sales.ss_sales_price * store_sales.ss_quantity,Int64(0)))] - │ ProjectionExec: expr=[CAST(ss_sales_price@1 AS Float64) * ss_quantity@0 as __common_expr_1, d_year@2 as d_year, d_moy@3 as d_moy, d_qoy@4 as d_qoy, s_store_id@5 as s_store_id, i_brand@6 as i_brand, i_class@7 as i_class, i_category@8 as i_category, i_product_name@9 as i_product_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_sales_price@2, d_year@3, d_moy@4, d_qoy@5, s_store_id@6, i_brand@8, i_class@9, i_category@10, i_product_name@11] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_product_name], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1, i_brand@2, i_product_name@3, d_year@4, d_qoy@5, d_moy@6, s_store_id@7, __grouping_id@8], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[(NULL as i_category, NULL as i_class, NULL as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, NULL as i_class, NULL as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, NULL as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, d_qoy@3 as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, d_qoy@3 as d_qoy, d_moy@2 as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, d_qoy@3 as d_qoy, d_moy@2 as d_moy, s_store_id@4 as s_store_id)], aggr=[sum(coalesce(store_sales.ss_sales_price * store_sales.ss_quantity,Int64(0)))] + │ ProjectionExec: expr=[CAST(ss_sales_price@1 AS Float64) * ss_quantity@0 as __common_expr_1, d_year@2 as d_year, d_moy@3 as d_moy, d_qoy@4 as d_qoy, s_store_id@5 as s_store_id, i_brand@6 as i_brand, i_class@7 as i_class, i_category@8 as i_category, i_product_name@9 as i_product_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_sales_price@2, d_year@3, d_moy@4, d_qoy@5, s_store_id@6, i_brand@8, i_class@9, i_category@10, i_product_name@11] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_product_name], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, ss_sales_price@3 as ss_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, d_qoy@6 as d_qoy, s_store_id@0 as s_store_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, ss_item_sk@3, ss_quantity@5, ss_sales_price@6, d_year@7, d_moy@8, d_qoy@9] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, ss_sales_price@6 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy, d_qoy@2 as d_qoy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@4, ss_sold_date_sk@0)], projection=[d_year@1, d_moy@2, d_qoy@3, ss_item_sk@6, ss_store_sk@7, ss_quantity@8, ss_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, ss_sales_price@3 as ss_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, d_qoy@6 as d_qoy, s_store_id@0 as s_store_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, ss_item_sk@3, ss_quantity@5, ss_sales_price@6, d_year@7, d_moy@8, d_qoy@9] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, ss_sales_price@6 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy, d_qoy@2 as d_qoy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@4, ss_sold_date_sk@0)], projection=[d_year@1, d_moy@2, d_qoy@3, ss_item_sk@6, ss_store_sk@7, ss_quantity@8, ss_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, d_qoy@3 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0, d_year@2, d_moy@3, d_qoy@4] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq, d_year, d_moy, d_qoy], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, d_qoy@3 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0, d_year@2, d_moy@3, d_qoy@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq, d_year, d_moy, d_qoy], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] - └────────────────────────────────────────────────── "#); Ok(()) } @@ -7336,61 +7000,59 @@ mod tests { │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@4)], projection=[ss_ticket_number@0, bought_city@2, extended_price@3, list_price@4, extended_tax@5, c_current_addr_sk@7, c_first_name@8, c_last_name@9] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] + ┌───── Stage 6 ── Tasks: t0:[p0..p5] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ca_city@3 as bought_city, sum(store_sales.ss_ext_sales_price)@4 as extended_price, sum(store_sales.ss_ext_list_price)@5 as list_price, sum(store_sales.ss_ext_tax)@6 as extended_tax] │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ss_addr_sk@2 as ss_addr_sk, ca_city@3 as ca_city], aggr=[sum(store_sales.ss_ext_sales_price), sum(store_sales.ss_ext_list_price), sum(store_sales.ss_ext_tax)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1, ss_addr_sk@2, ca_city@3], 6), input_partitions=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1, ss_addr_sk@2, ca_city@3], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@2 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk, ss_addr_sk@1 as ss_addr_sk, ca_city@6 as ca_city], aggr=[sum(store_sales.ss_ext_sales_price), sum(store_sales.ss_ext_list_price), sum(store_sales.ss_ext_tax)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_customer_sk@0, ss_addr_sk@1, ss_ticket_number@2, ss_ext_sales_price@3, ss_ext_list_price@4, ss_ext_tax@5, ca_city@7] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_city@1 as ca_city, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_city], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_addr_sk@4, ss_ticket_number@5, ss_ext_sales_price@6, ss_ext_list_price@7, ss_ext_tax@8] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_ticket_number@6, ss_ext_sales_price@7, ss_ext_list_price@8, ss_ext_tax@9] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_store_sk@6, ss_ticket_number@7, ss_ext_sales_price@8, ss_ext_list_price@9, ss_ext_tax@10] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_ticket_number, ss_ext_sales_price, ss_ext_list_price, ss_ext_tax], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 OR hd_vehicle_count@2 = 3, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 OR hd_vehicle_count@4 = 3, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 OR hd_vehicle_count_null_count@6 != row_count@3 AND hd_vehicle_count_min@4 <= 3 AND 3 <= hd_vehicle_count_max@5, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 OR hd_vehicle_count@4 = 3, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 OR hd_vehicle_count_null_count@6 != row_count@3 AND hd_vehicle_count_min@4 <= 3 AND 3 <= hd_vehicle_count_max@5, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_city@1 = Fairview OR s_city@1 = Midway, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_city], file_type=parquet, predicate=s_city@22 = Fairview OR s_city@22 = Midway, pruning_predicate=s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Fairview AND Fairview <= s_city_max@1 OR s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Midway AND Midway <= s_city_max@1, required_guarantees=[s_city in (Fairview, Midway)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_city], file_type=parquet, predicate=s_city@22 = Fairview OR s_city@22 = Midway, pruning_predicate=s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Fairview AND Fairview <= s_city_max@1 OR s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Midway AND Midway <= s_city_max@1, required_guarantees=[s_city in (Fairview, Midway)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_dom@2 >= 1 AND d_dom@2 <= 2 AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -7406,98 +7068,94 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [cd_gender@0 ASC NULLS LAST, cd_marital_status@1 ASC NULLS LAST, cd_education_status@2 ASC NULLS LAST, cd_purchase_estimate@4 ASC NULLS LAST, cd_credit_rating@6 ASC NULLS LAST], fetch=100 - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[cd_gender@0 ASC NULLS LAST, cd_marital_status@1 ASC NULLS LAST, cd_education_status@2 ASC NULLS LAST, cd_purchase_estimate@4 ASC NULLS LAST, cd_credit_rating@6 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, count(Int64(1))@5 as cnt1, cd_purchase_estimate@3 as cd_purchase_estimate, count(Int64(1))@5 as cnt2, cd_credit_rating@4 as cd_credit_rating, count(Int64(1))@5 as cnt3] │ AggregateExec: mode=FinalPartitioned, gby=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, cd_purchase_estimate@3 as cd_purchase_estimate, cd_credit_rating@4 as cd_credit_rating], aggr=[count(Int64(1))] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cd_gender@0, cd_marital_status@1, cd_education_status@2, cd_purchase_estimate@3, cd_credit_rating@4], 6), input_partitions=2 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cd_gender@0, cd_marital_status@1, cd_education_status@2, cd_purchase_estimate@3, cd_credit_rating@4], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, cd_purchase_estimate@3 as cd_purchase_estimate, cd_credit_rating@4 as cd_credit_rating], aggr=[count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(cs_ship_customer_sk@0, CAST(c.c_customer_sk AS Float64)@6)], projection=[cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(ws_bill_customer_sk@0, CAST(c.c_customer_sk AS Float64)@6)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(ss_customer_sk@0, CAST(c.c_customer_sk AS Float64)@6)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@6)], projection=[c_customer_sk@0, cd_gender@3, cd_marital_status@4, cd_education_status@5, cd_purchase_estimate@6, cd_credit_rating@7] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status, cd_purchase_estimate, cd_credit_rating, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 AND d_moy@2 >= 4 AND d_moy@2 <= 6, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_sk@3] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 AND d_moy@2 >= 4 AND d_moy@2 <= 6, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 6 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 AND d_moy@2 >= 4 AND d_moy@2 <= 6, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[c_customer_sk@1, c_current_cdemo_sk@2] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] + ┌───── Stage 7 ── Tasks: t0:[p0..p5] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ FilterExec: ca_state@1 = KY OR ca_state@1 = GA OR ca_state@1 = NM, projection=[ca_address_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet, predicate=ca_state@8 = KY OR ca_state@8 = GA OR ca_state@8 = NM, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NM AND NM <= ca_state_max@1, required_guarantees=[ca_state in (GA, KY, NM)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet, predicate=ca_state@8 = KY OR ca_state@8 = GA OR ca_state@8 = NM, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NM AND NM <= ca_state_max@1, required_guarantees=[ca_state in (GA, KY, NM)] └────────────────────────────────────────────────── "); Ok(()) @@ -7515,21 +7173,21 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ext_price@4 DESC, brand_id@0 ASC, t_hour@2 ASC] - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: expr=[ext_price@4 DESC, brand_id@0 ASC, t_hour@2 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, t_hour@2 as t_hour, t_minute@3 as t_minute, sum(tmp.ext_price)@4 as ext_price] │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id, t_hour@2 as t_hour, t_minute@3 as t_minute], aggr=[sum(tmp.ext_price)] │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1, t_hour@2, t_minute@3], 6), input_partitions=3 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1, t_hour@2, t_minute@3], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_brand@1 as i_brand, i_brand_id@0 as i_brand_id, t_hour@3 as t_hour, t_minute@4 as t_minute], aggr=[sum(tmp.ext_price)] │ ProjectionExec: expr=[i_brand_id@2 as i_brand_id, i_brand@3 as i_brand, ext_price@4 as ext_price, t_hour@0 as t_hour, t_minute@1 as t_minute] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@3, time_sk@3)], projection=[t_hour@1, t_minute@2, i_brand_id@4, i_brand@5, ext_price@6] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[i_brand_id@2 as i_brand_id, i_brand@3 as i_brand, ext_price@0 as ext_price, time_sk@1 as time_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(sold_item_sk@1, i_item_sk@0)], projection=[ext_price@0, time_sk@2, i_brand_id@4, i_brand@5] │ CoalescePartitionsExec @@ -7539,13 +7197,12 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_brand, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 1 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 1 AND 1 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (1)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p12..p23] + ┌───── Stage 1 ── Tasks: t0:[p0..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, t_hour@1 as t_hour, t_minute@2 as t_minute, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_meal_time@3 = breakfast OR t_meal_time@3 = dinner, projection=[t_time_sk@0, t_hour@1, t_minute@2] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute, t_meal_time], file_type=parquet, predicate=t_meal_time@9 = breakfast OR t_meal_time@9 = dinner, pruning_predicate=t_meal_time_null_count@2 != row_count@3 AND t_meal_time_min@0 <= breakfast AND breakfast <= t_meal_time_max@1 OR t_meal_time_null_count@2 != row_count@3 AND t_meal_time_min@0 <= dinner AND dinner <= t_meal_time_max@1, required_guarantees=[t_meal_time in (breakfast, dinner)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute, t_meal_time], file_type=parquet, predicate=t_meal_time@9 = breakfast OR t_meal_time@9 = dinner, pruning_predicate=t_meal_time_null_count@2 != row_count@3 AND t_meal_time_min@0 <= breakfast AND breakfast <= t_meal_time_max@1 OR t_meal_time_null_count@2 != row_count@3 AND t_meal_time_min@0 <= dinner AND dinner <= t_meal_time_max@1, required_guarantees=[t_meal_time in (breakfast, dinner)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 @@ -7571,17 +7228,16 @@ mod tests { │ ProjectionExec: expr=[ss_ext_sales_price@2 as ext_price, ss_item_sk@1 as sold_item_sk, ss_sold_time_sk@0 as time_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_sold_time_sk@3, ss_item_sk@4, ss_ext_sales_price@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_sold_time_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_moy@2 = 11 AND d_year@1 = 1999, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] └────────────────────────────────────────────────── "); Ok(()) @@ -7599,104 +7255,100 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_desc@1 as i_item_desc, w_warehouse_name@0 as w_warehouse_name, d_week_seq@2 as d_week_seq], aggr=[sum(CASE WHEN promotion.p_promo_sk IS NULL THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN promotion.p_promo_sk IS NOT NULL THEN Int64(1) ELSE Int64(0) END), count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(cs_item_sk@0, cr_item_sk@0), (cs_order_number@1, cr_order_number@1)], projection=[w_warehouse_name@2, i_item_desc@3, d_week_seq@4, p_promo_sk@5] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_item_sk, cr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, w_warehouse_name@3 as w_warehouse_name, i_item_desc@4 as i_item_desc, d_week_seq@5 as d_week_seq, p_promo_sk@0 as p_promo_sk] │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(CAST(promotion.p_promo_sk AS Float64)@1, cs_promo_sk@1)], projection=[p_promo_sk@0, cs_item_sk@2, cs_order_number@4, w_warehouse_name@5, i_item_desc@6, d_week_seq@7] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_date_sk@0, CAST(d3.d_date_sk AS Float64)@2)], filter=d_date@1 > d_date@0 + IntervalMonthDayNano { months: 0, days: 5, nanoseconds: 0 }, projection=[cs_item_sk@1, cs_promo_sk@2, cs_order_number@3, w_warehouse_name@4, i_item_desc@5, d_week_seq@7] │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=4 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p3] + │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] t3:[p18..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0), (d_week_seq@8, d_week_seq@1)], projection=[cs_ship_date_sk@0, cs_item_sk@1, cs_promo_sk@2, cs_order_number@3, w_warehouse_name@5, i_item_desc@6, d_date@7, d_week_seq@8] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[cs_ship_date_sk@2 as cs_ship_date_sk, cs_item_sk@3 as cs_item_sk, cs_promo_sk@4 as cs_promo_sk, cs_order_number@5 as cs_order_number, inv_date_sk@6 as inv_date_sk, w_warehouse_name@7 as w_warehouse_name, i_item_desc@8 as i_item_desc, d_date@0 as d_date, d_week_seq@1 as d_week_seq] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@3, cs_sold_date_sk@0)], projection=[d_date@1, d_week_seq@2, cs_ship_date_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9, w_warehouse_name@10, i_item_desc@11] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, cs_bill_hdemo_sk@2)], projection=[cs_sold_date_sk@2, cs_ship_date_sk@3, cs_item_sk@5, cs_promo_sk@6, cs_order_number@7, inv_date_sk@8, w_warehouse_name@9, i_item_desc@10] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, cs_bill_cdemo_sk@2)], projection=[cs_sold_date_sk@2, cs_ship_date_sk@3, cs_bill_hdemo_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9, w_warehouse_name@10, i_item_desc@11] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, d_week_seq@2 as d_week_seq, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] │ FilterExec: d_year@3 = 1999, projection=[d_date_sk@0, d_date@1, d_week_seq@2] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_week_seq, d_year], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_buy_potential@1 = >10000, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential], file_type=parquet, predicate=hd_buy_potential@2 = >10000, pruning_predicate=hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1, required_guarantees=[hd_buy_potential in (>10000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential], file_type=parquet, predicate=hd_buy_potential@2 = >10000, pruning_predicate=hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1, required_guarantees=[hd_buy_potential in (>10000)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 12), input_partitions=2 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] │ FilterExec: cd_marital_status@1 = D, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status], file_type=parquet, predicate=cd_marital_status@2 = D, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= D AND D <= cd_marital_status_max@1, required_guarantees=[cd_marital_status in (D)] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@2], 9), input_partitions=3 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@2], 12), input_partitions=3 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ship_date_sk@2 as cs_ship_date_sk, cs_bill_cdemo_sk@3 as cs_bill_cdemo_sk, cs_bill_hdemo_sk@4 as cs_bill_hdemo_sk, cs_item_sk@5 as cs_item_sk, cs_promo_sk@6 as cs_promo_sk, cs_order_number@7 as cs_order_number, inv_date_sk@8 as inv_date_sk, w_warehouse_name@9 as w_warehouse_name, i_item_desc@0 as i_item_desc] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@4)], projection=[i_item_desc@1, cs_sold_date_sk@2, cs_ship_date_sk@3, cs_bill_cdemo_sk@4, cs_bill_hdemo_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9, w_warehouse_name@10] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ship_date_sk@2 as cs_ship_date_sk, cs_bill_cdemo_sk@3 as cs_bill_cdemo_sk, cs_bill_hdemo_sk@4 as cs_bill_hdemo_sk, cs_item_sk@5 as cs_item_sk, cs_promo_sk@6 as cs_promo_sk, cs_order_number@7 as cs_order_number, inv_date_sk@8 as inv_date_sk, w_warehouse_name@0 as w_warehouse_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@8)], projection=[w_warehouse_name@1, cs_sold_date_sk@2, cs_ship_date_sk@3, cs_bill_cdemo_sk@4, cs_bill_hdemo_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_item_sk@4, inv_item_sk@1)], filter=inv_quantity_on_hand@1 < cs_quantity@0, projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, cs_bill_cdemo_sk@2, cs_bill_hdemo_sk@3, cs_item_sk@4, cs_promo_sk@5, cs_order_number@6, inv_date_sk@8, inv_warehouse_sk@10] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc], file_type=parquet + ┌───── Stage 5 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_item_sk@4], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_item_sk@4], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_date_sk, cs_bill_cdemo_sk, cs_bill_hdemo_sk, cs_item_sk, cs_promo_sk, cs_order_number, cs_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([inv_item_sk@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([inv_item_sk@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -7708,59 +7360,57 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [cnt@5 DESC, c_last_name@0 ASC NULLS LAST] - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] │ SortExec: expr=[cnt@5 DESC, c_last_name@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[c_last_name@4 as c_last_name, c_first_name@3 as c_first_name, c_salutation@2 as c_salutation, c_preferred_cust_flag@5 as c_preferred_cust_flag, ss_ticket_number@0 as ss_ticket_number, cnt@1 as cnt] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_ticket_number@0, cnt@2, c_salutation@4, c_first_name@5, c_last_name@6, c_preferred_cust_flag@7] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_salutation@1 as c_salutation, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, count(Int64(1))@2 as cnt] │ FilterExec: count(Int64(1))@2 >= 1 AND count(Int64(1))@2 <= 5 │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk], aggr=[count(Int64(1))] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1], 6), input_partitions=2 + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@1 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk], aggr=[count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_ticket_number@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@2)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_ticket_number@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_store_sk@5, ss_ticket_number@6] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_hdemo_sk, ss_store_sk, ss_ticket_number], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: (hd_buy_potential@1 = Unknown OR hd_buy_potential@1 = >10000) AND hd_vehicle_count@3 > 0 AND CASE WHEN hd_vehicle_count@3 > 0 THEN CAST(hd_dep_count@2 AS Float64) / CAST(hd_vehicle_count@3 AS Float64) END > 1, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=(hd_buy_potential@2 = Unknown OR hd_buy_potential@2 = >10000) AND hd_vehicle_count@4 > 0 AND CASE WHEN hd_vehicle_count@4 > 0 THEN CAST(hd_dep_count@3 AS Float64) / CAST(hd_vehicle_count@4 AS Float64) END > 1, pruning_predicate=(hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknown AND Unknown <= hd_buy_potential_max@1 OR hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1) AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 0, required_guarantees=[hd_buy_potential in (>10000, Unknown)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=(hd_buy_potential@2 = Unknown OR hd_buy_potential@2 = >10000) AND hd_vehicle_count@4 > 0 AND CASE WHEN hd_vehicle_count@4 > 0 THEN CAST(hd_dep_count@3 AS Float64) / CAST(hd_vehicle_count@4 AS Float64) END > 1, pruning_predicate=(hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknown AND Unknown <= hd_buy_potential_max@1 OR hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1) AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 0, required_guarantees=[hd_buy_potential in (>10000, Unknown)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_county@1 IN (SET) ([Orange County, Bronx County, Franklin Parish, Williamson County]), projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_county], file_type=parquet, predicate=s_county@23 IN (SET) ([Orange County, Bronx County, Franklin Parish, Williamson County]), pruning_predicate=s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Orange County AND Orange County <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Bronx County AND Bronx County <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Franklin Parish AND Franklin Parish <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Williamson County AND Williamson County <= s_county_max@1, required_guarantees=[s_county in (Bronx County, Franklin Parish, Orange County, Williamson County)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_county], file_type=parquet, predicate=s_county@23 IN (SET) ([Orange County, Bronx County, Franklin Parish, Williamson County]), pruning_predicate=s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Orange County AND Orange County <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Bronx County AND Bronx County <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Franklin Parish AND Franklin Parish <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Williamson County AND Williamson County <= s_county_max@1, required_guarantees=[s_county in (Bronx County, Franklin Parish, Orange County, Williamson County)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_dom@2 >= 1 AND d_dom@2 <= 2 AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -7776,156 +7426,152 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [customer_id@0 ASC], fetch=100 - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC], preserve_partitioning=[true] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),17,2 THEN year_total@3 / year_total@2 END > CASE WHEN year_total@0 > Some(0),17,2 THEN year_total@1 / year_total@0 END, projection=[customer_id@2, customer_first_name@3, customer_last_name@4] │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_net_paid)@4 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(web_sales.ws_net_paid)], ordering_mode=PartiallySorted([3]) - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 12 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[customer_id@1 as customer_id, year_total@2 as year_total, customer_id@3 as customer_id, customer_first_name@4 as customer_first_name, customer_last_name@5 as customer_last_name, year_total@6 as year_total, year_total@0 as year_total] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, year_total@3, customer_id@4, customer_first_name@5, customer_last_name@6, year_total@7] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, sum(store_sales.ss_net_paid)@4 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_net_paid)@1 as year_total] │ FilterExec: sum(web_sales.ws_net_paid)@1 > Some(0),17,2 │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(web_sales.ws_net_paid)@4 as sum(web_sales.ws_net_paid)] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(web_sales.ws_net_paid)], ordering_mode=PartiallySorted([3]) - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 6), input_partitions=2 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@4 as d_year], aggr=[sum(web_sales.ws_net_paid)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, ws_net_paid@4 as ws_net_paid, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@3)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, ws_net_paid@7] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ws_sold_date_sk@5, ws_net_paid@7] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(store_sales.ss_net_paid)@1 as year_total] │ FilterExec: sum(store_sales.ss_net_paid)@1 > Some(0),17,2 │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(store_sales.ss_net_paid)@4 as sum(store_sales.ss_net_paid)] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 6), input_partitions=2 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@4 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, ss_net_paid@4 as ss_net_paid, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@3)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, ss_net_paid@7] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ss_sold_date_sk@5, ss_net_paid@7] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 6), input_partitions=2 + ┌───── Stage 11 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@4 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, ss_net_paid@4 as ss_net_paid, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@3)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, ss_net_paid@7] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ss_sold_date_sk@5, ss_net_paid@7] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2002 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 6), input_partitions=2 + ┌───── Stage 15 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@4 as d_year], aggr=[sum(web_sales.ws_net_paid)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, ws_net_paid@4 as ws_net_paid, d_year@0 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@3)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, ws_net_paid@7] │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ws_sold_date_sk@5, ws_net_paid@7] │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 13 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2002 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 14 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── "); @@ -7944,233 +7590,215 @@ mod tests { │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 │ ProjectionExec: expr=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sum(sales_detail.sales_cnt)@5 as sales_cnt, sum(sales_detail.sales_amt)@6 as sales_amt] │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id], aggr=[sum(sales_detail.sales_cnt), sum(sales_detail.sales_amt)], ordering_mode=PartiallySorted([0]) - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id], aggr=[sum(sales_detail.sales_cnt), sum(sales_detail.sales_amt)], ordering_mode=PartiallySorted([0]) + │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sales_cnt@5 as sales_cnt, sales_amt@6 as sales_amt], aggr=[], ordering_mode=PartiallySorted([0]) + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sum(sales_detail.sales_cnt)@5 as sales_cnt, sum(sales_detail.sales_amt)@6 as sales_amt] │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id], aggr=[sum(sales_detail.sales_cnt), sum(sales_detail.sales_amt)], ordering_mode=PartiallySorted([0]) - │ [Stage 22] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id], aggr=[sum(sales_detail.sales_cnt), sum(sales_detail.sales_amt)], ordering_mode=PartiallySorted([0]) + │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sales_cnt@5 as sales_cnt, sales_amt@6 as sales_amt], aggr=[], ordering_mode=PartiallySorted([0]) + │ [Stage 20] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id], aggr=[sum(sales_detail.sales_cnt), sum(sales_detail.sales_amt)], ordering_mode=PartiallySorted([0]) - │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sales_cnt@5 as sales_cnt, sales_amt@6 as sales_amt], aggr=[], ordering_mode=PartiallySorted([0]) - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, sales_cnt@5, sales_amt@6], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sales_cnt@5 as sales_cnt, sales_amt@6 as sales_amt], aggr=[], ordering_mode=PartiallySorted([0]) - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] - │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, cs_quantity@1 - CASE WHEN cr_return_quantity@8 IS NOT NULL THEN cr_return_quantity@8 ELSE 0 END as sales_cnt, cs_ext_sales_price@2 - CASE WHEN __common_expr_1@0 IS NOT NULL THEN __common_expr_1@0 ELSE Some(0),30,15 END as sales_amt] - │ ProjectionExec: expr=[CAST(cr_return_amount@8 AS Decimal128(30, 15)) as __common_expr_1, cs_quantity@0 as cs_quantity, cs_ext_sales_price@1 as cs_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, cr_return_quantity@7 as cr_return_quantity] - │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(cs_order_number@1, cr_order_number@1), (cs_item_sk@0, cr_item_sk@0)], projection=[cs_quantity@2, cs_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, cr_return_quantity@11, cr_return_amount@12] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_item_sk, cr_order_number, cr_return_quantity, cr_return_amount], file_type=parquet - │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ss_quantity@1 - CASE WHEN sr_return_quantity@8 IS NOT NULL THEN sr_return_quantity@8 ELSE 0 END as sales_cnt, ss_ext_sales_price@2 - CASE WHEN __common_expr_2@0 IS NOT NULL THEN __common_expr_2@0 ELSE Some(0),30,15 END as sales_amt] - │ ProjectionExec: expr=[CAST(sr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_2, ss_quantity@0 as ss_quantity, ss_ext_sales_price@1 as ss_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, sr_return_quantity@7 as sr_return_quantity] - │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ss_ticket_number@1, sr_ticket_number@1), (ss_item_sk@0, sr_item_sk@0)], projection=[ss_quantity@2, ss_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, sr_return_quantity@11, sr_return_amt@12] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-3.parquet:..]]}, projection=[sr_item_sk, sr_ticket_number, sr_return_quantity, sr_return_amt], file_type=parquet - │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ws_quantity@1 - CASE WHEN wr_return_quantity@8 IS NOT NULL THEN wr_return_quantity@8 ELSE 0 END as sales_cnt, ws_ext_sales_price@2 - CASE WHEN __common_expr_3@0 IS NOT NULL THEN __common_expr_3@0 ELSE Some(0),30,15 END as sales_amt] - │ ProjectionExec: expr=[CAST(wr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_3, ws_quantity@0 as ws_quantity, ws_ext_sales_price@1 as ws_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, wr_return_quantity@7 as wr_return_quantity] - │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ws_order_number@1, wr_order_number@1), (ws_item_sk@0, wr_item_sk@0)], projection=[ws_quantity@2, ws_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, wr_return_quantity@11, wr_return_amt@12] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number, wr_return_quantity, wr_return_amt], file_type=parquet + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, sales_cnt@5, sales_amt@6], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sales_cnt@5 as sales_cnt, sales_amt@6 as sales_amt], aggr=[], ordering_mode=PartiallySorted([0]) + │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] t3:[] + │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, cs_quantity@1 - CASE WHEN cr_return_quantity@8 IS NOT NULL THEN cr_return_quantity@8 ELSE 0 END as sales_cnt, cs_ext_sales_price@2 - CASE WHEN __common_expr_1@0 IS NOT NULL THEN __common_expr_1@0 ELSE Some(0),30,15 END as sales_amt] + │ ProjectionExec: expr=[CAST(cr_return_amount@8 AS Decimal128(30, 15)) as __common_expr_1, cs_quantity@0 as cs_quantity, cs_ext_sales_price@1 as cs_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, cr_return_quantity@7 as cr_return_quantity] + │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(cs_order_number@1, cr_order_number@1), (cs_item_sk@0, cr_item_sk@0)], projection=[cs_quantity@2, cs_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, cr_return_quantity@11, cr_return_amount@12] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_item_sk, cr_order_number, cr_return_quantity, cr_return_amount], file_type=parquet + │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ss_quantity@1 - CASE WHEN sr_return_quantity@8 IS NOT NULL THEN sr_return_quantity@8 ELSE 0 END as sales_cnt, ss_ext_sales_price@2 - CASE WHEN __common_expr_2@0 IS NOT NULL THEN __common_expr_2@0 ELSE Some(0),30,15 END as sales_amt] + │ ProjectionExec: expr=[CAST(sr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_2, ss_quantity@0 as ss_quantity, ss_ext_sales_price@1 as ss_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, sr_return_quantity@7 as sr_return_quantity] + │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ss_ticket_number@1, sr_ticket_number@1), (ss_item_sk@0, sr_item_sk@0)], projection=[ss_quantity@2, ss_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, sr_return_quantity@11, sr_return_amt@12] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-3.parquet:..]]}, projection=[sr_item_sk, sr_ticket_number, sr_return_quantity, sr_return_amt], file_type=parquet + │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ws_quantity@1 - CASE WHEN wr_return_quantity@8 IS NOT NULL THEN wr_return_quantity@8 ELSE 0 END as sales_cnt, ws_ext_sales_price@2 - CASE WHEN __common_expr_3@0 IS NOT NULL THEN __common_expr_3@0 ELSE Some(0),30,15 END as sales_amt] + │ ProjectionExec: expr=[CAST(wr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_3, ws_quantity@0 as ws_quantity, ws_ext_sales_price@1 as ws_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, wr_return_quantity@7 as wr_return_quantity] + │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ws_order_number@1, wr_order_number@1), (ws_item_sk@0, wr_item_sk@0)], projection=[ws_quantity@2, ws_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, wr_return_quantity@11, wr_return_amt@12] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number, wr_return_quantity, wr_return_amt], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, cs_quantity@3 as cs_quantity, cs_ext_sales_price@4 as cs_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_item_sk@4, cs_order_number@5, cs_quantity@6, cs_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@4 as cs_sold_date_sk, cs_item_sk@5 as cs_item_sk, cs_order_number@6 as cs_order_number, cs_quantity@7 as cs_quantity, cs_ext_sales_price@8 as cs_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, cs_sold_date_sk@5, cs_item_sk@6, cs_order_number@7, cs_quantity@8, cs_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_order_number, cs_quantity, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_ticket_number@2 as ss_ticket_number, ss_quantity@3 as ss_quantity, ss_ext_sales_price@4 as ss_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ticket_number@5, ss_quantity@6, ss_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_ticket_number@6 as ss_ticket_number, ss_quantity@7 as ss_quantity, ss_ext_sales_price@8 as ss_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, ss_sold_date_sk@5, ss_item_sk@6, ss_ticket_number@7, ss_quantity@8, ss_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ticket_number, ss_quantity, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_order_number@2 as ws_order_number, ws_quantity@3 as ws_quantity, ws_ext_sales_price@4 as ws_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_year@1, ws_item_sk@4, ws_order_number@5, ws_quantity@6, ws_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@4 as ws_sold_date_sk, ws_item_sk@5 as ws_item_sk, ws_order_number@6 as ws_order_number, ws_quantity@7 as ws_quantity, ws_ext_sales_price@8 as ws_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, ws_sold_date_sk@5, ws_item_sk@6, ws_order_number@7, ws_quantity@8, ws_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_order_number, ws_quantity, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] + └────────────────────────────────────────────────── + ┌───── Stage 20 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, sales_cnt@5, sales_amt@6], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sales_cnt@5 as sales_cnt, sales_amt@6 as sales_amt], aggr=[], ordering_mode=PartiallySorted([0]) + │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] t3:[] + │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, cs_quantity@1 - CASE WHEN cr_return_quantity@8 IS NOT NULL THEN cr_return_quantity@8 ELSE 0 END as sales_cnt, cs_ext_sales_price@2 - CASE WHEN __common_expr_4@0 IS NOT NULL THEN __common_expr_4@0 ELSE Some(0),30,15 END as sales_amt] + │ ProjectionExec: expr=[CAST(cr_return_amount@8 AS Decimal128(30, 15)) as __common_expr_4, cs_quantity@0 as cs_quantity, cs_ext_sales_price@1 as cs_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, cr_return_quantity@7 as cr_return_quantity] + │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(cs_order_number@1, cr_order_number@1), (cs_item_sk@0, cr_item_sk@0)], projection=[cs_quantity@2, cs_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, cr_return_quantity@11, cr_return_amount@12] + │ CoalescePartitionsExec + │ [Stage 13] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_item_sk, cr_order_number, cr_return_quantity, cr_return_amount], file_type=parquet + │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ss_quantity@1 - CASE WHEN sr_return_quantity@8 IS NOT NULL THEN sr_return_quantity@8 ELSE 0 END as sales_cnt, ss_ext_sales_price@2 - CASE WHEN __common_expr_5@0 IS NOT NULL THEN __common_expr_5@0 ELSE Some(0),30,15 END as sales_amt] + │ ProjectionExec: expr=[CAST(sr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_5, ss_quantity@0 as ss_quantity, ss_ext_sales_price@1 as ss_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, sr_return_quantity@7 as sr_return_quantity] + │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ss_ticket_number@1, sr_ticket_number@1), (ss_item_sk@0, sr_item_sk@0)], projection=[ss_quantity@2, ss_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, sr_return_quantity@11, sr_return_amt@12] + │ CoalescePartitionsExec + │ [Stage 16] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-3.parquet:..]]}, projection=[sr_item_sk, sr_ticket_number, sr_return_quantity, sr_return_amt], file_type=parquet + │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ws_quantity@1 - CASE WHEN wr_return_quantity@8 IS NOT NULL THEN wr_return_quantity@8 ELSE 0 END as sales_cnt, ws_ext_sales_price@2 - CASE WHEN __common_expr_6@0 IS NOT NULL THEN __common_expr_6@0 ELSE Some(0),30,15 END as sales_amt] + │ ProjectionExec: expr=[CAST(wr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_6, ws_quantity@0 as ws_quantity, ws_ext_sales_price@1 as ws_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, wr_return_quantity@7 as wr_return_quantity] + │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ws_order_number@1, wr_order_number@1), (ws_item_sk@0, wr_item_sk@0)], projection=[ws_quantity@2, ws_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, wr_return_quantity@11, wr_return_amt@12] + │ CoalescePartitionsExec + │ [Stage 19] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number, wr_return_quantity, wr_return_amt], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, cs_quantity@3 as cs_quantity, cs_ext_sales_price@4 as cs_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_item_sk@4, cs_order_number@5, cs_quantity@6, cs_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@4 as cs_sold_date_sk, cs_item_sk@5 as cs_item_sk, cs_order_number@6 as cs_order_number, cs_quantity@7 as cs_quantity, cs_ext_sales_price@8 as cs_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, cs_sold_date_sk@5, cs_item_sk@6, cs_order_number@7, cs_quantity@8, cs_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_order_number, cs_quantity, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, cs_quantity@3 as cs_quantity, cs_ext_sales_price@4 as cs_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_item_sk@4, cs_order_number@5, cs_quantity@6, cs_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@4 as cs_sold_date_sk, cs_item_sk@5 as cs_item_sk, cs_order_number@6 as cs_order_number, cs_quantity@7 as cs_quantity, cs_ext_sales_price@8 as cs_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, cs_sold_date_sk@5, cs_item_sk@6, cs_order_number@7, cs_quantity@8, cs_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_order_number, cs_quantity, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_ticket_number@2 as ss_ticket_number, ss_quantity@3 as ss_quantity, ss_ext_sales_price@4 as ss_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ticket_number@5, ss_quantity@6, ss_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_ticket_number@6 as ss_ticket_number, ss_quantity@7 as ss_quantity, ss_ext_sales_price@8 as ss_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, ss_sold_date_sk@5, ss_item_sk@6, ss_ticket_number@7, ss_quantity@8, ss_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ticket_number, ss_quantity, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_order_number@2 as ws_order_number, ws_quantity@3 as ws_quantity, ws_ext_sales_price@4 as ws_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_year@1, ws_item_sk@4, ws_order_number@5, ws_quantity@6, ws_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@4 as ws_sold_date_sk, ws_item_sk@5 as ws_item_sk, ws_order_number@6 as ws_order_number, ws_quantity@7 as ws_quantity, ws_ext_sales_price@8 as ws_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, ws_sold_date_sk@5, ws_item_sk@6, ws_order_number@7, ws_quantity@8, ws_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_order_number, ws_quantity, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 12 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] - └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id], aggr=[sum(sales_detail.sales_cnt), sum(sales_detail.sales_amt)], ordering_mode=PartiallySorted([0]) - │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sales_cnt@5 as sales_cnt, sales_amt@6 as sales_amt], aggr=[], ordering_mode=PartiallySorted([0]) - │ [Stage 21] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, sales_cnt@5, sales_amt@6], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sales_cnt@5 as sales_cnt, sales_amt@6 as sales_amt], aggr=[], ordering_mode=PartiallySorted([0]) - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] - │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, cs_quantity@1 - CASE WHEN cr_return_quantity@8 IS NOT NULL THEN cr_return_quantity@8 ELSE 0 END as sales_cnt, cs_ext_sales_price@2 - CASE WHEN __common_expr_4@0 IS NOT NULL THEN __common_expr_4@0 ELSE Some(0),30,15 END as sales_amt] - │ ProjectionExec: expr=[CAST(cr_return_amount@8 AS Decimal128(30, 15)) as __common_expr_4, cs_quantity@0 as cs_quantity, cs_ext_sales_price@1 as cs_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, cr_return_quantity@7 as cr_return_quantity] - │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(cs_order_number@1, cr_order_number@1), (cs_item_sk@0, cr_item_sk@0)], projection=[cs_quantity@2, cs_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, cr_return_quantity@11, cr_return_amount@12] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_item_sk, cr_order_number, cr_return_quantity, cr_return_amount], file_type=parquet - │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ss_quantity@1 - CASE WHEN sr_return_quantity@8 IS NOT NULL THEN sr_return_quantity@8 ELSE 0 END as sales_cnt, ss_ext_sales_price@2 - CASE WHEN __common_expr_5@0 IS NOT NULL THEN __common_expr_5@0 ELSE Some(0),30,15 END as sales_amt] - │ ProjectionExec: expr=[CAST(sr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_5, ss_quantity@0 as ss_quantity, ss_ext_sales_price@1 as ss_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, sr_return_quantity@7 as sr_return_quantity] - │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ss_ticket_number@1, sr_ticket_number@1), (ss_item_sk@0, sr_item_sk@0)], projection=[ss_quantity@2, ss_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, sr_return_quantity@11, sr_return_amt@12] - │ CoalescePartitionsExec - │ [Stage 17] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-3.parquet:..]]}, projection=[sr_item_sk, sr_ticket_number, sr_return_quantity, sr_return_amt], file_type=parquet - │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ws_quantity@1 - CASE WHEN wr_return_quantity@8 IS NOT NULL THEN wr_return_quantity@8 ELSE 0 END as sales_cnt, ws_ext_sales_price@2 - CASE WHEN __common_expr_6@0 IS NOT NULL THEN __common_expr_6@0 ELSE Some(0),30,15 END as sales_amt] - │ ProjectionExec: expr=[CAST(wr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_6, ws_quantity@0 as ws_quantity, ws_ext_sales_price@1 as ws_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, wr_return_quantity@7 as wr_return_quantity] - │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ws_order_number@1, wr_order_number@1), (ws_item_sk@0, wr_item_sk@0)], projection=[ws_quantity@2, ws_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, wr_return_quantity@11, wr_return_amt@12] - │ CoalescePartitionsExec - │ [Stage 20] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number, wr_return_quantity, wr_return_amt], file_type=parquet + ┌───── Stage 16 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_ticket_number@2 as ss_ticket_number, ss_quantity@3 as ss_quantity, ss_ext_sales_price@4 as ss_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ticket_number@5, ss_quantity@6, ss_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_ticket_number@6 as ss_ticket_number, ss_quantity@7 as ss_quantity, ss_ext_sales_price@8 as ss_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, ss_sold_date_sk@5, ss_item_sk@6, ss_ticket_number@7, ss_quantity@8, ss_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ticket_number, ss_quantity, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, cs_quantity@3 as cs_quantity, cs_ext_sales_price@4 as cs_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_item_sk@4, cs_order_number@5, cs_quantity@6, cs_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@4 as cs_sold_date_sk, cs_item_sk@5 as cs_item_sk, cs_order_number@6 as cs_order_number, cs_quantity@7 as cs_quantity, cs_ext_sales_price@8 as cs_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, cs_sold_date_sk@5, cs_item_sk@6, cs_order_number@7, cs_quantity@8, cs_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_order_number, cs_quantity, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 14 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] - └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_ticket_number@2 as ss_ticket_number, ss_quantity@3 as ss_quantity, ss_ext_sales_price@4 as ss_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ticket_number@5, ss_quantity@6, ss_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_ticket_number@6 as ss_ticket_number, ss_quantity@7 as ss_quantity, ss_ext_sales_price@8 as ss_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, ss_sold_date_sk@5, ss_item_sk@6, ss_ticket_number@7, ss_quantity@8, ss_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ticket_number, ss_quantity, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 15 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] - └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_order_number@2 as ws_order_number, ws_quantity@3 as ws_quantity, ws_ext_sales_price@4 as ws_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_year@1, ws_item_sk@4, ws_order_number@5, ws_quantity@6, ws_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@4 as ws_sold_date_sk, ws_item_sk@5 as ws_item_sk, ws_order_number@6 as ws_order_number, ws_quantity@7 as ws_quantity, ws_ext_sales_price@8 as ws_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, ws_sold_date_sk@5, ws_item_sk@6, ws_order_number@7, ws_quantity@8, ws_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_order_number, ws_quantity, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 19 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_order_number@2 as ws_order_number, ws_quantity@3 as ws_quantity, ws_ext_sales_price@4 as ws_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_year@1, ws_item_sk@4, ws_order_number@5, ws_quantity@6, ws_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@4 as ws_sold_date_sk, ws_item_sk@5 as ws_item_sk, ws_order_number@6 as ws_order_number, ws_quantity@7 as ws_quantity, ws_ext_sales_price@8 as ws_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, ws_sold_date_sk@5, ws_item_sk@6, ws_order_number@7, ws_quantity@8, ws_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_order_number, ws_quantity, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 17 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 18 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] - └────────────────────────────────────────────────── "); Ok(()) } @@ -8180,30 +7808,30 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [channel@0 ASC, col_name@1 ASC, d_year@2 ASC, d_qoy@3 ASC, i_category@4 ASC], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, col_name@1 ASC, d_year@2 ASC, d_qoy@3 ASC, i_category@4 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[channel@0 as channel, col_name@1 as col_name, d_year@2 as d_year, d_qoy@3 as d_qoy, i_category@4 as i_category, count(Int64(1))@5 as sales_cnt, sum(foo.ext_sales_price)@6 as sales_amt] │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, col_name@1 as col_name, d_year@2 as d_year, d_qoy@3 as d_qoy, i_category@4 as i_category], aggr=[count(Int64(1)), sum(foo.ext_sales_price)] │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([channel@0, col_name@1, d_year@2, d_qoy@3, i_category@4], 6), input_partitions=3 + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([channel@0, col_name@1, d_year@2, d_qoy@3, i_category@4], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[channel@0 as channel, col_name@1 as col_name, d_year@2 as d_year, d_qoy@3 as d_qoy, i_category@4 as i_category], aggr=[count(Int64(1)), sum(foo.ext_sales_price)], ordering_mode=PartiallySorted([0, 1]) │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] │ ProjectionExec: expr=[store as channel, ss_store_sk as col_name, d_year@0 as d_year, d_qoy@1 as d_qoy, i_category@3 as i_category, ss_ext_sales_price@2 as ext_sales_price] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_ext_sales_price@5, i_category@6] │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[web as channel, ws_ship_customer_sk as col_name, d_year@0 as d_year, d_qoy@1 as d_qoy, i_category@3 as i_category, ws_ext_sales_price@2 as ext_sales_price] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_ext_sales_price@5, i_category@6] │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[catalog as channel, cs_ship_addr_sk as col_name, d_year@0 as d_year, d_qoy@1 as d_qoy, i_category@3 as i_category, cs_ext_sales_price@2 as ext_sales_price] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, cs_ext_sales_price@5, i_category@6] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@3], 3), input_partitions=3 @@ -8212,20 +7840,19 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 3), input_partitions=2 │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_ext_sales_price@2 as ss_ext_sales_price, i_category@0 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_category@1, ss_sold_date_sk@2, ss_ext_sales_price@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ FilterExec: ss_store_sk@2 IS NULL, projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_ext_sales_price@3] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=ss_store_sk@7 IS NULL AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=ss_store_sk_null_count@0 > 0, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@3], 3), input_partitions=3 @@ -8234,20 +7861,19 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 3), input_partitions=2 │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ext_sales_price@2 as ws_ext_sales_price, i_category@0 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_category@1, ws_sold_date_sk@2, ws_ext_sales_price@4] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ FilterExec: ws_ship_customer_sk@2 IS NULL, projection=[ws_sold_date_sk@0, ws_item_sk@1, ws_ext_sales_price@3] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ship_customer_sk, ws_ext_sales_price], file_type=parquet, predicate=ws_ship_customer_sk@8 IS NULL AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=ws_ship_customer_sk_null_count@0 > 0, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet + ┌───── Stage 5 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@3], 6), input_partitions=3 @@ -8256,20 +7882,19 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_qoy], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 6), input_partitions=2 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ext_sales_price@2 as cs_ext_sales_price, i_category@0 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_category@1, cs_sold_date_sk@2, cs_ext_sales_price@4] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ FilterExec: cs_ship_addr_sk@1 IS NULL, projection=[cs_sold_date_sk@0, cs_item_sk@2, cs_ext_sales_price@3] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=cs_ship_addr_sk@10 IS NULL AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=cs_ship_addr_sk_null_count@0 > 0, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet + ┌───── Stage 8 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -8280,195 +7905,182 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [channel@0 ASC, id@1 ASC, returns_@3 DESC], fetch=100 - │ [Stage 21] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 20] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 20 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, id@1 ASC, returns_@3 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[channel@0 as channel, id@1 as id, sum(x.sales)@3 as sales, sum(x.returns_)@4 as returns_, sum(x.profit)@5 as profit] │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, id@1 as id, __grouping_id@2 as __grouping_id], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] - │ [Stage 20] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 19] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([channel@0, id@1, __grouping_id@2], 6), input_partitions=3 + ┌───── Stage 19 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([channel@0, id@1, __grouping_id@2], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[(NULL as channel, NULL as id), (channel@0 as channel, NULL as id), (channel@0 as channel, id@1 as id)], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1] t3:[c2] │ ProjectionExec: expr=[store channel as channel, CAST(s_store_sk@2 AS Float64) as id, sales@3 as sales, CASE WHEN __common_expr_1@0 IS NOT NULL THEN __common_expr_1@0 ELSE Some(0),22,2 END as returns_, profit@4 - CASE WHEN __common_expr_2@1 IS NOT NULL THEN __common_expr_2@1 ELSE Some(0),22,2 END as profit] │ ProjectionExec: expr=[CAST(returns_@0 AS Decimal128(22, 2)) as __common_expr_1, CAST(profit_loss@1 AS Decimal128(22, 2)) as __common_expr_2, s_store_sk@2 as s_store_sk, sales@3 as sales, profit@4 as profit] │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(s_store_sk@0, s_store_sk@0)], projection=[returns_@1, profit_loss@2, s_store_sk@3, sales@4, profit@5] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, sum(store_sales.ss_ext_sales_price)@1 as sales, sum(store_sales.ss_net_profit)@2 as profit] │ AggregateExec: mode=FinalPartitioned, gby=[s_store_sk@0 as s_store_sk], aggr=[sum(store_sales.ss_ext_sales_price), sum(store_sales.ss_net_profit)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[catalog channel as channel, cs_call_center_sk@0 as id, sales@1 as sales, CAST(returns_@3 AS Decimal128(22, 2)) as returns_, CAST(profit@2 - profit_loss@4 AS Decimal128(23, 2)) as profit] │ CrossJoinExec │ CoalescePartitionsExec - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 │ ProjectionExec: expr=[sum(catalog_returns.cr_return_amount)@1 as returns_, sum(catalog_returns.cr_net_loss)@2 as profit_loss] │ AggregateExec: mode=FinalPartitioned, gby=[cr_call_center_sk@0 as cr_call_center_sk], aggr=[sum(catalog_returns.cr_return_amount), sum(catalog_returns.cr_net_loss)] - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[web channel as channel, CAST(wp_web_page_sk@2 AS Float64) as id, sales@3 as sales, CASE WHEN __common_expr_3@0 IS NOT NULL THEN __common_expr_3@0 ELSE Some(0),22,2 END as returns_, profit@4 - CASE WHEN __common_expr_4@1 IS NOT NULL THEN __common_expr_4@1 ELSE Some(0),22,2 END as profit] │ ProjectionExec: expr=[CAST(returns_@3 AS Decimal128(22, 2)) as __common_expr_3, CAST(profit_loss@4 AS Decimal128(22, 2)) as __common_expr_4, wp_web_page_sk@0 as wp_web_page_sk, sales@1 as sales, profit@2 as profit] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(wp_web_page_sk@0, wp_web_page_sk@0)], projection=[wp_web_page_sk@0, sales@1, profit@2, returns_@4, profit_loss@5] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, sum(web_sales.ws_ext_sales_price)@1 as sales, sum(web_sales.ws_net_profit)@2 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[wp_web_page_sk@0 as wp_web_page_sk], aggr=[sum(web_sales.ws_ext_sales_price), sum(web_sales.ws_net_profit)] + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, sum(web_returns.wr_return_amt)@1 as returns_, sum(web_returns.wr_net_loss)@2 as profit_loss] │ AggregateExec: mode=FinalPartitioned, gby=[wp_web_page_sk@0 as wp_web_page_sk], aggr=[sum(web_returns.wr_return_amt), sum(web_returns.wr_net_loss)] - │ [Stage 19] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, sum(store_returns.sr_return_amt)@1 as returns_, sum(store_returns.sr_net_loss)@2 as profit_loss] │ AggregateExec: mode=FinalPartitioned, gby=[s_store_sk@0 as s_store_sk], aggr=[sum(store_returns.sr_return_amt), sum(store_returns.sr_net_loss)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([s_store_sk@0], 6), input_partitions=2 + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([s_store_sk@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[s_store_sk@2 as s_store_sk], aggr=[sum(store_returns.sr_return_amt), sum(store_returns.sr_net_loss)] │ ProjectionExec: expr=[sr_return_amt@1 as sr_return_amt, sr_net_loss@2 as sr_net_loss, s_store_sk@0 as s_store_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, sr_store_sk@0)], projection=[s_store_sk@0, sr_return_amt@3, sr_net_loss@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sr_returned_date_sk@0)], projection=[sr_store_sk@3, sr_return_amt@4, sr_net_loss@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_store_sk, sr_return_amt, sr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([s_store_sk@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[s_store_sk@2 as s_store_sk], aggr=[sum(store_sales.ss_ext_sales_price), sum(store_sales.ss_net_profit)] │ ProjectionExec: expr=[ss_ext_sales_price@1 as ss_ext_sales_price, ss_net_profit@2 as ss_net_profit, s_store_sk@0 as s_store_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[s_store_sk@0, ss_ext_sales_price@3, ss_net_profit@4] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_store_sk@3, ss_ext_sales_price@4, ss_net_profit@5] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 5 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ ProjectionExec: expr=[cs_call_center_sk@0 as cs_call_center_sk, sum(catalog_sales.cs_ext_sales_price)@1 as sales, sum(catalog_sales.cs_net_profit)@2 as profit] │ AggregateExec: mode=FinalPartitioned, gby=[cs_call_center_sk@0 as cs_call_center_sk], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(catalog_sales.cs_net_profit)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cs_call_center_sk@0], 6), input_partitions=2 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_call_center_sk@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[cs_call_center_sk@0 as cs_call_center_sk], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(catalog_sales.cs_net_profit)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_call_center_sk@3, cs_ext_sales_price@4, cs_net_profit@5] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_ext_sales_price, cs_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([cr_call_center_sk@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[cr_call_center_sk@0 as cr_call_center_sk], aggr=[sum(catalog_returns.cr_return_amount), sum(catalog_returns.cr_net_loss)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[cr_call_center_sk@2, cr_return_amount@3, cr_net_loss@4] │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_call_center_sk, cr_return_amount, cr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, sum(web_sales.ws_ext_sales_price)@1 as sales, sum(web_sales.ws_net_profit)@2 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[wp_web_page_sk@0 as wp_web_page_sk], aggr=[sum(web_sales.ws_ext_sales_price), sum(web_sales.ws_net_profit)] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([wp_web_page_sk@0], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[wp_web_page_sk@2 as wp_web_page_sk], aggr=[sum(web_sales.ws_ext_sales_price), sum(web_sales.ws_net_profit)] - │ ProjectionExec: expr=[ws_ext_sales_price@1 as ws_ext_sales_price, ws_net_profit@2 as ws_net_profit, wp_web_page_sk@0 as wp_web_page_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@0)], projection=[wp_web_page_sk@0, ws_ext_sales_price@3, ws_net_profit@4] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_web_page_sk@3, ws_ext_sales_price@4, ws_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_web_page_sk, ws_ext_sales_price, ws_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([wp_web_page_sk@0], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[wp_web_page_sk@2 as wp_web_page_sk], aggr=[sum(web_sales.ws_ext_sales_price), sum(web_sales.ws_net_profit)] + │ ProjectionExec: expr=[ws_ext_sales_price@1 as ws_ext_sales_price, ws_net_profit@2 as ws_net_profit, wp_web_page_sk@0 as wp_web_page_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@0)], projection=[wp_web_page_sk@0, ws_ext_sales_price@3, ws_net_profit@4] + │ CoalescePartitionsExec + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_web_page_sk@3, ws_ext_sales_price@4, ws_net_profit@5] + │ CoalescePartitionsExec + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_web_page_sk, ws_ext_sales_price, ws_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([wp_web_page_sk@0], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[wp_web_page_sk@2 as wp_web_page_sk], aggr=[sum(web_returns.wr_return_amt), sum(web_returns.wr_net_loss)] │ ProjectionExec: expr=[wr_return_amt@1 as wr_return_amt, wr_net_loss@2 as wr_net_loss, wp_web_page_sk@0 as wp_web_page_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, wr_web_page_sk@0)], projection=[wp_web_page_sk@0, wr_return_amt@3, wr_net_loss@4] │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, wr_returned_date_sk@0)], projection=[wr_web_page_sk@3, wr_return_amt@4, wr_net_loss@5] │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_returned_date_sk, wr_web_page_sk, wr_return_amt, wr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet + ┌───── Stage 16 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 17 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -8488,111 +8100,103 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ss_sold_year@0, ws_sold_year@0), (ss_item_sk@1, ws_item_sk@1), (ss_customer_sk@2, ws_customer_sk@2)], projection=[ss_sold_year@0, ss_item_sk@1, ss_customer_sk@2, ss_qty@3, ss_wc@4, ss_sp@5, ws_qty@9, ws_wc@10, ws_sp@11] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[d_year@0 as ss_sold_year, ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, sum(store_sales.ss_quantity)@3 as ss_qty, sum(store_sales.ss_wholesale_cost)@4 as ss_wc, sum(store_sales.ss_sales_price)@5 as ss_sp] + │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk], aggr=[sum(store_sales.ss_quantity), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_sales_price)], ordering_mode=PartiallySorted([0]) + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[d_year@0 as ws_sold_year, ws_item_sk@1 as ws_item_sk, ws_bill_customer_sk@2 as ws_customer_sk, sum(web_sales.ws_quantity)@3 as ws_qty, sum(web_sales.ws_wholesale_cost)@4 as ws_wc, sum(web_sales.ws_sales_price)@5 as ws_sp] │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, ws_item_sk@1 as ws_item_sk, ws_bill_customer_sk@2 as ws_bill_customer_sk], aggr=[sum(web_sales.ws_quantity), sum(web_sales.ws_wholesale_cost), sum(web_sales.ws_sales_price)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ ProjectionExec: expr=[d_year@0 as cs_sold_year, cs_item_sk@1 as cs_item_sk, cs_bill_customer_sk@2 as cs_customer_sk, sum(catalog_sales.cs_quantity)@3 as cs_qty, sum(catalog_sales.cs_wholesale_cost)@4 as cs_wc, sum(catalog_sales.cs_sales_price)@5 as cs_sp] │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, cs_item_sk@1 as cs_item_sk, cs_bill_customer_sk@2 as cs_bill_customer_sk], aggr=[sum(catalog_sales.cs_quantity), sum(catalog_sales.cs_wholesale_cost), sum(catalog_sales.cs_sales_price)] - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ ProjectionExec: expr=[d_year@0 as ss_sold_year, ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, sum(store_sales.ss_quantity)@3 as ss_qty, sum(store_sales.ss_wholesale_cost)@4 as ss_wc, sum(store_sales.ss_sales_price)@5 as ss_sp] - │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk], aggr=[sum(store_sales.ss_quantity), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_sales_price)], ordering_mode=PartiallySorted([0]) - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([d_year@0, ss_item_sk@1, ss_customer_sk@2], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[d_year@5 as d_year, ss_item_sk@0 as ss_item_sk, ss_customer_sk@1 as ss_customer_sk], aggr=[sum(store_sales.ss_quantity), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_sales_price)], ordering_mode=PartiallySorted([0]) - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_quantity@3 as ss_quantity, ss_wholesale_cost@4 as ss_wholesale_cost, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_quantity@6, ss_wholesale_cost@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ FilterExec: sr_ticket_number@6 IS NULL, projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_customer_sk@2, ss_quantity@3, ss_wholesale_cost@4, ss_sales_price@5] - │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_quantity@4 as ss_quantity, ss_wholesale_cost@5 as ss_wholesale_cost, ss_sales_price@6 as ss_sales_price, sr_ticket_number@0 as sr_ticket_number] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@1)], projection=[sr_ticket_number@1, ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_quantity@6, ss_wholesale_cost@7, ss_sales_price@8] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_ticket_number, ss_quantity, ss_wholesale_cost, ss_sales_price], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([d_year@0, ss_item_sk@1, ss_customer_sk@2], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[d_year@5 as d_year, ss_item_sk@0 as ss_item_sk, ss_customer_sk@1 as ss_customer_sk], aggr=[sum(store_sales.ss_quantity), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_sales_price)], ordering_mode=PartiallySorted([0]) + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_quantity@3 as ss_quantity, ss_wholesale_cost@4 as ss_wholesale_cost, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_quantity@6, ss_wholesale_cost@7, ss_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ FilterExec: sr_ticket_number@6 IS NULL, projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_customer_sk@2, ss_quantity@3, ss_wholesale_cost@4, ss_sales_price@5] + │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_quantity@4 as ss_quantity, ss_wholesale_cost@5 as ss_wholesale_cost, ss_sales_price@6 as ss_sales_price, sr_ticket_number@0 as sr_ticket_number] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@1)], projection=[sr_ticket_number@1, ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_quantity@6, ss_wholesale_cost@7, ss_sales_price@8] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 12), input_partitions=3 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_ticket_number, ss_quantity, ss_wholesale_cost, ss_sales_price], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([d_year@0, ws_item_sk@1, ws_bill_customer_sk@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_year@5 as d_year, ws_item_sk@0 as ws_item_sk, ws_bill_customer_sk@1 as ws_bill_customer_sk], aggr=[sum(web_sales.ws_quantity), sum(web_sales.ws_wholesale_cost), sum(web_sales.ws_sales_price)] │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_bill_customer_sk@2 as ws_bill_customer_sk, ws_quantity@3 as ws_quantity, ws_wholesale_cost@4 as ws_wholesale_cost, ws_sales_price@5 as ws_sales_price, d_year@0 as d_year] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_year@1, ws_item_sk@4, ws_bill_customer_sk@5, ws_quantity@6, ws_wholesale_cost@7, ws_sales_price@8] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 6), input_partitions=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 12), input_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 6), input_partitions=2 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 12), input_partitions=2 │ FilterExec: wr_order_number@6 IS NULL, projection=[ws_sold_date_sk@0, ws_item_sk@1, ws_bill_customer_sk@2, ws_quantity@3, ws_wholesale_cost@4, ws_sales_price@5] │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_item_sk@2 as ws_item_sk, ws_bill_customer_sk@3 as ws_bill_customer_sk, ws_quantity@4 as ws_quantity, ws_wholesale_cost@5 as ws_wholesale_cost, ws_sales_price@6 as ws_sales_price, wr_order_number@0 as wr_order_number] │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(wr_order_number@1, ws_order_number@3), (wr_item_sk@0, ws_item_sk@1)], projection=[wr_order_number@1, ws_sold_date_sk@2, ws_item_sk@3, ws_bill_customer_sk@4, ws_quantity@6, ws_wholesale_cost@7, ws_sales_price@8] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_customer_sk, ws_order_number, ws_quantity, ws_wholesale_cost, ws_sales_price], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([d_year@0, cs_item_sk@1, cs_bill_customer_sk@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_year@5 as d_year, cs_item_sk@1 as cs_item_sk, cs_bill_customer_sk@0 as cs_bill_customer_sk], aggr=[sum(catalog_sales.cs_quantity), sum(catalog_sales.cs_wholesale_cost), sum(catalog_sales.cs_sales_price)] │ ProjectionExec: expr=[cs_bill_customer_sk@1 as cs_bill_customer_sk, cs_item_sk@2 as cs_item_sk, cs_quantity@3 as cs_quantity, cs_wholesale_cost@4 as cs_wholesale_cost, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_bill_customer_sk@4, cs_item_sk@5, cs_quantity@6, cs_wholesale_cost@7, cs_sales_price@8] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 6), input_partitions=3 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 12), input_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 6), input_partitions=3 + ┌───── Stage 12 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 12), input_partitions=3 │ FilterExec: cr_order_number@6 IS NULL, projection=[cs_sold_date_sk@0, cs_bill_customer_sk@1, cs_item_sk@2, cs_quantity@3, cs_wholesale_cost@4, cs_sales_price@5] │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_bill_customer_sk@2 as cs_bill_customer_sk, cs_item_sk@3 as cs_item_sk, cs_quantity@4 as cs_quantity, cs_wholesale_cost@5 as cs_wholesale_cost, cs_sales_price@6 as cs_sales_price, cr_order_number@0 as cr_order_number] │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@3), (cr_item_sk@0, cs_item_sk@2)], projection=[cr_order_number@1, cs_sold_date_sk@2, cs_bill_customer_sk@3, cs_item_sk@4, cs_quantity@6, cs_wholesale_cost@7, cs_sales_price@8] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number], file_type=parquet + ┌───── Stage 10 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 12), input_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_item_sk, cr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_order_number@3, cs_item_sk@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + ┌───── Stage 11 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_order_number@3, cs_item_sk@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_order_number, cs_quantity, cs_wholesale_cost, cs_sales_price], file_type=parquet └────────────────────────────────────────────────── "); @@ -8604,59 +8208,57 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, substr(ms.s_city,Int64(1),Int64(30))@2 ASC, profit@5 ASC, ss_ticket_number@3 ASC NULLS LAST], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, c_first_name@1 ASC, substr(ms.s_city,Int64(1),Int64(30))@2 ASC, profit@5 ASC, ss_ticket_number@3 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[c_last_name@5 as c_last_name, c_first_name@4 as c_first_name, substr(s_city@1, 1, 30) as substr(ms.s_city,Int64(1),Int64(30)), ss_ticket_number@0 as ss_ticket_number, amt@2 as amt, profit@3 as profit] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@3)], projection=[ss_ticket_number@0, s_city@2, amt@3, profit@4, c_first_name@6, c_last_name@7] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, s_city@3 as s_city, sum(store_sales.ss_coupon_amt)@4 as amt, sum(store_sales.ss_net_profit)@5 as profit] │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ss_addr_sk@2 as ss_addr_sk, s_city@3 as s_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1, ss_addr_sk@2, s_city@3], 6), input_partitions=2 + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1, ss_addr_sk@2, s_city@3], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@2 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk, ss_addr_sk@1 as ss_addr_sk, s_city@5 as s_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_addr_sk@4, ss_ticket_number@5, ss_coupon_amt@6, ss_net_profit@7, s_city@8] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[ss_customer_sk@1 as ss_customer_sk, ss_hdemo_sk@2 as ss_hdemo_sk, ss_addr_sk@3 as ss_addr_sk, ss_ticket_number@4 as ss_ticket_number, ss_coupon_amt@5 as ss_coupon_amt, ss_net_profit@6 as ss_net_profit, s_city@0 as s_city] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@3)], projection=[s_city@1, ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_ticket_number@7, ss_coupon_amt@8, ss_net_profit@9] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_store_sk@6, ss_ticket_number@7, ss_coupon_amt@8, ss_net_profit@9] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_ticket_number, ss_coupon_amt, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 6 OR hd_vehicle_count@2 > 2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 6 OR hd_vehicle_count@4 > 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 6 AND 6 <= hd_dep_count_max@1 OR hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 2, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 6 OR hd_vehicle_count@4 > 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 6 AND 6 <= hd_dep_count_max@1 OR hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 2, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_city@1 as s_city, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_number_employees@1 >= 200 AND s_number_employees@1 <= 295, projection=[s_store_sk@0, s_city@2] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_number_employees, s_city], file_type=parquet, predicate=s_number_employees@6 >= 200 AND s_number_employees@6 <= 295, pruning_predicate=s_number_employees_null_count@1 != row_count@2 AND s_number_employees_max@0 >= 200 AND s_number_employees_null_count@1 != row_count@2 AND s_number_employees_min@3 <= 295, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_number_employees, s_city], file_type=parquet, predicate=s_number_employees@6 >= 200 AND s_number_employees@6 <= 295, pruning_predicate=s_number_employees_null_count@1 != row_count@2 AND s_number_employees_max@0 >= 200 AND s_number_employees_null_count@1 != row_count@2 AND s_number_employees_min@3 <= 295, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_dow@2 = 1 AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -8675,204 +8277,187 @@ mod tests { │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, id@1 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[channel@0 as channel, id@1 as id, sum(x.sales)@3 as sales, sum(x.returns_)@4 as returns_, sum(x.profit)@5 as profit] │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, id@1 as id, __grouping_id@2 as __grouping_id], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] - │ [Stage 22] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ RepartitionExec: partitioning=Hash([channel@0, id@1, __grouping_id@2], 3), input_partitions=9 + │ AggregateExec: mode=Partial, gby=[(NULL as channel, NULL as id), (channel@0 as channel, NULL as id), (channel@0 as channel, id@1 as id)], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] + │ DistributedUnionExec: t0:[c0, c1, c2] + │ ProjectionExec: expr=[store channel as channel, concat(store, s_store_id@0) as id, sum(store_sales.ss_ext_sales_price)@1 as sales, sum(coalesce(store_returns.sr_return_amt,Int64(0)))@2 as returns_, sum(store_sales.ss_net_profit - coalesce(store_returns.sr_net_loss,Int64(0)))@3 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[s_store_id@0 as s_store_id], aggr=[sum(store_sales.ss_ext_sales_price), sum(coalesce(store_returns.sr_return_amt,Int64(0))), sum(store_sales.ss_net_profit - coalesce(store_returns.sr_net_loss,Int64(0)))] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[catalog channel as channel, concat(catalog_page, cp_catalog_page_id@0) as id, sum(catalog_sales.cs_ext_sales_price)@1 as sales, sum(coalesce(catalog_returns.cr_return_amount,Int64(0)))@2 as returns_, sum(catalog_sales.cs_net_profit - coalesce(catalog_returns.cr_net_loss,Int64(0)))@3 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[cp_catalog_page_id@0 as cp_catalog_page_id], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(coalesce(catalog_returns.cr_return_amount,Int64(0))), sum(catalog_sales.cs_net_profit - coalesce(catalog_returns.cr_net_loss,Int64(0)))] + │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[web channel as channel, concat(web_site, web_site_id@0) as id, sum(web_sales.ws_ext_sales_price)@1 as sales, sum(coalesce(web_returns.wr_return_amt,Int64(0)))@2 as returns_, sum(web_sales.ws_net_profit - coalesce(web_returns.wr_net_loss,Int64(0)))@3 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[web_site_id@0 as web_site_id], aggr=[sum(web_sales.ws_ext_sales_price), sum(coalesce(web_returns.wr_return_amt,Int64(0))), sum(web_sales.ws_net_profit - coalesce(web_returns.wr_net_loss,Int64(0)))] + │ RepartitionExec: partitioning=Hash([web_site_id@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[web_site_id@4 as web_site_id], aggr=[sum(web_sales.ws_ext_sales_price), sum(coalesce(web_returns.wr_return_amt,Int64(0))), sum(web_sales.ws_net_profit - coalesce(web_returns.wr_net_loss,Int64(0)))] + │ ProjectionExec: expr=[CAST(wr_return_amt@2 AS Decimal128(22, 2)) as __common_expr_5, CAST(wr_net_loss@3 AS Decimal128(22, 2)) as __common_expr_6, ws_ext_sales_price@0 as ws_ext_sales_price, ws_net_profit@1 as ws_net_profit, web_site_id@4 as web_site_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_promo_sk@0, CAST(promotion.p_promo_sk AS Float64)@1)], projection=[ws_ext_sales_price@1, ws_net_profit@2, wr_return_amt@3, wr_net_loss@4, web_site_id@5] + │ CoalescePartitionsExec + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=4 + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] - │ RepartitionExec: partitioning=Hash([channel@0, id@1, __grouping_id@2], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[(NULL as channel, NULL as id), (channel@0 as channel, NULL as id), (channel@0 as channel, id@1 as id)], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] - │ ProjectionExec: expr=[store channel as channel, concat(store, s_store_id@0) as id, sum(store_sales.ss_ext_sales_price)@1 as sales, sum(coalesce(store_returns.sr_return_amt,Int64(0)))@2 as returns_, sum(store_sales.ss_net_profit - coalesce(store_returns.sr_net_loss,Int64(0)))@3 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[s_store_id@0 as s_store_id], aggr=[sum(store_sales.ss_ext_sales_price), sum(coalesce(store_returns.sr_return_amt,Int64(0))), sum(store_sales.ss_net_profit - coalesce(store_returns.sr_net_loss,Int64(0)))] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ ProjectionExec: expr=[catalog channel as channel, concat(catalog_page, cp_catalog_page_id@0) as id, sum(catalog_sales.cs_ext_sales_price)@1 as sales, sum(coalesce(catalog_returns.cr_return_amount,Int64(0)))@2 as returns_, sum(catalog_sales.cs_net_profit - coalesce(catalog_returns.cr_net_loss,Int64(0)))@3 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[cp_catalog_page_id@0 as cp_catalog_page_id], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(coalesce(catalog_returns.cr_return_amount,Int64(0))), sum(catalog_sales.cs_net_profit - coalesce(catalog_returns.cr_net_loss,Int64(0)))] - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ ProjectionExec: expr=[web channel as channel, concat(web_site, web_site_id@0) as id, sum(web_sales.ws_ext_sales_price)@1 as sales, sum(coalesce(web_returns.wr_return_amt,Int64(0)))@2 as returns_, sum(web_sales.ws_net_profit - coalesce(web_returns.wr_net_loss,Int64(0)))@3 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[web_site_id@0 as web_site_id], aggr=[sum(web_sales.ws_ext_sales_price), sum(coalesce(web_returns.wr_return_amt,Int64(0))), sum(web_sales.ws_net_profit - coalesce(web_returns.wr_net_loss,Int64(0)))] - │ [Stage 21] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([s_store_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[s_store_id@4 as s_store_id], aggr=[sum(store_sales.ss_ext_sales_price), sum(coalesce(store_returns.sr_return_amt,Int64(0))), sum(store_sales.ss_net_profit - coalesce(store_returns.sr_net_loss,Int64(0)))] - │ ProjectionExec: expr=[CAST(sr_return_amt@2 AS Decimal128(22, 2)) as __common_expr_1, CAST(sr_net_loss@3 AS Decimal128(22, 2)) as __common_expr_2, ss_ext_sales_price@0 as ss_ext_sales_price, ss_net_profit@1 as ss_net_profit, s_store_id@4 as s_store_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, sr_return_amt@5, sr_net_loss@6, s_store_id@7] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([s_store_id@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[s_store_id@4 as s_store_id], aggr=[sum(store_sales.ss_ext_sales_price), sum(coalesce(store_returns.sr_return_amt,Int64(0))), sum(store_sales.ss_net_profit - coalesce(store_returns.sr_net_loss,Int64(0)))] + │ ProjectionExec: expr=[CAST(sr_return_amt@2 AS Decimal128(22, 2)) as __common_expr_1, CAST(sr_net_loss@3 AS Decimal128(22, 2)) as __common_expr_2, ss_ext_sales_price@0 as ss_ext_sales_price, ss_net_profit@1 as ss_net_profit, s_store_id@4 as s_store_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, sr_return_amt@5, sr_net_loss@6, s_store_id@7] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@0)], projection=[ss_promo_sk@2, ss_ext_sales_price@3, ss_net_profit@4, sr_return_amt@5, sr_net_loss@6, s_store_id@7] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_promo_sk@2 as ss_promo_sk, ss_ext_sales_price@3 as ss_ext_sales_price, ss_net_profit@4 as ss_net_profit, sr_return_amt@5 as sr_return_amt, sr_net_loss@6 as sr_net_loss, s_store_id@0 as s_store_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, ss_item_sk@3, ss_promo_sk@5, ss_ext_sales_price@6, ss_net_profit@7, sr_return_amt@8, sr_net_loss@9] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_promo_sk@5, ss_ext_sales_price@6, ss_net_profit@7, sr_return_amt@8, sr_net_loss@9] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_promo_sk@5 as ss_promo_sk, ss_ext_sales_price@6 as ss_ext_sales_price, ss_net_profit@7 as ss_net_profit, sr_return_amt@0 as sr_return_amt, sr_net_loss@1 as sr_net_loss] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@4)], projection=[sr_return_amt@2, sr_net_loss@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_promo_sk@7, ss_ext_sales_price@9, ss_net_profit@10] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number, sr_return_amt, sr_net_loss], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@4], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_promo_sk, ss_ticket_number, ss_ext_sales_price, ss_net_profit], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([cp_catalog_page_id@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[cp_catalog_page_id@4 as cp_catalog_page_id], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(coalesce(catalog_returns.cr_return_amount,Int64(0))), sum(catalog_sales.cs_net_profit - coalesce(catalog_returns.cr_net_loss,Int64(0)))] + │ ProjectionExec: expr=[CAST(cr_return_amount@2 AS Decimal128(22, 2)) as __common_expr_3, CAST(cr_net_loss@3 AS Decimal128(22, 2)) as __common_expr_4, cs_ext_sales_price@0 as cs_ext_sales_price, cs_net_profit@1 as cs_net_profit, cp_catalog_page_id@4 as cp_catalog_page_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, cs_promo_sk@0)], projection=[cs_ext_sales_price@3, cs_net_profit@4, cr_return_amount@5, cr_net_loss@6, cp_catalog_page_id@7] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@0)], projection=[cs_promo_sk@2, cs_ext_sales_price@3, cs_net_profit@4, cr_return_amount@5, cr_net_loss@6, cp_catalog_page_id@7] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_promo_sk@2 as cs_promo_sk, cs_ext_sales_price@3 as cs_ext_sales_price, cs_net_profit@4 as cs_net_profit, cr_return_amount@5 as cr_return_amount, cr_net_loss@6 as cr_net_loss, cp_catalog_page_id@0 as cp_catalog_page_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(catalog_page.cp_catalog_page_sk AS Float64)@2, cs_catalog_page_sk@0)], projection=[cp_catalog_page_id@1, cs_item_sk@4, cs_promo_sk@5, cs_ext_sales_price@6, cs_net_profit@7, cr_return_amount@8, cr_net_loss@9] + │ CoalescePartitionsExec + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_catalog_page_sk@3, cs_item_sk@4, cs_promo_sk@5, cs_ext_sales_price@6, cs_net_profit@7, cr_return_amount@8, cr_net_loss@9] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_catalog_page_sk@3 as cs_catalog_page_sk, cs_item_sk@4 as cs_item_sk, cs_promo_sk@5 as cs_promo_sk, cs_ext_sales_price@6 as cs_ext_sales_price, cs_net_profit@7 as cs_net_profit, cr_return_amount@0 as cr_return_amount, cr_net_loss@1 as cr_net_loss] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_item_sk@0, cs_item_sk@2), (cr_order_number@1, cs_order_number@4)], projection=[cr_return_amount@2, cr_net_loss@3, cs_sold_date_sk@4, cs_catalog_page_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_ext_sales_price@9, cs_net_profit@10] + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[cp_catalog_page_sk@0 as cp_catalog_page_sk, cp_catalog_page_id@1 as cp_catalog_page_id, CAST(cp_catalog_page_sk@0 AS Float64) as CAST(catalog_page.cp_catalog_page_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/catalog_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/catalog_page/part-3.parquet]]}, projection=[cp_catalog_page_sk, cp_catalog_page_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_return_amount, cr_net_loss], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_item_sk@2, cs_order_number@4], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_catalog_page_sk, cs_item_sk, cs_promo_sk, cs_order_number, cs_ext_sales_price, cs_net_profit], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 20 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_promo_sk@1, ws_ext_sales_price@2, ws_net_profit@3, wr_return_amt@4, wr_net_loss@5, web_site_id@6] + │ CoalescePartitionsExec + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2 AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 19 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_promo_sk@2 as ws_promo_sk, ws_ext_sales_price@3 as ws_ext_sales_price, ws_net_profit@4 as ws_net_profit, wr_return_amt@5 as wr_return_amt, wr_net_loss@6 as wr_net_loss, web_site_id@0 as web_site_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@2, ws_web_site_sk@1)], projection=[web_site_id@1, ws_item_sk@3, ws_promo_sk@5, ws_ext_sales_price@6, ws_net_profit@7, wr_return_amt@8, wr_net_loss@9] + │ CoalescePartitionsExec + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_web_site_sk@4, ws_promo_sk@5, ws_ext_sales_price@6, ws_net_profit@7, wr_return_amt@8, wr_net_loss@9] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@0)], projection=[ss_promo_sk@2, ss_ext_sales_price@3, ss_net_profit@4, sr_return_amt@5, sr_net_loss@6, s_store_id@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_promo_sk@2 as ss_promo_sk, ss_ext_sales_price@3 as ss_ext_sales_price, ss_net_profit@4 as ss_net_profit, sr_return_amt@5 as sr_return_amt, sr_net_loss@6 as sr_net_loss, s_store_id@0 as s_store_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, ss_item_sk@3, ss_promo_sk@5, ss_ext_sales_price@6, ss_net_profit@7, sr_return_amt@8, sr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_promo_sk@5, ss_ext_sales_price@6, ss_net_profit@7, sr_return_amt@8, sr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_promo_sk@5 as ss_promo_sk, ss_ext_sales_price@6 as ss_ext_sales_price, ss_net_profit@7 as ss_net_profit, sr_return_amt@0 as sr_return_amt, sr_net_loss@1 as sr_net_loss] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@4)], projection=[sr_return_amt@2, sr_net_loss@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_promo_sk@7, ss_ext_sales_price@9, ss_net_profit@10] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_web_site_sk@4 as ws_web_site_sk, ws_promo_sk@5 as ws_promo_sk, ws_ext_sales_price@6 as ws_ext_sales_price, ws_net_profit@7 as ws_net_profit, wr_return_amt@0 as wr_return_amt, wr_net_loss@1 as wr_net_loss] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@1, ws_order_number@4)], projection=[wr_return_amt@2, wr_net_loss@3, ws_sold_date_sk@4, ws_item_sk@5, ws_web_site_sk@6, ws_promo_sk@7, ws_ext_sales_price@9, ws_net_profit@10] + │ [Stage 17] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 15 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_site_id, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 16 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number, sr_return_amt, sr_net_loss], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@4], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_promo_sk, ss_ticket_number, ss_ext_sales_price, ss_net_profit], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] - │ RepartitionExec: partitioning=Hash([cp_catalog_page_id@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[cp_catalog_page_id@4 as cp_catalog_page_id], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(coalesce(catalog_returns.cr_return_amount,Int64(0))), sum(catalog_sales.cs_net_profit - coalesce(catalog_returns.cr_net_loss,Int64(0)))] - │ ProjectionExec: expr=[CAST(cr_return_amount@2 AS Decimal128(22, 2)) as __common_expr_3, CAST(cr_net_loss@3 AS Decimal128(22, 2)) as __common_expr_4, cs_ext_sales_price@0 as cs_ext_sales_price, cs_net_profit@1 as cs_net_profit, cp_catalog_page_id@4 as cp_catalog_page_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, cs_promo_sk@0)], projection=[cs_ext_sales_price@3, cs_net_profit@4, cr_return_amount@5, cr_net_loss@6, cp_catalog_page_id@7] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@0)], projection=[cs_promo_sk@2, cs_ext_sales_price@3, cs_net_profit@4, cr_return_amount@5, cr_net_loss@6, cp_catalog_page_id@7] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_promo_sk@2 as cs_promo_sk, cs_ext_sales_price@3 as cs_ext_sales_price, cs_net_profit@4 as cs_net_profit, cr_return_amount@5 as cr_return_amount, cr_net_loss@6 as cr_net_loss, cp_catalog_page_id@0 as cp_catalog_page_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(catalog_page.cp_catalog_page_sk AS Float64)@2, cs_catalog_page_sk@0)], projection=[cp_catalog_page_id@1, cs_item_sk@4, cs_promo_sk@5, cs_ext_sales_price@6, cs_net_profit@7, cr_return_amount@8, cr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_catalog_page_sk@3, cs_item_sk@4, cs_promo_sk@5, cs_ext_sales_price@6, cs_net_profit@7, cr_return_amount@8, cr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_catalog_page_sk@3 as cs_catalog_page_sk, cs_item_sk@4 as cs_item_sk, cs_promo_sk@5 as cs_promo_sk, cs_ext_sales_price@6 as cs_ext_sales_price, cs_net_profit@7 as cs_net_profit, cr_return_amount@0 as cr_return_amount, cr_net_loss@1 as cr_net_loss] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_item_sk@0, cs_item_sk@2), (cr_order_number@1, cs_order_number@4)], projection=[cr_return_amount@2, cr_net_loss@3, cs_sold_date_sk@4, cs_catalog_page_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_ext_sales_price@9, cs_net_profit@10] - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[cp_catalog_page_sk@0 as cp_catalog_page_sk, cp_catalog_page_id@1 as cp_catalog_page_id, CAST(cp_catalog_page_sk@0 AS Float64) as CAST(catalog_page.cp_catalog_page_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-3.parquet]]}, projection=[cp_catalog_page_sk, cp_catalog_page_id], file_type=parquet + ┌───── Stage 17 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([wr_item_sk@0, wr_order_number@1], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number, wr_return_amt, wr_net_loss], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_return_amount, cr_net_loss], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([cs_item_sk@2, cs_order_number@4], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_catalog_page_sk, cs_item_sk, cs_promo_sk, cs_order_number, cs_ext_sales_price, cs_net_profit], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([web_site_id@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[web_site_id@4 as web_site_id], aggr=[sum(web_sales.ws_ext_sales_price), sum(coalesce(web_returns.wr_return_amt,Int64(0))), sum(web_sales.ws_net_profit - coalesce(web_returns.wr_net_loss,Int64(0)))] - │ ProjectionExec: expr=[CAST(wr_return_amt@2 AS Decimal128(22, 2)) as __common_expr_5, CAST(wr_net_loss@3 AS Decimal128(22, 2)) as __common_expr_6, ws_ext_sales_price@0 as ws_ext_sales_price, ws_net_profit@1 as ws_net_profit, web_site_id@4 as web_site_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_promo_sk@0, CAST(promotion.p_promo_sk AS Float64)@1)], projection=[ws_ext_sales_price@1, ws_net_profit@2, wr_return_amt@3, wr_net_loss@4, web_site_id@5] - │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] - └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_promo_sk@1, ws_ext_sales_price@2, ws_net_profit@3, wr_return_amt@4, wr_net_loss@5, web_site_id@6] - │ CoalescePartitionsExec - │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2 AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] + ┌───── Stage 18 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ws_item_sk@1, ws_order_number@4], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_web_site_sk, ws_promo_sk, ws_order_number, ws_ext_sales_price, ws_net_profit], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_promo_sk@2 as ws_promo_sk, ws_ext_sales_price@3 as ws_ext_sales_price, ws_net_profit@4 as ws_net_profit, wr_return_amt@5 as wr_return_amt, wr_net_loss@6 as wr_net_loss, web_site_id@0 as web_site_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@2, ws_web_site_sk@1)], projection=[web_site_id@1, ws_item_sk@3, ws_promo_sk@5, ws_ext_sales_price@6, ws_net_profit@7, wr_return_amt@8, wr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_web_site_sk@4, ws_promo_sk@5, ws_ext_sales_price@6, ws_net_profit@7, wr_return_amt@8, wr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_web_site_sk@4 as ws_web_site_sk, ws_promo_sk@5 as ws_promo_sk, ws_ext_sales_price@6 as ws_ext_sales_price, ws_net_profit@7 as ws_net_profit, wr_return_amt@0 as wr_return_amt, wr_net_loss@1 as wr_net_loss] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@1, ws_order_number@4)], projection=[wr_return_amt@2, wr_net_loss@3, ws_sold_date_sk@4, ws_item_sk@5, ws_web_site_sk@6, ws_promo_sk@7, ws_ext_sales_price@9, ws_net_profit@10] - │ [Stage 17] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_site_id, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([wr_item_sk@0, wr_order_number@1], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number, wr_return_amt, wr_net_loss], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ws_item_sk@1, ws_order_number@4], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_web_site_sk, ws_promo_sk, ws_order_number, ws_ext_sales_price, ws_net_profit], file_type=parquet - └────────────────────────────────────────────────── "); Ok(()) } @@ -8882,103 +8467,101 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_customer_id@0 ASC NULLS LAST, c_salutation@1 ASC NULLS LAST, c_first_name@2 ASC NULLS LAST, c_last_name@3 ASC NULLS LAST, ca_street_number@4 ASC NULLS LAST, ca_street_name@5 ASC NULLS LAST, ca_street_type@6 ASC NULLS LAST, ca_suite_number@7 ASC NULLS LAST, ca_city@8 ASC NULLS LAST, ca_county@9 ASC NULLS LAST, ca_state@10 ASC NULLS LAST, ca_zip@11 ASC NULLS LAST, ca_country@12 ASC NULLS LAST, ca_gmt_offset@13 ASC NULLS LAST, ca_location_type@14 ASC NULLS LAST, ctr_total_return@15 ASC NULLS LAST], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[c_customer_id@0 ASC NULLS LAST, c_salutation@1 ASC NULLS LAST, c_first_name@2 ASC NULLS LAST, c_last_name@3 ASC NULLS LAST, ca_street_number@4 ASC NULLS LAST, ca_street_name@5 ASC NULLS LAST, ca_street_type@6 ASC NULLS LAST, ca_suite_number@7 ASC NULLS LAST, ca_city@8 ASC NULLS LAST, ca_county@9 ASC NULLS LAST, ca_zip@11 ASC NULLS LAST, ca_country@12 ASC NULLS LAST, ca_gmt_offset@13 ASC NULLS LAST, ca_location_type@14 ASC NULLS LAST, ctr_total_return@15 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[c_customer_id@12 as c_customer_id, c_salutation@13 as c_salutation, c_first_name@14 as c_first_name, c_last_name@15 as c_last_name, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_street_type@3 as ca_street_type, ca_suite_number@4 as ca_suite_number, ca_city@5 as ca_city, ca_county@6 as ca_county, ca_state@7 as ca_state, ca_zip@8 as ca_zip, ca_country@9 as ca_country, ca_gmt_offset@10 as ca_gmt_offset, ca_location_type@11 as ca_location_type, ctr_total_return@0 as ctr_total_return] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_state@0, ctr_state@1)], filter=CAST(ctr_total_return@0 AS Decimal128(30, 15)) > avg(ctr2.ctr_total_return) * Float64(1.2)@1, projection=[ctr_total_return@1, ca_street_number@2, ca_street_name@3, ca_street_type@4, ca_suite_number@5, ca_city@6, ca_county@7, ca_state@8, ca_zip@9, ca_country@10, ca_gmt_offset@11, ca_location_type@12, c_customer_id@13, c_salutation@14, c_first_name@15, c_last_name@16] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ ProjectionExec: expr=[CAST(CAST(avg(ctr2.ctr_total_return)@1 AS Float64) * 1.2 AS Decimal128(30, 15)) as avg(ctr2.ctr_total_return) * Float64(1.2), ctr_state@0 as ctr_state] │ AggregateExec: mode=FinalPartitioned, gby=[ctr_state@0 as ctr_state], aggr=[avg(ctr2.ctr_total_return)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ctr_state@11 as ctr_state, ctr_total_return@12 as ctr_total_return, ca_street_number@0 as ca_street_number, ca_street_name@1 as ca_street_name, ca_street_type@2 as ca_street_type, ca_suite_number@3 as ca_suite_number, ca_city@4 as ca_city, ca_county@5 as ca_county, ca_state@6 as ca_state, ca_zip@7 as ca_zip, ca_country@8 as ca_country, ca_gmt_offset@9 as ca_gmt_offset, ca_location_type@10 as ca_location_type, c_customer_id@13 as c_customer_id, c_salutation@14 as c_salutation, c_first_name@15 as c_first_name, c_last_name@16 as c_last_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[ca_street_number@1, ca_street_name@2, ca_street_type@3, ca_suite_number@4, ca_city@5, ca_county@6, ca_state@7, ca_zip@8, ca_country@9, ca_gmt_offset@10, ca_location_type@11, ctr_state@12, ctr_total_return@13, c_customer_id@14, c_salutation@16, c_first_name@17, c_last_name@18] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ctr_state@1, ctr_total_return@2, c_customer_id@4, c_current_addr_sk@5, c_salutation@6, c_first_name@7, c_last_name@8] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_current_addr_sk@2 as c_current_addr_sk, c_salutation@3 as c_salutation, c_first_name@4 as c_first_name, c_last_name@5 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_current_addr_sk, c_salutation, c_first_name, c_last_name], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: ca_state@7 = GA │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_street_number, ca_street_name, ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset, ca_location_type], file_type=parquet, predicate=ca_state@8 = GA, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1, required_guarantees=[ca_state in (GA)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[cr_returning_customer_sk@0 as ctr_customer_sk, ca_state@1 as ctr_state, sum(catalog_returns.cr_return_amt_inc_tax)@2 as ctr_total_return] │ AggregateExec: mode=FinalPartitioned, gby=[cr_returning_customer_sk@0 as cr_returning_customer_sk, ca_state@1 as ca_state], aggr=[sum(catalog_returns.cr_return_amt_inc_tax)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cr_returning_customer_sk@0, ca_state@1], 6), input_partitions=3 + ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] t3:[p0..p8] + │ RepartitionExec: partitioning=Hash([cr_returning_customer_sk@0, ca_state@1], 9), input_partitions=3 │ AggregateExec: mode=Partial, gby=[cr_returning_customer_sk@0 as cr_returning_customer_sk, ca_state@2 as ca_state], aggr=[sum(catalog_returns.cr_return_amt_inc_tax)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returning_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[cr_returning_customer_sk@0, cr_return_amt_inc_tax@2, ca_state@4] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 3 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[cr_returning_customer_sk@2, cr_returning_addr_sk@3, cr_return_amt_inc_tax@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_returning_addr_sk, cr_return_amt_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ RepartitionExec: partitioning=Hash([ctr_state@0], 6), input_partitions=3 + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ctr_state@0], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ctr_state@0 as ctr_state], aggr=[avg(ctr2.ctr_total_return)] │ ProjectionExec: expr=[ca_state@1 as ctr_state, sum(catalog_returns.cr_return_amt_inc_tax)@2 as ctr_total_return] │ AggregateExec: mode=FinalPartitioned, gby=[cr_returning_customer_sk@0 as cr_returning_customer_sk, ca_state@1 as ca_state], aggr=[sum(catalog_returns.cr_return_amt_inc_tax)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cr_returning_customer_sk@0, ca_state@1], 6), input_partitions=3 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cr_returning_customer_sk@0, ca_state@1], 12), input_partitions=3 │ AggregateExec: mode=Partial, gby=[cr_returning_customer_sk@0 as cr_returning_customer_sk, ca_state@2 as ca_state], aggr=[sum(catalog_returns.cr_return_amt_inc_tax)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returning_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[cr_returning_customer_sk@0, cr_return_amt_inc_tax@2, ca_state@4] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 8 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[cr_returning_customer_sk@2, cr_returning_addr_sk@3, cr_return_amt_inc_tax@4] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_returning_addr_sk, cr_return_amt_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── "); Ok(()) @@ -8989,48 +8572,45 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_current_price@2], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@0)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_current_price@2], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@0)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ FilterExec: d_date@1 >= 2000-05-25 AND d_date@1 <= 2000-07-24, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-05-25 AND d_date@2 <= 2000-07-24 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-05-25 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-07-24, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3, inv_date_sk@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ FilterExec: d_date@1 >= 2000-05-25 AND d_date@1 <= 2000-07-24, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-05-25 AND d_date@2 <= 2000-07-24 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-05-25 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-07-24, required_guarantees=[] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@3 >= 100 AND inv_quantity_on_hand@3 <= 500 AND DynamicFilter [ empty ], pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3, inv_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@3 >= 100 AND inv_quantity_on_hand@3 <= 500 AND DynamicFilter [ empty ], pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@3 >= Some(6200),4,2 AND i_current_price@3 <= Some(9200),4,2 AND i_manufact_id@4 IN (SET) ([129, 270, 821, 423]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_manufact_id], file_type=parquet, predicate=i_current_price@5 >= Some(6200),4,2 AND i_current_price@5 <= Some(9200),4,2 AND i_manufact_id@13 IN (SET) ([129, 270, 821, 423]) AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(6200),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(9200),4,2 AND (i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 129 AND 129 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 270 AND 270 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 821 AND 821 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 423 AND 423 <= i_manufact_id_max@5), required_guarantees=[i_manufact_id in (129, 270, 423, 821)] - └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_current_price@3 >= Some(6200),4,2 AND i_current_price@3 <= Some(9200),4,2 AND i_manufact_id@4 IN (SET) ([129, 270, 821, 423]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_manufact_id], file_type=parquet, predicate=i_current_price@5 >= Some(6200),4,2 AND i_current_price@5 <= Some(9200),4,2 AND i_manufact_id@13 IN (SET) ([129, 270, 821, 423]) AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(6200),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(9200),4,2 AND (i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 129 AND 129 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 270 AND 270 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 821 AND 821 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 423 AND 423 <= i_manufact_id_max@5), required_guarantees=[i_manufact_id in (129, 270, 423, 821)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -9040,31 +8620,31 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [item_id@0 ASC, sr_item_qty@1 ASC], fetch=100 - │ [Stage 17] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 15] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ SortExec: TopK(fetch=100), expr=[item_id@0 ASC, sr_item_qty@1 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[item_id@1 as item_id, sr_item_qty@2 as sr_item_qty, sr_item_qty@2 / __common_expr_7@0 / 3 * 100 as sr_dev, cr_item_qty@3 as cr_item_qty, cr_item_qty@3 / __common_expr_7@0 / 3 * 100 as cr_dev, wr_item_qty@4 as wr_item_qty, wr_item_qty@4 / __common_expr_7@0 / 3 * 100 as wr_dev, __common_expr_7@0 / 3 as average] │ ProjectionExec: expr=[sr_item_qty@2 + cr_item_qty@3 + wr_item_qty@0 as __common_expr_7, item_id@1 as item_id, sr_item_qty@2 as sr_item_qty, cr_item_qty@3 as cr_item_qty, wr_item_qty@0 as wr_item_qty] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], projection=[wr_item_qty@1, item_id@2, sr_item_qty@3, cr_item_qty@4] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], projection=[item_id@0, sr_item_qty@1, cr_item_qty@3] │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(catalog_returns.cr_return_quantity)@1 as cr_item_qty] │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(catalog_returns.cr_return_quantity)] - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(web_returns.wr_return_quantity)@1 as wr_item_qty] │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_returns.wr_return_quantity)] │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_returns.wr_return_quantity)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[wr_return_quantity@0, i_item_id@1] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] │ CoalescePartitionsExec │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 @@ -9074,126 +8654,111 @@ mod tests { │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_returned_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[wr_return_quantity@1, i_item_id@2, d_date@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] + │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 │ ProjectionExec: expr=[wr_returned_date_sk@1 as wr_returned_date_sk, wr_return_quantity@2 as wr_return_quantity, i_item_id@0 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, wr_item_sk@1)], projection=[i_item_id@1, wr_returned_date_sk@2, wr_return_quantity@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=1 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_returned_date_sk, wr_item_sk, wr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(store_returns.sr_return_quantity)@1 as sr_item_qty] │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(store_returns.sr_return_quantity)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_returns.sr_return_quantity)] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_date@0, d_date@2)], projection=[sr_return_quantity@0, i_item_id@1] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[sr_return_quantity@1 as sr_return_quantity, i_item_id@2 as i_item_id, d_date@0 as d_date] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, sr_returned_date_sk@0)], projection=[d_date@1, sr_return_quantity@4, i_item_id@5] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, projection=[d_week_seq@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, projection=[d_week_seq@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 12), input_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_returned_date_sk@0], 9), input_partitions=2 + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_returned_date_sk@0], 12), input_partitions=2 │ ProjectionExec: expr=[sr_returned_date_sk@1 as sr_returned_date_sk, sr_return_quantity@2 as sr_return_quantity, i_item_id@0 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, sr_item_sk@1)], projection=[i_item_id@1, sr_returned_date_sk@2, sr_return_quantity@4] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=2 + ┌───── Stage 14 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_returns.cr_return_quantity)] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_date@0, d_date@2)], projection=[cr_return_quantity@0, i_item_id@1] │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[cr_return_quantity@1 as cr_return_quantity, i_item_id@2 as i_item_id, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[d_date@1, cr_return_quantity@3, i_item_id@4] │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ ProjectionExec: expr=[cr_returned_date_sk@1 as cr_returned_date_sk, cr_return_quantity@2 as cr_return_quantity, i_item_id@0 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cr_item_sk@1)], projection=[i_item_id@1, cr_returned_date_sk@2, cr_return_quantity@4] │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_item_sk, cr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, projection=[d_week_seq@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, projection=[d_week_seq@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet + ┌───── Stage 12 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + ┌───── Stage 13 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -9205,60 +8770,57 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[customer_id@0 as customer_id, customername@1 as customername] │ SortPreservingMergeExec: [c_customer_id@2 ASC], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC], preserve_partitioning=[true] │ ProjectionExec: expr=[c_customer_id@0 as customer_id, concat(concat(CASE WHEN c_last_name@2 IS NOT NULL THEN c_last_name@2 ELSE END, , ), CASE WHEN c_first_name@1 IS NOT NULL THEN c_first_name@1 ELSE END) as customername, c_customer_id@0 as c_customer_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@4, sr_cdemo_sk@0)], projection=[c_customer_id@0, c_first_name@1, c_last_name@2] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_cdemo_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 5 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, cd_demo_sk@3 as cd_demo_sk, CAST(cd_demo_sk@3 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@4)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, cd_demo_sk@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, cd_demo_sk@4 as cd_demo_sk, hd_income_band_sk@0 as hd_income_band_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@2, c_current_hdemo_sk@1)], projection=[hd_income_band_sk@1, c_customer_id@3, c_first_name@5, c_last_name@6, cd_demo_sk@7] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@1)], projection=[c_customer_id@0, c_current_hdemo_sk@2, c_first_name@3, c_last_name@4, cd_demo_sk@5] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: ib_lower_bound@1 >= 38128 AND ib_upper_bound@2 <= 88128, projection=[ib_income_band_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk, ib_lower_bound, ib_upper_bound], file_type=parquet, predicate=ib_lower_bound@1 >= 38128 AND ib_upper_bound@2 <= 88128, pruning_predicate=ib_lower_bound_null_count@1 != row_count@2 AND ib_lower_bound_max@0 >= 38128 AND ib_upper_bound_null_count@4 != row_count@2 AND ib_upper_bound_min@3 <= 88128, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk, ib_lower_bound, ib_upper_bound], file_type=parquet, predicate=ib_lower_bound@1 >= 38128 AND ib_upper_bound@2 <= 88128, pruning_predicate=ib_lower_bound_null_count@1 != row_count@2 AND ib_lower_bound_max@0 >= 38128 AND ib_upper_bound_null_count@4 != row_count@2 AND ib_upper_bound_min@3 <= 88128, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[c_customer_id@1, c_current_cdemo_sk@2, c_current_hdemo_sk@3, c_first_name@5, c_last_name@6] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_id, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_name, c_last_name], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: ca_city@1 = Edgewood, projection=[ca_address_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_city], file_type=parquet, predicate=ca_city@6 = Edgewood, pruning_predicate=ca_city_null_count@2 != row_count@3 AND ca_city_min@0 <= Edgewood AND Edgewood <= ca_city_max@1, required_guarantees=[ca_city in (Edgewood)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_city], file_type=parquet, predicate=ca_city@6 = Edgewood, pruning_predicate=ca_city_null_count@2 != row_count@3 AND ca_city_min@0 <= Edgewood AND Edgewood <= ca_city_max@1, required_guarantees=[ca_city in (Edgewood)] └────────────────────────────────────────────────── "); Ok(()) @@ -9269,90 +8831,85 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [substr(reason.r_reason_desc,Int64(1),Int64(20))@0 ASC NULLS LAST, avg1@1 ASC NULLS LAST, avg2@2 ASC NULLS LAST, avg(web_returns.wr_fee)@3 ASC NULLS LAST], fetch=100 - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[substr(reason.r_reason_desc,Int64(1),Int64(20))@0 ASC NULLS LAST, avg1@1 ASC NULLS LAST, avg2@2 ASC NULLS LAST, avg(web_returns.wr_fee)@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[substr(r_reason_desc@0, 1, 20) as substr(reason.r_reason_desc,Int64(1),Int64(20)), avg(web_sales.ws_quantity)@1 as avg1, avg(web_returns.wr_refunded_cash)@2 as avg2, avg(web_returns.wr_fee)@3 as avg(web_returns.wr_fee)] + │ AggregateExec: mode=FinalPartitioned, gby=[r_reason_desc@0 as r_reason_desc], aggr=[avg(web_sales.ws_quantity), avg(web_returns.wr_refunded_cash), avg(web_returns.wr_fee)] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[substr(reason.r_reason_desc,Int64(1),Int64(20))@0 ASC NULLS LAST, avg1@1 ASC NULLS LAST, avg2@2 ASC NULLS LAST, avg(web_returns.wr_fee)@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[substr(r_reason_desc@0, 1, 20) as substr(reason.r_reason_desc,Int64(1),Int64(20)), avg(web_sales.ws_quantity)@1 as avg1, avg(web_returns.wr_refunded_cash)@2 as avg2, avg(web_returns.wr_fee)@3 as avg(web_returns.wr_fee)] - │ AggregateExec: mode=FinalPartitioned, gby=[r_reason_desc@0 as r_reason_desc], aggr=[avg(web_sales.ws_quantity), avg(web_returns.wr_refunded_cash), avg(web_returns.wr_fee)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([r_reason_desc@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[r_reason_desc@3 as r_reason_desc], aggr=[avg(web_sales.ws_quantity), avg(web_returns.wr_refunded_cash), avg(web_returns.wr_fee)] - │ ProjectionExec: expr=[ws_quantity@1 as ws_quantity, wr_fee@2 as wr_fee, wr_refunded_cash@3 as wr_refunded_cash, r_reason_desc@0 as r_reason_desc] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(reason.r_reason_sk AS Float64)@2, wr_reason_sk@1)], projection=[r_reason_desc@1, ws_quantity@3, wr_fee@5, wr_refunded_cash@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ws_quantity@1, wr_reason_sk@2, wr_fee@3, wr_refunded_cash@4] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([r_reason_desc@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[r_reason_desc@3 as r_reason_desc], aggr=[avg(web_sales.ws_quantity), avg(web_returns.wr_refunded_cash), avg(web_returns.wr_fee)] + │ ProjectionExec: expr=[ws_quantity@1 as ws_quantity, wr_fee@2 as wr_fee, wr_refunded_cash@3 as wr_refunded_cash, r_reason_desc@0 as r_reason_desc] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(reason.r_reason_sk AS Float64)@2, wr_reason_sk@1)], projection=[r_reason_desc@1, ws_quantity@3, wr_fee@5, wr_refunded_cash@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ws_quantity@1, wr_reason_sk@2, wr_fee@3, wr_refunded_cash@4] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@6 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk, r_reason_desc, CAST(r_reason_sk@0 AS Float64) as CAST(reason.r_reason_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_refunded_addr_sk@3, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = IN OR ca_state@1 = OH OR ca_state@1 = NJ) AND ws_net_profit@0 >= Some(10000),7,2 AND ws_net_profit@0 <= Some(20000),7,2 OR (ca_state@1 = WI OR ca_state@1 = CT OR ca_state@1 = KY) AND ws_net_profit@0 >= Some(15000),7,2 AND ws_net_profit@0 <= Some(30000),7,2 OR (ca_state@1 = LA OR ca_state@1 = IA OR ca_state@1 = AR) AND ws_net_profit@0 >= Some(5000),7,2 AND ws_net_profit@0 <= Some(25000),7,2, projection=[ws_sold_date_sk@0, ws_quantity@1, wr_reason_sk@4, wr_fee@5, wr_refunded_cash@6] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: (ca_state@1 = IN OR ca_state@1 = OH OR ca_state@1 = NJ OR ca_state@1 = WI OR ca_state@1 = CT OR ca_state@1 = KY OR ca_state@1 = LA OR ca_state@1 = IA OR ca_state@1 = AR) AND ca_state@1 IN (SET) ([IN, OH, NJ, WI, CT, KY, LA, IA, AR]) AND ca_country@2 = United States, projection=[ca_address_sk@0, ca_state@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=(ca_state@8 = IN OR ca_state@8 = OH OR ca_state@8 = NJ OR ca_state@8 = WI OR ca_state@8 = CT OR ca_state@8 = KY OR ca_state@8 = LA OR ca_state@8 = IA OR ca_state@8 = AR) AND ca_state@8 IN (SET) ([IN, OH, NJ, WI, CT, KY, LA, IA, AR]) AND ca_country@10 = United States, pruning_predicate=(ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NJ AND NJ <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= WI AND WI <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CT AND CT <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= LA AND LA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IA AND IA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= AR AND AR <= ca_state_max@1) AND (ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NJ AND NJ <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= WI AND WI <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CT AND CT <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= LA AND LA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IA AND IA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= AR AND AR <= ca_state_max@1) AND ca_country_null_count@6 != row_count@3 AND ca_country_min@4 <= United States AND United States <= ca_country_max@5, required_guarantees=[ca_country in (United States), ca_state in (AR, CT, IA, IN, KY, LA, NJ, OH, WI)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk, r_reason_desc, CAST(r_reason_sk@0 AS Float64) as CAST(reason.r_reason_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_refunded_addr_sk@3, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = IN OR ca_state@1 = OH OR ca_state@1 = NJ) AND ws_net_profit@0 >= Some(10000),7,2 AND ws_net_profit@0 <= Some(20000),7,2 OR (ca_state@1 = WI OR ca_state@1 = CT OR ca_state@1 = KY) AND ws_net_profit@0 >= Some(15000),7,2 AND ws_net_profit@0 <= Some(30000),7,2 OR (ca_state@1 = LA OR ca_state@1 = IA OR ca_state@1 = AR) AND ws_net_profit@0 >= Some(5000),7,2 AND ws_net_profit@0 <= Some(25000),7,2, projection=[ws_sold_date_sk@0, ws_quantity@1, wr_reason_sk@4, wr_fee@5, wr_refunded_cash@6] + ┌───── Stage 7 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_returning_cdemo_sk@4, CAST(cd2.cd_demo_sk AS Float64)@3), (cd_marital_status@8, cd_marital_status@1), (cd_education_status@9, cd_education_status@2)], projection=[ws_sold_date_sk@0, ws_quantity@1, ws_net_profit@2, wr_refunded_addr_sk@3, wr_reason_sk@5, wr_fee@6, wr_refunded_cash@7] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: (ca_state@1 = IN OR ca_state@1 = OH OR ca_state@1 = NJ OR ca_state@1 = WI OR ca_state@1 = CT OR ca_state@1 = KY OR ca_state@1 = LA OR ca_state@1 = IA OR ca_state@1 = AR) AND ca_state@1 IN (SET) ([IN, OH, NJ, WI, CT, KY, LA, IA, AR]) AND ca_country@2 = United States, projection=[ca_address_sk@0, ca_state@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=(ca_state@8 = IN OR ca_state@8 = OH OR ca_state@8 = NJ OR ca_state@8 = WI OR ca_state@8 = CT OR ca_state@8 = KY OR ca_state@8 = LA OR ca_state@8 = IA OR ca_state@8 = AR) AND ca_state@8 IN (SET) ([IN, OH, NJ, WI, CT, KY, LA, IA, AR]) AND ca_country@10 = United States, pruning_predicate=(ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NJ AND NJ <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= WI AND WI <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CT AND CT <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= LA AND LA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IA AND IA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= AR AND AR <= ca_state_max@1) AND (ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NJ AND NJ <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= WI AND WI <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CT AND CT <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= LA AND LA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IA AND IA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= AR AND AR <= ca_state_max@1) AND ca_country_null_count@6 != row_count@3 AND ca_country_min@4 <= United States AND United States <= ca_country_max@5, required_guarantees=[ca_country in (United States), ca_state in (AR, CT, IA, IN, KY, LA, NJ, OH, WI)] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_returning_cdemo_sk@4, CAST(cd2.cd_demo_sk AS Float64)@3), (cd_marital_status@8, cd_marital_status@1), (cd_education_status@9, cd_education_status@2)], projection=[ws_sold_date_sk@0, ws_quantity@1, ws_net_profit@2, wr_refunded_addr_sk@3, wr_reason_sk@5, wr_fee@6, wr_refunded_cash@7] + ┌───── Stage 6 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_refunded_cdemo_sk@4, CAST(cd1.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ws_sales_price@0 >= Some(10000),5,2 AND ws_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ws_sales_price@0 >= Some(5000),5,2 AND ws_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ws_sales_price@0 >= Some(15000),5,2 AND ws_sales_price@0 <= Some(20000),5,2, projection=[ws_sold_date_sk@0, ws_quantity@1, ws_net_profit@3, wr_refunded_addr_sk@5, wr_returning_cdemo_sk@6, wr_reason_sk@7, wr_fee@8, wr_refunded_cash@9, cd_marital_status@11, cd_education_status@12] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)] + │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Advanced Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College OR cd_marital_status@2 = W AND cd_education_status@3 = 2 yr Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, Advanced Degree, College), cd_marital_status in (M, S, W)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_refunded_cdemo_sk@4, CAST(cd1.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ws_sales_price@0 >= Some(10000),5,2 AND ws_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ws_sales_price@0 >= Some(5000),5,2 AND ws_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ws_sales_price@0 >= Some(15000),5,2 AND ws_sales_price@0 <= Some(20000),5,2, projection=[ws_sold_date_sk@0, ws_quantity@1, ws_net_profit@3, wr_refunded_addr_sk@5, wr_returning_cdemo_sk@6, wr_reason_sk@7, wr_fee@8, wr_refunded_cash@9, cd_marital_status@11, cd_education_status@12] + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@1)], projection=[ws_sold_date_sk@2, ws_quantity@4, ws_sales_price@5, ws_net_profit@6, wr_refunded_cdemo_sk@7, wr_refunded_addr_sk@8, wr_returning_cdemo_sk@9, wr_reason_sk@10, wr_fee@11, wr_refunded_cash@12] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)] - │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Advanced Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College OR cd_marital_status@2 = W AND cd_education_status@3 = 2 yr Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, Advanced Degree, College), cd_marital_status in (M, S, W)] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[ws_sold_date_sk@6 as ws_sold_date_sk, ws_web_page_sk@7 as ws_web_page_sk, ws_quantity@8 as ws_quantity, ws_sales_price@9 as ws_sales_price, ws_net_profit@10 as ws_net_profit, wr_refunded_cdemo_sk@0 as wr_refunded_cdemo_sk, wr_refunded_addr_sk@1 as wr_refunded_addr_sk, wr_returning_cdemo_sk@2 as wr_returning_cdemo_sk, wr_reason_sk@3 as wr_reason_sk, wr_fee@4 as wr_fee, wr_refunded_cash@5 as wr_refunded_cash] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@5, ws_order_number@3)], projection=[wr_refunded_cdemo_sk@1, wr_refunded_addr_sk@2, wr_returning_cdemo_sk@3, wr_reason_sk@4, wr_fee@6, wr_refunded_cash@7, ws_sold_date_sk@8, ws_web_page_sk@10, ws_quantity@12, ws_sales_price@13, ws_net_profit@14] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@1)], projection=[ws_sold_date_sk@2, ws_quantity@4, ws_sales_price@5, ws_net_profit@6, wr_refunded_cdemo_sk@7, wr_refunded_addr_sk@8, wr_returning_cdemo_sk@9, wr_reason_sk@10, wr_fee@11, wr_refunded_cash@12] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@6 as ws_sold_date_sk, ws_web_page_sk@7 as ws_web_page_sk, ws_quantity@8 as ws_quantity, ws_sales_price@9 as ws_sales_price, ws_net_profit@10 as ws_net_profit, wr_refunded_cdemo_sk@0 as wr_refunded_cdemo_sk, wr_refunded_addr_sk@1 as wr_refunded_addr_sk, wr_returning_cdemo_sk@2 as wr_returning_cdemo_sk, wr_reason_sk@3 as wr_reason_sk, wr_fee@4 as wr_fee, wr_refunded_cash@5 as wr_refunded_cash] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@5, ws_order_number@3)], projection=[wr_refunded_cdemo_sk@1, wr_refunded_addr_sk@2, wr_returning_cdemo_sk@3, wr_reason_sk@4, wr_fee@6, wr_refunded_cash@7, ws_sold_date_sk@8, ws_web_page_sk@10, ws_quantity@12, ws_sales_price@13, ws_net_profit@14] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([wr_item_sk@0, wr_order_number@5], 12), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_refunded_cdemo_sk, wr_refunded_addr_sk, wr_returning_cdemo_sk, wr_reason_sk, wr_order_number, wr_fee, wr_refunded_cash], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ws_item_sk@1, ws_order_number@3], 12), input_partitions=2 + │ FilterExec: (ws_net_profit@6 >= Some(10000),7,2 AND ws_net_profit@6 <= Some(20000),7,2 OR ws_net_profit@6 >= Some(15000),7,2 AND ws_net_profit@6 <= Some(30000),7,2 OR ws_net_profit@6 >= Some(5000),7,2 AND ws_net_profit@6 <= Some(25000),7,2) AND (ws_sales_price@5 >= Some(10000),5,2 AND ws_sales_price@5 <= Some(15000),5,2 OR ws_sales_price@5 >= Some(5000),5,2 AND ws_sales_price@5 <= Some(10000),5,2 OR ws_sales_price@5 >= Some(15000),5,2 AND ws_sales_price@5 <= Some(20000),5,2) + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_web_page_sk, ws_order_number, ws_quantity, ws_sales_price, ws_net_profit], file_type=parquet, predicate=(ws_net_profit@33 >= Some(10000),7,2 AND ws_net_profit@33 <= Some(20000),7,2 OR ws_net_profit@33 >= Some(15000),7,2 AND ws_net_profit@33 <= Some(30000),7,2 OR ws_net_profit@33 >= Some(5000),7,2 AND ws_net_profit@33 <= Some(25000),7,2) AND (ws_sales_price@21 >= Some(10000),5,2 AND ws_sales_price@21 <= Some(15000),5,2 OR ws_sales_price@21 >= Some(5000),5,2 AND ws_sales_price@21 <= Some(10000),5,2 OR ws_sales_price@21 >= Some(15000),5,2 AND ws_sales_price@21 <= Some(20000),5,2) AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=(ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 >= Some(10000),7,2 AND ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_min@3 <= Some(20000),7,2 OR ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 >= Some(15000),7,2 AND ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_min@3 <= Some(30000),7,2 OR ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 >= Some(5000),7,2 AND ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_min@3 <= Some(25000),7,2) AND (ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_max@4 >= Some(10000),5,2 AND ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_min@6 <= Some(15000),5,2 OR ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_max@4 >= Some(5000),5,2 AND ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_min@6 <= Some(10000),5,2 OR ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_max@4 >= Some(15000),5,2 AND ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ RepartitionExec: partitioning=Hash([wr_item_sk@0, wr_order_number@5], 6), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_refunded_cdemo_sk, wr_refunded_addr_sk, wr_returning_cdemo_sk, wr_reason_sk, wr_order_number, wr_fee, wr_refunded_cash], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ws_item_sk@1, ws_order_number@3], 6), input_partitions=2 - │ FilterExec: (ws_net_profit@6 >= Some(10000),7,2 AND ws_net_profit@6 <= Some(20000),7,2 OR ws_net_profit@6 >= Some(15000),7,2 AND ws_net_profit@6 <= Some(30000),7,2 OR ws_net_profit@6 >= Some(5000),7,2 AND ws_net_profit@6 <= Some(25000),7,2) AND (ws_sales_price@5 >= Some(10000),5,2 AND ws_sales_price@5 <= Some(15000),5,2 OR ws_sales_price@5 >= Some(5000),5,2 AND ws_sales_price@5 <= Some(10000),5,2 OR ws_sales_price@5 >= Some(15000),5,2 AND ws_sales_price@5 <= Some(20000),5,2) - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_web_page_sk, ws_order_number, ws_quantity, ws_sales_price, ws_net_profit], file_type=parquet, predicate=(ws_net_profit@33 >= Some(10000),7,2 AND ws_net_profit@33 <= Some(20000),7,2 OR ws_net_profit@33 >= Some(15000),7,2 AND ws_net_profit@33 <= Some(30000),7,2 OR ws_net_profit@33 >= Some(5000),7,2 AND ws_net_profit@33 <= Some(25000),7,2) AND (ws_sales_price@21 >= Some(10000),5,2 AND ws_sales_price@21 <= Some(15000),5,2 OR ws_sales_price@21 >= Some(5000),5,2 AND ws_sales_price@21 <= Some(10000),5,2 OR ws_sales_price@21 >= Some(15000),5,2 AND ws_sales_price@21 <= Some(20000),5,2) AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=(ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 >= Some(10000),7,2 AND ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_min@3 <= Some(20000),7,2 OR ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 >= Some(15000),7,2 AND ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_min@3 <= Some(30000),7,2 OR ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 >= Some(5000),7,2 AND ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_min@3 <= Some(25000),7,2) AND (ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_max@4 >= Some(10000),5,2 AND ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_min@6 <= Some(15000),5,2 OR ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_max@4 >= Some(5000),5,2 AND ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_min@6 <= Some(10000),5,2 OR ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_max@4 >= Some(15000),5,2 AND ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] - └────────────────────────────────────────────────── "); Ok(()) } @@ -9371,118 +8928,115 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=1 │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=1 │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[p0..p8] + │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ws_bill_customer_sk@1 as ws_bill_customer_sk, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_bill_customer_sk@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + ┌───── Stage 8 ── Tasks: t0:[p0..p8] + │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 6 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[cs_bill_customer_sk@1 as cs_bill_customer_sk, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_date@1, cs_bill_customer_sk@4] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 + ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] t3:[p0..p8] + │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 9), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 + ┌───── Stage 10 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 │ ProjectionExec: expr=[ss_customer_sk@1 as ss_customer_sk, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_customer_sk@4] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -9497,358 +9051,342 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as h12_to_12_30] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@3 as h9_30_to_10, h10_to_10_30@4 as h10_to_10_30, h10_30_to_11@5 as h10_30_to_11, h11_to_11_30@6 as h11_to_11_30, h11_30_to_12@0 as h11_30_to_12] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h11_30_to_12] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@3 as h9_30_to_10, h10_to_10_30@4 as h10_to_10_30, h10_30_to_11@5 as h10_30_to_11, h11_to_11_30@0 as h11_to_11_30] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h11_to_11_30] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@3 as h9_30_to_10, h10_to_10_30@4 as h10_to_10_30, h10_30_to_11@0 as h10_30_to_11] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h10_30_to_11] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@3 as h9_30_to_10, h10_to_10_30@0 as h10_to_10_30] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h10_to_10_30] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 20] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 20] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@0 as h9_30_to_10] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h9_30_to_10] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 24] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 24] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h8_30_to_9] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 28] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 28] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[count(Int64(1))@0 as h9_to_9_30] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 32] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 32] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 = 12 AND t_minute@2 < 30, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 12 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 12 AND 12 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (12)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 = 11 AND t_minute@2 >= 30, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 11 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 11 AND 11 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (11)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 9 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 10 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 = 11 AND t_minute@2 < 30, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 11 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 11 AND 11 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (11)] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 11 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 16 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 13 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 14 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 = 10 AND t_minute@2 >= 30, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 10 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 10 AND 10 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (10)] └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 15 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 20 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] │ CoalescePartitionsExec - │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 17 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 18 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 = 10 AND t_minute@2 < 30, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 10 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 10 AND 10 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (10)] └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 19 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 24 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] │ CoalescePartitionsExec - │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 21 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 22 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 = 9 AND t_minute@2 >= 30, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 9 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 9 AND 9 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (9)] └────────────────────────────────────────────────── - ┌───── Stage 23 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 23 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 28 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 28 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] │ CoalescePartitionsExec - │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 25 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 25 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 26 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 26 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 = 8 AND t_minute@2 >= 30, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 8 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 8 AND 8 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (8)] └────────────────────────────────────────────────── - ┌───── Stage 27 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 27 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 32 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 32 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 30] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 30] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] │ CoalescePartitionsExec - │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 29 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 29 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 30 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 30 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 = 9 AND t_minute@2 < 30, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 9 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 9 AND 9 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (9)] └────────────────────────────────────────────────── - ┌───── Stage 31 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 31 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── "); Ok(()) @@ -9864,50 +9402,44 @@ mod tests { │ FilterExec: CASE WHEN avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@7 != Some(0),19,6 THEN abs(sum(store_sales.ss_sales_price)@6 - avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@7) / avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@7 END > Some(1000000000),30,10 │ WindowAggExec: wdw=[avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(19, 6), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST, s_company_name@4 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@2, s_store_name@3, s_company_name@4], 3), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class, i_brand@2 as i_brand, s_store_name@3 as s_store_name, s_company_name@4 as s_company_name, d_moy@5 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@2, s_store_name@3, s_company_name@4], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class, i_brand@2 as i_brand, s_store_name@3 as s_store_name, s_company_name@4 as s_company_name, d_moy@5 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1, i_brand@2, s_store_name@3, s_company_name@4, d_moy@5], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_category@2 as i_category, i_class@1 as i_class, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] - │ ProjectionExec: expr=[i_brand@2 as i_brand, i_class@3 as i_class, i_category@4 as i_category, ss_sales_price@5 as ss_sales_price, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@3)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_class@5, i_category@6, ss_sales_price@8, d_moy@9] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_moy@0 as d_moy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@3)], projection=[d_moy@1, i_brand@3, i_class@4, i_category@5, ss_store_sk@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_class@2, i_category@3, ss_sold_date_sk@4, ss_store_sk@6, ss_sales_price@7] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1, i_brand@2, s_store_name@3, s_company_name@4, d_moy@5], 3), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_category@2 as i_category, i_class@1 as i_class, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] + │ ProjectionExec: expr=[i_brand@2 as i_brand, i_class@3 as i_class, i_category@4 as i_category, ss_sales_price@5 as ss_sales_price, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@3)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_class@5, i_category@6, ss_sales_price@8, d_moy@9] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ ProjectionExec: expr=[i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_moy@0 as d_moy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@3)], projection=[d_moy@1, i_brand@3, i_class@4, i_category@5, ss_store_sk@7, ss_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_class@2, i_category@3, ss_sold_date_sk@4, ss_store_sk@6, ss_sales_price@7] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_moy@1 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999, projection=[d_date_sk@0, d_moy@2] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: (i_category@3 = Books OR i_category@3 = Electronics OR i_category@3 = Sports) AND (i_class@2 = computers OR i_class@2 = stereo OR i_class@2 = football) OR (i_category@3 = Men OR i_category@3 = Jewelry OR i_category@3 = Women) AND (i_class@2 = shirts OR i_class@2 = birdal OR i_class@2 = dresses) + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Electronics OR i_category@12 = Sports) AND (i_class@10 = computers OR i_class@10 = stereo OR i_class@10 = football) OR (i_category@12 = Men OR i_category@12 = Jewelry OR i_category@12 = Women) AND (i_class@10 = shirts OR i_class@10 = birdal OR i_class@10 = dresses), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= computers AND computers <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= stereo AND stereo <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= football AND football <= i_class_max@5) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= shirts AND shirts <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= birdal AND birdal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= dresses AND dresses <= i_class_max@5), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_moy@1 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999, projection=[d_date_sk@0, d_moy@2] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: (i_category@3 = Books OR i_category@3 = Electronics OR i_category@3 = Sports) AND (i_class@2 = computers OR i_class@2 = stereo OR i_class@2 = football) OR (i_category@3 = Men OR i_category@3 = Jewelry OR i_category@3 = Women) AND (i_class@2 = shirts OR i_class@2 = birdal OR i_class@2 = dresses) - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Electronics OR i_category@12 = Sports) AND (i_class@10 = computers OR i_class@10 = stereo OR i_class@10 = football) OR (i_category@12 = Men OR i_category@12 = Jewelry OR i_category@12 = Women) AND (i_class@10 = shirts OR i_class@10 = birdal OR i_class@10 = dresses), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= computers AND computers <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= stereo AND stereo <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= football AND football <= i_class_max@5) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= shirts AND shirts <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= birdal AND birdal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= dresses AND dresses <= i_class_max@5), required_guarantees=[] - └────────────────────────────────────────────────── "#); Ok(()) } @@ -9922,89 +9454,83 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as amc] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[count(Int64(1))@0 as pmc] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@0)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ws_sold_time_sk@0)], projection=[ws_web_page_sk@3] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ws_ship_hdemo_sk@1)], projection=[ws_sold_time_sk@2, ws_web_page_sk@4] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_time_sk, ws_ship_hdemo_sk, ws_web_page_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)] │ FilterExec: wp_char_count@1 >= 5000 AND wp_char_count@1 <= 5200, projection=[wp_web_page_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, wp_char_count], file_type=parquet, predicate=wp_char_count@10 >= 5000 AND wp_char_count@10 <= 5200, pruning_predicate=wp_char_count_null_count@1 != row_count@2 AND wp_char_count_max@0 >= 5000 AND wp_char_count_null_count@1 != row_count@2 AND wp_char_count_min@3 <= 5200, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, wp_char_count], file_type=parquet, predicate=wp_char_count@10 >= 5000 AND wp_char_count@10 <= 5200, pruning_predicate=wp_char_count_null_count@1 != row_count@2 AND wp_char_count_max@0 >= 5000 AND wp_char_count_null_count@1 != row_count@2 AND wp_char_count_min@3 <= 5200, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 >= 8 AND t_hour@1 <= 9, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour], file_type=parquet, predicate=t_hour@3 >= 8 AND t_hour@3 <= 9, pruning_predicate=t_hour_null_count@1 != row_count@2 AND t_hour_max@0 >= 8 AND t_hour_null_count@1 != row_count@2 AND t_hour_min@3 <= 9, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour], file_type=parquet, predicate=t_hour@3 >= 8 AND t_hour@3 <= 9, pruning_predicate=t_hour_null_count@1 != row_count@2 AND t_hour_max@0 >= 8 AND t_hour_null_count@1 != row_count@2 AND t_hour_min@3 <= 9, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 6, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count], file_type=parquet, predicate=hd_dep_count@3 = 6, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 6 AND 6 <= hd_dep_count_max@1, required_guarantees=[hd_dep_count in (6)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count], file_type=parquet, predicate=hd_dep_count@3 = 6, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 6 AND 6 <= hd_dep_count_max@1, required_guarantees=[hd_dep_count in (6)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@0)] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ws_sold_time_sk@0)], projection=[ws_web_page_sk@3] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ws_ship_hdemo_sk@1)], projection=[ws_sold_time_sk@2, ws_web_page_sk@4] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_time_sk, ws_ship_hdemo_sk, ws_web_page_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)] │ FilterExec: wp_char_count@1 >= 5000 AND wp_char_count@1 <= 5200, projection=[wp_web_page_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, wp_char_count], file_type=parquet, predicate=wp_char_count@10 >= 5000 AND wp_char_count@10 <= 5200, pruning_predicate=wp_char_count_null_count@1 != row_count@2 AND wp_char_count_max@0 >= 5000 AND wp_char_count_null_count@1 != row_count@2 AND wp_char_count_min@3 <= 5200, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, wp_char_count], file_type=parquet, predicate=wp_char_count@10 >= 5000 AND wp_char_count@10 <= 5200, pruning_predicate=wp_char_count_null_count@1 != row_count@2 AND wp_char_count_max@0 >= 5000 AND wp_char_count_null_count@1 != row_count@2 AND wp_char_count_min@3 <= 5200, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 6 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 >= 19 AND t_hour@1 <= 20, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour], file_type=parquet, predicate=t_hour@3 >= 19 AND t_hour@3 <= 20, pruning_predicate=t_hour_null_count@1 != row_count@2 AND t_hour_max@0 >= 19 AND t_hour_null_count@1 != row_count@2 AND t_hour_min@3 <= 20, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour], file_type=parquet, predicate=t_hour@3 >= 19 AND t_hour@3 <= 20, pruning_predicate=t_hour_null_count@1 != row_count@2 AND t_hour_max@0 >= 19 AND t_hour_null_count@1 != row_count@2 AND t_hour_min@3 <= 20, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 7 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 6, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count], file_type=parquet, predicate=hd_dep_count@3 = 6, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 6 AND 6 <= hd_dep_count_max@1, required_guarantees=[hd_dep_count in (6)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count], file_type=parquet, predicate=hd_dep_count@3 = 6, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 6 AND 6 <= hd_dep_count_max@1, required_guarantees=[hd_dep_count in (6)] └────────────────────────────────────────────────── "); Ok(()) @@ -10015,79 +9541,71 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [returns_loss@3 DESC] - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: expr=[returns_loss@3 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[cc_call_center_id@0 as call_center, cc_name@1 as call_center_name, cc_manager@2 as manager, sum(catalog_returns.cr_net_loss)@5 as returns_loss] + │ AggregateExec: mode=FinalPartitioned, gby=[cc_call_center_id@0 as cc_call_center_id, cc_name@1 as cc_name, cc_manager@2 as cc_manager, cd_marital_status@3 as cd_marital_status, cd_education_status@4 as cd_education_status], aggr=[sum(catalog_returns.cr_net_loss)] + │ RepartitionExec: partitioning=Hash([cc_call_center_id@0, cc_name@1, cc_manager@2, cd_marital_status@3, cd_education_status@4], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[cc_call_center_id@0 as cc_call_center_id, cc_name@1 as cc_name, cc_manager@2 as cc_manager, cd_marital_status@4 as cd_marital_status, cd_education_status@5 as cd_education_status], aggr=[sum(catalog_returns.cr_net_loss)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@4, CAST(household_demographics.hd_demo_sk AS Float64)@1)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, cd_marital_status@5, cd_education_status@6] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=4 + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_buy_potential@1 LIKE Unknown%, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential], file_type=parquet, predicate=hd_buy_potential@2 LIKE Unknown%, pruning_predicate=hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknowo AND Unknown <= hd_buy_potential_max@1, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[returns_loss@3 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[cc_call_center_id@0 as call_center, cc_name@1 as call_center_name, cc_manager@2 as manager, sum(catalog_returns.cr_net_loss)@5 as returns_loss] - │ AggregateExec: mode=FinalPartitioned, gby=[cc_call_center_id@0 as cc_call_center_id, cc_name@1 as cc_name, cc_manager@2 as cc_manager, cd_marital_status@3 as cd_marital_status, cd_education_status@4 as cd_education_status], aggr=[sum(catalog_returns.cr_net_loss)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cc_call_center_id@0, cc_name@1, cc_manager@2, cd_marital_status@3, cd_education_status@4], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[cc_call_center_id@0 as cc_call_center_id, cc_name@1 as cc_name, cc_manager@2 as cc_manager, cd_marital_status@4 as cd_marital_status, cd_education_status@5 as cd_education_status], aggr=[sum(catalog_returns.cr_net_loss)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@4, CAST(household_demographics.hd_demo_sk AS Float64)@1)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, cd_marital_status@5, cd_education_status@6] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_buy_potential@1 LIKE Unknown%, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential], file_type=parquet, predicate=hd_buy_potential@2 LIKE Unknown%, pruning_predicate=hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknowo AND Unknown <= hd_buy_potential_max@1, required_guarantees=[] + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@4, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, c_current_hdemo_sk@5, cd_marital_status@7, cd_education_status@8] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Unknown OR cd_marital_status@1 = W AND cd_education_status@2 = Advanced Degree + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Unknown OR cd_marital_status@2 = W AND cd_education_status@3 = Advanced Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Unknown AND Unknown <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (Advanced Degree, Unknown), cd_marital_status in (M, W)] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@6, ca_address_sk@0)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, c_current_cdemo_sk@4, c_current_hdemo_sk@5] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=4 + │ FilterExec: ca_gmt_offset@1 = Some(-700),4,2, projection=[ca_address_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-700),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-700),4,2 AND Some(-700),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-700),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@4, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, c_current_hdemo_sk@5, cd_marital_status@7, cd_education_status@8] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] t3:[p18..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returning_customer_sk@3, CAST(customer.c_customer_sk AS Float64)@4)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@4, c_current_cdemo_sk@6, c_current_hdemo_sk@7, c_current_addr_sk@8] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Unknown OR cd_marital_status@1 = W AND cd_education_status@2 = Advanced Degree - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Unknown OR cd_marital_status@2 = W AND cd_education_status@3 = Advanced Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Unknown AND Unknown <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (Advanced Degree, Unknown), cd_marital_status in (M, W)] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@6, ca_address_sk@0)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, c_current_cdemo_sk@4, c_current_hdemo_sk@5] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returned_date_sk@3, d_date_sk@0)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_returning_customer_sk@4, cr_net_loss@5] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: ca_gmt_offset@1 = Some(-700),4,2, projection=[ca_address_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-700),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-700),4,2 AND Some(-700),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-700),4,2)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returning_customer_sk@3, CAST(customer.c_customer_sk AS Float64)@4)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@4, c_current_cdemo_sk@6, c_current_hdemo_sk@7, c_current_addr_sk@8] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 11, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11 AND DynamicFilter [ empty ], pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@4, cr_call_center_sk@2)], projection=[cc_call_center_id@1, cc_name@2, cc_manager@3, cr_returned_date_sk@5, cr_returning_customer_sk@6, cr_net_loss@8] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk], file_type=parquet + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_call_center_sk, cr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returned_date_sk@3, d_date_sk@0)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_returning_customer_sk@4, cr_net_loss@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 11, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11 AND DynamicFilter [ empty ], pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_call_center_id, cc_name, cc_manager, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@4, cr_call_center_sk@2)], projection=[cc_call_center_id@1, cc_name@2, cc_manager@3, cr_returned_date_sk@5, cr_returning_customer_sk@6, cr_net_loss@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_call_center_sk, cr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_call_center_id, cc_name, cc_manager, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── "); Ok(()) } @@ -10100,60 +9618,57 @@ mod tests { │ GlobalLimitExec: skip=0, fetch=100 │ AggregateExec: mode=Final, gby=[], aggr=[sum(web_sales.ws_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(web_sales.ws_ext_discount_amt)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@1, i_item_sk@1)], filter=CAST(ws_ext_discount_amt@0 AS Decimal128(30, 15)) > Float64(1.3) * avg(web_sales.ws_ext_discount_amt)@1, projection=[ws_ext_discount_amt@2] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_ext_discount_amt@3, i_item_sk@4] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ext_discount_amt@2 as ws_ext_discount_amt, i_item_sk@0 as i_item_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_sk@0, ws_sold_date_sk@1, ws_ext_discount_amt@3] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ext_discount_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[CAST(1.3 * CAST(avg(web_sales.ws_ext_discount_amt)@1 AS Float64) AS Decimal128(30, 15)) as Float64(1.3) * avg(web_sales.ws_ext_discount_amt), ws_item_sk@0 as ws_item_sk] │ AggregateExec: mode=FinalPartitioned, gby=[ws_item_sk@0 as ws_item_sk], aggr=[avg(web_sales.ws_ext_discount_amt)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ws_item_sk@0], 6), input_partitions=2 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ws_item_sk@0], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ws_item_sk@0 as ws_item_sk], aggr=[avg(web_sales.ws_ext_discount_amt)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_ext_discount_amt@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ext_discount_amt], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 5 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ FilterExec: i_manufact_id@1 = 350, projection=[i_item_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=i_manufact_id@13 = 350 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 350 AND 350 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (350)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=i_manufact_id@13 = 350 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 350 AND 350 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (350)] └────────────────────────────────────────────────── "); Ok(()) @@ -10164,44 +9679,40 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [sumsales@1 ASC, ss_customer_sk@0 ASC], fetch=100 - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[sumsales@1 ASC, ss_customer_sk@0 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[ss_customer_sk@0 as ss_customer_sk, sum(t.act_sales)@1 as sumsales] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_customer_sk@0 as ss_customer_sk], aggr=[sum(t.act_sales)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[sumsales@1 ASC, ss_customer_sk@0 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[ss_customer_sk@0 as ss_customer_sk, sum(t.act_sales)@1 as sumsales] - │ AggregateExec: mode=FinalPartitioned, gby=[ss_customer_sk@0 as ss_customer_sk], aggr=[sum(t.act_sales)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[ss_customer_sk@0 as ss_customer_sk], aggr=[sum(t.act_sales)] - │ ProjectionExec: expr=[ss_customer_sk@0 as ss_customer_sk, CASE WHEN sr_return_quantity@3 IS NOT NULL THEN (ss_quantity@1 - sr_return_quantity@3) * CAST(ss_sales_price@2 AS Float64) ELSE ss_quantity@1 * CAST(ss_sales_price@2 AS Float64) END as act_sales] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(reason.r_reason_sk AS Float64)@1, sr_reason_sk@3)], projection=[ss_customer_sk@2, ss_quantity@3, ss_sales_price@4, sr_return_quantity@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_customer_sk@2 as ss_customer_sk, ss_quantity@3 as ss_quantity, ss_sales_price@4 as ss_sales_price, sr_reason_sk@0 as sr_reason_sk, sr_return_quantity@1 as sr_return_quantity] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_item_sk@0, ss_item_sk@0), (sr_ticket_number@2, ss_ticket_number@2)], projection=[sr_reason_sk@1, sr_return_quantity@3, ss_customer_sk@5, ss_quantity@7, ss_sales_price@8] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[ss_customer_sk@0 as ss_customer_sk], aggr=[sum(t.act_sales)] + │ ProjectionExec: expr=[ss_customer_sk@0 as ss_customer_sk, CASE WHEN sr_return_quantity@3 IS NOT NULL THEN (ss_quantity@1 - sr_return_quantity@3) * CAST(ss_sales_price@2 AS Float64) ELSE ss_quantity@1 * CAST(ss_sales_price@2 AS Float64) END as act_sales] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(reason.r_reason_sk AS Float64)@1, sr_reason_sk@3)], projection=[ss_customer_sk@2, ss_quantity@3, ss_sales_price@4, sr_return_quantity@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_customer_sk@2 as ss_customer_sk, ss_quantity@3 as ss_quantity, ss_sales_price@4 as ss_sales_price, sr_reason_sk@0 as sr_reason_sk, sr_return_quantity@1 as sr_return_quantity] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_item_sk@0, ss_item_sk@0), (sr_ticket_number@2, ss_ticket_number@2)], projection=[sr_reason_sk@1, sr_return_quantity@3, ss_customer_sk@5, ss_quantity@7, ss_sales_price@8] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[r_reason_sk@0 as r_reason_sk, CAST(r_reason_sk@0 AS Float64) as CAST(reason.r_reason_sk AS Float64)] + │ FilterExec: r_reason_desc@1 = reason 28, projection=[r_reason_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk, r_reason_desc], file_type=parquet, predicate=r_reason_desc@2 = reason 28, pruning_predicate=r_reason_desc_null_count@2 != row_count@3 AND r_reason_desc_min@0 <= reason 28 AND reason 28 <= r_reason_desc_max@1, required_guarantees=[r_reason_desc in (reason 28)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_reason_sk, sr_ticket_number, sr_return_quantity], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([ss_item_sk@0, ss_ticket_number@2], 12), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_customer_sk, ss_ticket_number, ss_quantity, ss_sales_price], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[r_reason_sk@0 as r_reason_sk, CAST(r_reason_sk@0 AS Float64) as CAST(reason.r_reason_sk AS Float64)] - │ FilterExec: r_reason_desc@1 = reason 28, projection=[r_reason_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk, r_reason_desc], file_type=parquet, predicate=r_reason_desc@2 = reason 28, pruning_predicate=r_reason_desc_null_count@2 != row_count@3 AND r_reason_desc_min@0 <= reason 28 AND reason 28 <= r_reason_desc_max@1, required_guarantees=[r_reason_desc in (reason 28)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_reason_sk, sr_ticket_number, sr_return_quantity], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] - │ RepartitionExec: partitioning=Hash([ss_item_sk@0, ss_ticket_number@2], 9), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_customer_sk, ss_ticket_number, ss_quantity, ss_sales_price], file_type=parquet - └────────────────────────────────────────────────── "); Ok(()) } @@ -10222,47 +9733,44 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ws_order_number@1, ws_order_number@1)], filter=ws_warehouse_sk@0 != ws_warehouse_sk@1, projection=[ws_order_number@1, ws_ext_ship_cost@2, ws_net_profit@3] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-3.parquet:..]]}, projection=[ws_warehouse_sk, ws_order_number], file_type=parquet │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@1, ws_web_site_sk@0)], projection=[ws_warehouse_sk@3, ws_order_number@4, ws_ext_ship_cost@5, ws_net_profit@6] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_address.ca_address_sk AS Float64)@1, ws_ship_addr_sk@0)], projection=[ws_web_site_sk@3, ws_warehouse_sk@4, ws_order_number@5, ws_ext_ship_cost@6, ws_net_profit@7] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_ship_date_sk@0)], projection=[ws_ship_addr_sk@3, ws_web_site_sk@4, ws_warehouse_sk@5, ws_order_number@6, ws_ext_ship_cost@7, ws_net_profit@8] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_ship_date_sk, ws_ship_addr_sk, ws_web_site_sk, ws_warehouse_sk, ws_order_number, ws_ext_ship_cost, ws_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[web_site_sk@0 as web_site_sk, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)] │ FilterExec: web_company_name@1 = pri, projection=[web_site_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_company_name], file_type=parquet, predicate=web_company_name@14 = pri, pruning_predicate=web_company_name_null_count@2 != row_count@3 AND web_company_name_min@0 <= pri AND pri <= web_company_name_max@1, required_guarantees=[web_company_name in (pri)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_company_name], file_type=parquet, predicate=web_company_name@14 = pri, pruning_predicate=web_company_name_null_count@2 != row_count@3 AND web_company_name_min@0 <= pri AND pri <= web_company_name_max@1, required_guarantees=[web_company_name in (pri)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] │ FilterExec: ca_state@1 = IL, projection=[ca_address_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet, predicate=ca_state@8 = IL, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IL AND IL <= ca_state_max@1, required_guarantees=[ca_state in (IL)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet, predicate=ca_state@8 = IL, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IL AND IL <= ca_state_max@1, required_guarantees=[ca_state in (IL)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 1999-02-01 AND d_date@1 <= 1999-04-02, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-01 AND d_date@2 <= 1999-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-04-02, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-01 AND d_date@2 <= 1999-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-04-02, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -10284,73 +9792,70 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ws_order_number@0, ws_order_number@0)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ws_order_number@1, ws_order_number@1)], filter=ws_warehouse_sk@1 != ws_warehouse_sk@0, projection=[ws_order_number@1] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_order_number@0, ws_order_number@0)], projection=[wr_order_number@0] │ CoalescePartitionsExec │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_order_number], file_type=parquet │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ws_order_number@1, ws_order_number@1)], filter=ws_warehouse_sk@1 != ws_warehouse_sk@0, projection=[ws_order_number@1] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@1, ws_web_site_sk@0)], projection=[ws_order_number@3, ws_ext_ship_cost@4, ws_net_profit@5] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_address.ca_address_sk AS Float64)@1, ws_ship_addr_sk@0)], projection=[ws_web_site_sk@3, ws_order_number@4, ws_ext_ship_cost@5, ws_net_profit@6] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_ship_date_sk@0)], projection=[ws_ship_addr_sk@3, ws_web_site_sk@4, ws_order_number@5, ws_ext_ship_cost@6, ws_net_profit@7] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_ship_date_sk, ws_ship_addr_sk, ws_web_site_sk, ws_order_number, ws_ext_ship_cost, ws_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[web_site_sk@0 as web_site_sk, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)] │ FilterExec: web_company_name@1 = pri, projection=[web_site_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_company_name], file_type=parquet, predicate=web_company_name@14 = pri, pruning_predicate=web_company_name_null_count@2 != row_count@3 AND web_company_name_min@0 <= pri AND pri <= web_company_name_max@1, required_guarantees=[web_company_name in (pri)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_company_name], file_type=parquet, predicate=web_company_name@14 = pri, pruning_predicate=web_company_name_null_count@2 != row_count@3 AND web_company_name_min@0 <= pri AND pri <= web_company_name_max@1, required_guarantees=[web_company_name in (pri)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] │ FilterExec: ca_state@1 = IL, projection=[ca_address_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet, predicate=ca_state@8 = IL, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IL AND IL <= ca_state_max@1, required_guarantees=[ca_state in (IL)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet, predicate=ca_state@8 = IL, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IL AND IL <= ca_state_max@1, required_guarantees=[ca_state in (IL)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 1999-02-01 AND d_date@1 <= 1999-04-02, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-01 AND d_date@2 <= 1999-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-04-02, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-01 AND d_date@2 <= 1999-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-04-02, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_warehouse_sk, ws_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_warehouse_sk, ws_order_number], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_warehouse_sk, ws_order_number], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_warehouse_sk, ws_order_number], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -10365,46 +9870,44 @@ mod tests { │ GlobalLimitExec: skip=0, fetch=100 │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_hour@1 = 20 AND t_minute@2 >= 30, projection=[t_time_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 20 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 20 AND 20 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (20)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_dep_count@1 = 7, projection=[hd_demo_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count], file_type=parquet, predicate=hd_dep_count@3 = 7, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 7 AND 7 <= hd_dep_count_max@1, required_guarantees=[hd_dep_count in (7)] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_dep_count], file_type=parquet, predicate=hd_dep_count@3 = 7, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 7 AND 7 <= hd_dep_count_max@1, required_guarantees=[hd_dep_count in (7)] └────────────────────────────────────────────────── "); Ok(()) @@ -10422,49 +9925,47 @@ mod tests { │ ProjectionExec: expr=[customer_sk@1 IS NOT NULL as __common_expr_1, customer_sk@1 as customer_sk, customer_sk@0 as customer_sk] │ HashJoinExec: mode=CollectLeft, join_type=Full, on=[(customer_sk@0, customer_sk@0), (item_sk@1, item_sk@1)], projection=[customer_sk@0, customer_sk@2] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 │ ProjectionExec: expr=[ss_customer_sk@0 as customer_sk, ss_item_sk@1 as item_sk] │ AggregateExec: mode=FinalPartitioned, gby=[ss_customer_sk@0 as ss_customer_sk, ss_item_sk@1 as ss_item_sk], aggr=[] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ ProjectionExec: expr=[cs_bill_customer_sk@0 as customer_sk, cs_item_sk@1 as item_sk] │ AggregateExec: mode=FinalPartitioned, gby=[cs_bill_customer_sk@0 as cs_bill_customer_sk, cs_item_sk@1 as cs_item_sk], aggr=[] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@0, cs_item_sk@1], 6), input_partitions=2 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@0, cs_item_sk@1], 12), input_partitions=2 │ AggregateExec: mode=Partial, gby=[cs_bill_customer_sk@0 as cs_bill_customer_sk, cs_item_sk@1 as cs_item_sk], aggr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_customer_sk@3, cs_item_sk@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([ss_customer_sk@0, ss_item_sk@1], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_customer_sk@1 as ss_customer_sk, ss_item_sk@0 as ss_item_sk], aggr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_customer_sk@4] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 + ┌───── Stage 4 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -10475,45 +9976,47 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_category@2 ASC, i_class@3 ASC, i_item_id@0 ASC, i_item_desc@1 ASC, revenueratio@6 ASC] - │ SortExec: expr=[i_category@2 ASC, i_class@3 ASC, i_item_id@0 ASC, i_item_desc@1 ASC, revenueratio@6 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price, sum(store_sales.ss_ext_sales_price)@5 as itemrevenue, CAST(sum(store_sales.ss_ext_sales_price)@5 AS Float64) * 100 / CAST(sum(sum(store_sales.ss_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@6 AS Float64) as revenueratio] - │ WindowAggExec: wdw=[sum(sum(store_sales.ss_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(sum(store_sales.ss_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(27, 2), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] - │ SortExec: expr=[i_class@3 ASC NULLS LAST], preserve_partitioning=[true] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([i_class@3], 3), input_partitions=3 - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price], aggr=[sum(store_sales.ss_ext_sales_price)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_category@2, i_class@3, i_current_price@4], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id, i_item_desc@2 as i_item_desc, i_category@5 as i_category, i_class@4 as i_class, i_current_price@3 as i_current_price], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_ext_sales_price@3, i_item_id@4, i_item_desc@5, i_current_price@6, i_class@7, i_category@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@5 as ss_sold_date_sk, ss_ext_sales_price@6 as ss_ext_sales_price, i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price, i_class@3 as i_class, i_category@4 as i_category] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3, i_class@4, i_category@5, ss_sold_date_sk@6, ss_ext_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] + │ SortExec: expr=[i_category@2 ASC, i_class@3 ASC, i_item_id@0 ASC, i_item_desc@1 ASC, revenueratio@6 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price, sum(store_sales.ss_ext_sales_price)@5 as itemrevenue, CAST(sum(store_sales.ss_ext_sales_price)@5 AS Float64) * 100 / CAST(sum(sum(store_sales.ss_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@6 AS Float64) as revenueratio] + │ WindowAggExec: wdw=[sum(sum(store_sales.ss_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(sum(store_sales.ss_ext_sales_price)) PARTITION BY [item.i_class] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(27, 2), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] + │ SortExec: expr=[i_class@3 ASC NULLS LAST], preserve_partitioning=[true] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_class@3], 12), input_partitions=3 + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_category@2 as i_category, i_class@3 as i_class, i_current_price@4 as i_current_price], aggr=[sum(store_sales.ss_ext_sales_price)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_category@2, i_class@3, i_current_price@4], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id, i_item_desc@2 as i_item_desc, i_category@5 as i_category, i_class@4 as i_class, i_current_price@3 as i_current_price], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_ext_sales_price@3, i_item_id@4, i_item_desc@5, i_current_price@6, i_class@7, i_category@8] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=1 + │ ProjectionExec: expr=[ss_sold_date_sk@5 as ss_sold_date_sk, ss_ext_sales_price@6 as ss_ext_sales_price, i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price, i_class@3 as i_class, i_category@4 as i_category] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3, i_class@4, i_category@5, ss_sold_date_sk@6, ss_ext_sales_price@8] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] + │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 + │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] + └────────────────────────────────────────────────── "#); Ok(()) } @@ -10523,57 +10026,49 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [w_substr@0 ASC, sm_type@1 ASC, cc_name_lower@2 ASC], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ SortExec: TopK(fetch=100), expr=[w_substr@0 ASC, sm_type@1 ASC, cc_name_lower@2 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[w_substr@0 as w_substr, sm_type@1 as sm_type, lower(cc_name@2) as cc_name_lower, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@3 as 30 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@4 as 31-60 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@5 as 61-90 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@6 as 91-120 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@7 as >120 days] + │ AggregateExec: mode=FinalPartitioned, gby=[w_substr@0 as w_substr, sm_type@1 as sm_type, cc_name@2 as cc_name], aggr=[sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] + │ RepartitionExec: partitioning=Hash([w_substr@0, sm_type@1, cc_name@2], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[w_substr@1 as w_substr, sm_type@2 as sm_type, cc_name@3 as cc_name], aggr=[sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] + │ ProjectionExec: expr=[cs_ship_date_sk@1 - cs_sold_date_sk@0 as __common_expr_1, w_substr@2 as w_substr, sm_type@3 as sm_type, cc_name@4 as cc_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, w_substr@2, sm_type@3, cc_name@4] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=4 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[w_substr@0 ASC, sm_type@1 ASC, cc_name_lower@2 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[w_substr@0 as w_substr, sm_type@1 as sm_type, lower(cc_name@2) as cc_name_lower, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@3 as 30 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@4 as 31-60 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@5 as 61-90 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@6 as 91-120 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@7 as >120 days] - │ AggregateExec: mode=FinalPartitioned, gby=[w_substr@0 as w_substr, sm_type@1 as sm_type, cc_name@2 as cc_name], aggr=[sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([w_substr@0, sm_type@1, cc_name@2], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[w_substr@1 as w_substr, sm_type@2 as sm_type, cc_name@3 as cc_name], aggr=[sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] - │ ProjectionExec: expr=[cs_ship_date_sk@1 - cs_sold_date_sk@0 as __common_expr_1, w_substr@2 as w_substr, sm_type@3 as sm_type, cc_name@4 as cc_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, w_substr@2, sm_type@3, cc_name@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_call_center_sk@2, CAST(call_center.cc_call_center_sk AS Float64)@2)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, w_substr@3, sm_type@4, cc_name@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_call_center_sk@2, CAST(call_center.cc_call_center_sk AS Float64)@2)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, w_substr@3, sm_type@4, cc_name@6] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_mode_sk@3, CAST(ship_mode.sm_ship_mode_sk AS Float64)@2)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, cs_call_center_sk@2, w_substr@4, sm_type@6] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_type, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] t3:[p24..p31] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ship_date_sk@2 as cs_ship_date_sk, cs_call_center_sk@3 as cs_call_center_sk, cs_ship_mode_sk@4 as cs_ship_mode_sk, w_substr@0 as w_substr] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(sq1.w_warehouse_sk AS Float64)@2, cs_warehouse_sk@4)], projection=[w_substr@0, cs_sold_date_sk@3, cs_ship_date_sk@4, cs_call_center_sk@5, cs_ship_mode_sk@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_date_sk, cs_call_center_sk, cs_ship_mode_sk, cs_warehouse_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_mode_sk@3, CAST(ship_mode.sm_ship_mode_sk AS Float64)@2)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, cs_call_center_sk@2, w_substr@4, sm_type@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_type, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ship_date_sk@2 as cs_ship_date_sk, cs_call_center_sk@3 as cs_call_center_sk, cs_ship_mode_sk@4 as cs_ship_mode_sk, w_substr@0 as w_substr] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(sq1.w_warehouse_sk AS Float64)@2, cs_warehouse_sk@4)], projection=[w_substr@0, cs_sold_date_sk@3, cs_ship_date_sk@4, cs_call_center_sk@5, cs_ship_mode_sk@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_date_sk, cs_call_center_sk, cs_ship_mode_sk, cs_warehouse_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[substr(w_warehouse_name@2, 1, 20) as w_substr, w_warehouse_sk, CAST(w_warehouse_sk@0 AS Float64) as CAST(sq1.w_warehouse_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p7] + │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 + │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[substr(w_warehouse_name@2, 1, 20) as w_substr, w_warehouse_sk, CAST(w_warehouse_sk@0 AS Float64) as CAST(sq1.w_warehouse_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── "); Ok(()) } @@ -10598,8 +10093,7 @@ mod tests { // Make distributed localhost context to run queries let (d_ctx, _guard, _) = start_localhost_context(NUM_WORKERS, DefaultSessionBuilder).await; let d_ctx = d_ctx - .with_distributed_files_per_task(FILES_PER_TASK)? - .with_distributed_cardinality_effect_task_scale_factor(CARDINALITY_TASK_COUNT_FACTOR)? + .with_distributed_bytes_processed_per_partition(BYTES_PROCESSED_PER_PARTITION)? .with_distributed_broadcast_joins(true)?; benchmarks_common::register_tables(&d_ctx, &data_dir).await?; diff --git a/tests/tpch_correctness_test.rs b/tests/tpch_correctness_test.rs index 114dc2e1..67b4e5f7 100644 --- a/tests/tpch_correctness_test.rs +++ b/tests/tpch_correctness_test.rs @@ -13,6 +13,7 @@ mod tests { use tokio::sync::OnceCell; const PARTITIONS: usize = 6; + const BYTES_PROCESSED_PER_PARTITION: usize = 1024 * 1024; const TPCH_SCALE_FACTOR: f64 = 1.0; const TPCH_DATA_PARTS: i32 = 16; @@ -136,7 +137,9 @@ mod tests { // in a non-distributed manner. For each query, it asserts that the results are identical. async fn test_tpch_query(sql: String) -> Result<(), Box> { let (d_ctx, _guard, _) = start_localhost_context(4, DefaultSessionBuilder).await; - let d_ctx = d_ctx.with_distributed_broadcast_joins(true)?; + let d_ctx = d_ctx + .with_distributed_bytes_processed_per_partition(BYTES_PROCESSED_PER_PARTITION)? + .with_distributed_broadcast_joins(true)?; let results_d = run_tpch_query(d_ctx, sql.clone()).await?; let results_s = run_tpch_query(SessionContext::new(), sql).await?; diff --git a/tests/tpch_plans_test.rs b/tests/tpch_plans_test.rs index 2d03fbfb..89cfdd9b 100644 --- a/tests/tpch_plans_test.rs +++ b/tests/tpch_plans_test.rs @@ -11,7 +11,8 @@ mod tests { use tokio::sync::OnceCell; const PARTITIONS: usize = 6; - const TPCH_SCALE_FACTOR: f64 = 0.02; + const BYTES_PROCESSED_PER_PARTITION: usize = 8 * 1024 * 1024; + const TPCH_SCALE_FACTOR: f64 = 1.0; const TPCH_DATA_PARTS: i32 = 16; #[tokio::test] @@ -20,21 +21,21 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..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))] │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] - │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 12), input_partitions=4 + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 24), input_partitions=6 │ 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] │ FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=parquet, predicate=l_shipdate@10 <= 1998-09-02, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@0 <= 1998-09-02, required_guarantees=[] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=parquet, predicate=l_shipdate@10 <= 1998-09-02, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@0 <= 1998-09-02, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -46,94 +47,87 @@ mod tests { 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] - │ [Stage 11] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[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] │ 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] - │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 - │ [Stage 10] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + │ [Stage 10] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=4 + ┌───── Stage 5 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] + │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 18), input_partitions=5 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=18, input_tasks=1 │ 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] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@4)], projection=[n_name@1, n_regionkey@2, p_partkey@3, p_mfgr@4, s_name@5, s_address@6, s_phone@8, s_acctbal@9, s_comment@10, ps_supplycost@11] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=18, input_tasks=1 │ 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] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@2)], projection=[s_name@1, s_address@2, s_nationkey@3, s_phone@4, s_acctbal@5, s_comment@6, p_partkey@7, p_mfgr@8, ps_supplycost@10] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=18, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet], [/testdata/tpch/plan_sf0.02/partsupp/10.parquet], [/testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet], [/testdata/tpch/plan_sf0.02/partsupp/13.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=18, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={15 groups: [[/testdata/tpch/plan_sf1/partsupp/1.parquet:..], [/testdata/tpch/plan_sf1/partsupp/10.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:.., /testdata/tpch/plan_sf1/partsupp/12.parquet:..], [/testdata/tpch/plan_sf1/partsupp/13.parquet:.., /testdata/tpch/plan_sf1/partsupp/14.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 1 ── Tasks: t0:[p0..p17] + │ BroadcastExec: input_partitions=6, consumer_tasks=3, output_partitions=18 │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/region/1.parquet, /testdata/tpch/plan_sf1/region/10.parquet, /testdata/tpch/plan_sf1/region/11.parquet], [/testdata/tpch/plan_sf1/region/12.parquet, /testdata/tpch/plan_sf1/region/13.parquet, /testdata/tpch/plan_sf1/region/14.parquet], [/testdata/tpch/plan_sf1/region/15.parquet, /testdata/tpch/plan_sf1/region/16.parquet, /testdata/tpch/plan_sf1/region/2.parquet], [/testdata/tpch/plan_sf1/region/3.parquet, /testdata/tpch/plan_sf1/region/4.parquet, /testdata/tpch/plan_sf1/region/5.parquet], [/testdata/tpch/plan_sf1/region/6.parquet, /testdata/tpch/plan_sf1/region/7.parquet, /testdata/tpch/plan_sf1/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..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p17] + │ BroadcastExec: input_partitions=6, consumer_tasks=3, output_partitions=18 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p17] + │ BroadcastExec: input_partitions=6, consumer_tasks=3, output_partitions=18 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 4 ── Tasks: t0:[p0..p17] + │ BroadcastExec: input_partitions=6, consumer_tasks=3, output_partitions=18 │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@5 = 15 AND p_type@4 LIKE %BRASS, pruning_predicate=p_size_null_count@2 != row_count@3 AND p_size_min@0 <= 15 AND 15 <= p_size_max@1, required_guarantees=[p_size in (15)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/part/1.parquet:.., /testdata/tpch/plan_sf1/part/10.parquet:.., /testdata/tpch/plan_sf1/part/11.parquet:..], [/testdata/tpch/plan_sf1/part/11.parquet:.., /testdata/tpch/plan_sf1/part/12.parquet:.., /testdata/tpch/plan_sf1/part/13.parquet:.., /testdata/tpch/plan_sf1/part/14.parquet:..], [/testdata/tpch/plan_sf1/part/14.parquet:.., /testdata/tpch/plan_sf1/part/15.parquet:.., /testdata/tpch/plan_sf1/part/16.parquet:..], [/testdata/tpch/plan_sf1/part/16.parquet:.., /testdata/tpch/plan_sf1/part/2.parquet:.., /testdata/tpch/plan_sf1/part/3.parquet:.., /testdata/tpch/plan_sf1/part/4.parquet:..], [/testdata/tpch/plan_sf1/part/4.parquet:.., /testdata/tpch/plan_sf1/part/5.parquet:.., /testdata/tpch/plan_sf1/part/6.parquet:.., /testdata/tpch/plan_sf1/part/7.parquet:..], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@5 = 15 AND p_type@4 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 10 ── 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 + ┌───── Stage 10 ── Tasks: t0:[p0..p17] + │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 18), input_partitions=6 │ ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 9] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=4 + ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 6), input_partitions=5 │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=18, input_tasks=1 │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=18, input_tasks=1 │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4] │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet], [/testdata/tpch/plan_sf0.02/partsupp/10.parquet], [/testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet], [/testdata/tpch/plan_sf0.02/partsupp/13.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=18, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={15 groups: [[/testdata/tpch/plan_sf1/partsupp/1.parquet:..], [/testdata/tpch/plan_sf1/partsupp/10.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:.., /testdata/tpch/plan_sf1/partsupp/12.parquet:..], [/testdata/tpch/plan_sf1/partsupp/13.parquet:.., /testdata/tpch/plan_sf1/partsupp/14.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 6 ── Tasks: t0:[p0..p17] + │ BroadcastExec: input_partitions=6, consumer_tasks=3, output_partitions=18 │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/region/1.parquet, /testdata/tpch/plan_sf1/region/10.parquet, /testdata/tpch/plan_sf1/region/11.parquet], [/testdata/tpch/plan_sf1/region/12.parquet, /testdata/tpch/plan_sf1/region/13.parquet, /testdata/tpch/plan_sf1/region/14.parquet], [/testdata/tpch/plan_sf1/region/15.parquet, /testdata/tpch/plan_sf1/region/16.parquet, /testdata/tpch/plan_sf1/region/2.parquet], [/testdata/tpch/plan_sf1/region/3.parquet, /testdata/tpch/plan_sf1/region/4.parquet, /testdata/tpch/plan_sf1/region/5.parquet], [/testdata/tpch/plan_sf1/region/6.parquet, /testdata/tpch/plan_sf1/region/7.parquet, /testdata/tpch/plan_sf1/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 7 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 7 ── Tasks: t0:[p0..p17] + │ BroadcastExec: input_partitions=6, consumer_tasks=3, output_partitions=18 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0..p17] + │ BroadcastExec: input_partitions=6, consumer_tasks=3, output_partitions=18 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -145,38 +139,37 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST] - │ [Stage 4] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + ┌───── Stage 4 ── 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)] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 18), input_partitions=4 + ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[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)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=2 │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 > 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 > 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 2 ── Tasks: t0:[p0..p23] t1:[p24..p47] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=12, input_tasks=1 │ FilterExec: o_orderdate@2 < 1995-03-15 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@4 < 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,p5] + │ DataSourceExec: file_groups={12 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:..], [/testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@4 < 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=6, consumer_tasks=2, output_partitions=12 │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey, c_mktsegment], file_type=parquet, predicate=c_mktsegment@6 = 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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/customer/1.parquet:.., /testdata/tpch/plan_sf1/customer/10.parquet:.., /testdata/tpch/plan_sf1/customer/11.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:.., /testdata/tpch/plan_sf1/customer/12.parquet:.., /testdata/tpch/plan_sf1/customer/13.parquet:.., /testdata/tpch/plan_sf1/customer/14.parquet:..], [/testdata/tpch/plan_sf1/customer/14.parquet:.., /testdata/tpch/plan_sf1/customer/15.parquet:.., /testdata/tpch/plan_sf1/customer/16.parquet:..], [/testdata/tpch/plan_sf1/customer/16.parquet:.., /testdata/tpch/plan_sf1/customer/2.parquet:.., /testdata/tpch/plan_sf1/customer/3.parquet:.., /testdata/tpch/plan_sf1/customer/4.parquet:..], [/testdata/tpch/plan_sf1/customer/4.parquet:.., /testdata/tpch/plan_sf1/customer/5.parquet:.., /testdata/tpch/plan_sf1/customer/6.parquet:.., /testdata/tpch/plan_sf1/customer/7.parquet:..], ...]}, projection=[c_custkey, c_mktsegment], file_type=parquet, predicate=c_mktsegment@6 = 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -191,19 +184,26 @@ mod tests { │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(Int64(1))@1 as order_count] │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] - │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 6), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 - │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@4 >= 1993-07-01 AND o_orderdate@4 < 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 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 6), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] + │ HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=6 + │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:.., /testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:.., /testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:.., /testdata/tpch/plan_sf1/orders/16.parquet:.., /testdata/tpch/plan_sf1/orders/2.parquet:..], [/testdata/tpch/plan_sf1/orders/2.parquet:.., /testdata/tpch/plan_sf1/orders/3.parquet:.., /testdata/tpch/plan_sf1/orders/4.parquet:..], [/testdata/tpch/plan_sf1/orders/4.parquet:.., /testdata/tpch/plan_sf1/orders/5.parquet:.., /testdata/tpch/plan_sf1/orders/6.parquet:.., /testdata/tpch/plan_sf1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@4 >= 1993-07-01 AND o_orderdate@4 < 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 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=6 + │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 + └────────────────────────────────────────────────── "); Ok(()) } @@ -214,64 +214,63 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC] - │ [Stage 7] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + ┌───── Stage 8 ── 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)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([n_name@0], 18), input_partitions=4 + ┌───── Stage 7 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[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)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 │ 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] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@0 as s_nationkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1), (s_nationkey@1, c_nationkey@0)], projection=[s_nationkey@1, l_extendedprice@4, l_discount@5] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 1 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/region/1.parquet, /testdata/tpch/plan_sf1/region/10.parquet, /testdata/tpch/plan_sf1/region/11.parquet], [/testdata/tpch/plan_sf1/region/12.parquet, /testdata/tpch/plan_sf1/region/13.parquet, /testdata/tpch/plan_sf1/region/14.parquet], [/testdata/tpch/plan_sf1/region/15.parquet, /testdata/tpch/plan_sf1/region/16.parquet, /testdata/tpch/plan_sf1/region/2.parquet], [/testdata/tpch/plan_sf1/region/3.parquet, /testdata/tpch/plan_sf1/region/4.parquet, /testdata/tpch/plan_sf1/region/5.parquet], [/testdata/tpch/plan_sf1/region/6.parquet, /testdata/tpch/plan_sf1/region/7.parquet, /testdata/tpch/plan_sf1/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..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ 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,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1994-01-01 AND o_orderdate@4 < 1995-01-01 AND DynamicFilter [ empty ], 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=[] + ┌───── Stage 5 ── Tasks: t0:[p0..p23] + │ RepartitionExec: partitioning=Hash([o_orderkey@1], 24), input_partitions=6 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=6 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/customer/1.parquet:.., /testdata/tpch/plan_sf1/customer/10.parquet:.., /testdata/tpch/plan_sf1/customer/11.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:.., /testdata/tpch/plan_sf1/customer/12.parquet:.., /testdata/tpch/plan_sf1/customer/13.parquet:.., /testdata/tpch/plan_sf1/customer/14.parquet:..], [/testdata/tpch/plan_sf1/customer/14.parquet:.., /testdata/tpch/plan_sf1/customer/15.parquet:.., /testdata/tpch/plan_sf1/customer/16.parquet:..], [/testdata/tpch/plan_sf1/customer/16.parquet:.., /testdata/tpch/plan_sf1/customer/2.parquet:.., /testdata/tpch/plan_sf1/customer/3.parquet:.., /testdata/tpch/plan_sf1/customer/4.parquet:..], [/testdata/tpch/plan_sf1/customer/4.parquet:.., /testdata/tpch/plan_sf1/customer/5.parquet:.., /testdata/tpch/plan_sf1/customer/6.parquet:.., /testdata/tpch/plan_sf1/customer/7.parquet:..], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] + │ RepartitionExec: partitioning=Hash([o_custkey@1], 6), input_partitions=6 + │ 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,p2,p3,p4,p5,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,p5] + │ DataSourceExec: file_groups={12 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:..], [/testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1994-01-01 AND o_orderdate@4 < 1995-01-01 AND DynamicFilter [ empty ], 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=[] └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=6 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── "); Ok(()) } @@ -284,13 +283,13 @@ mod tests { │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] t3:[p18..p23] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1994-01-01 AND l_shipdate@10 < 1995-01-01 AND l_discount@6 >= Some(5),15,2 AND l_discount@6 <= Some(7),15,2 AND l_quantity@4 < 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=[] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1994-01-01 AND l_shipdate@10 < 1995-01-01 AND l_discount@6 >= Some(5),15,2 AND l_discount@6 <= Some(7),15,2 AND l_quantity@4 < Some(2400),15,2, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01 AND l_discount_null_count@5 != row_count@2 AND l_discount_max@4 >= Some(5),15,2 AND l_discount_null_count@5 != row_count@2 AND l_discount_min@6 <= Some(7),15,2 AND l_quantity_null_count@8 != row_count@2 AND l_quantity_min@7 < Some(2400),15,2, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -302,67 +301,65 @@ mod tests { 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] - │ [Stage 7] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ 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)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] - │ 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)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 6), 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] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ 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] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ 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] + │ 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] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 18), input_partitions=4 - │ 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] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@0 as n_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, c_nationkey@0 as c_nationkey] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@0 as o_custkey] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@1)], projection=[o_custkey@1, s_nationkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1995-01-01 AND l_shipdate@10 <= 1996-12-31 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 <= 1996-12-31, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/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 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, 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..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, 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..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/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..p23] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 24), input_partitions=6 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/customer/1.parquet:.., /testdata/tpch/plan_sf1/customer/10.parquet:.., /testdata/tpch/plan_sf1/customer/11.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:.., /testdata/tpch/plan_sf1/customer/12.parquet:.., /testdata/tpch/plan_sf1/customer/13.parquet:.., /testdata/tpch/plan_sf1/customer/14.parquet:..], [/testdata/tpch/plan_sf1/customer/14.parquet:.., /testdata/tpch/plan_sf1/customer/15.parquet:.., /testdata/tpch/plan_sf1/customer/16.parquet:..], [/testdata/tpch/plan_sf1/customer/16.parquet:.., /testdata/tpch/plan_sf1/customer/2.parquet:.., /testdata/tpch/plan_sf1/customer/3.parquet:.., /testdata/tpch/plan_sf1/customer/4.parquet:..], [/testdata/tpch/plan_sf1/customer/4.parquet:.., /testdata/tpch/plan_sf1/customer/5.parquet:.., /testdata/tpch/plan_sf1/customer/6.parquet:.., /testdata/tpch/plan_sf1/customer/7.parquet:..], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([o_custkey@4], 24), input_partitions=6 + │ 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, o_custkey@0 as o_custkey] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@1)], projection=[o_custkey@1, s_nationkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p23] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 24), input_partitions=6 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:.., /testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:.., /testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:.., /testdata/tpch/plan_sf1/orders/16.parquet:.., /testdata/tpch/plan_sf1/orders/2.parquet:..], [/testdata/tpch/plan_sf1/orders/2.parquet:.., /testdata/tpch/plan_sf1/orders/3.parquet:.., /testdata/tpch/plan_sf1/orders/4.parquet:..], [/testdata/tpch/plan_sf1/orders/4.parquet:.., /testdata/tpch/plan_sf1/orders/5.parquet:.., /testdata/tpch/plan_sf1/orders/6.parquet:.., /testdata/tpch/plan_sf1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@1], 24), input_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1995-01-01 AND l_shipdate@10 <= 1996-12-31 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 <= 1996-12-31, required_guarantees=[] └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── "); Ok(()) } @@ -373,85 +370,82 @@ mod tests { assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] - │ [Stage 9] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ 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)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[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)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([o_year@0], 6), 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] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, o_orderdate@3, n_name@5] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ 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] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, l_extendedprice@2, l_discount@3, o_orderdate@5, n_regionkey@6] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ 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] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@4)], projection=[n_regionkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@5] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ 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, c_nationkey@0 as c_nationkey] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@3)], projection=[c_nationkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@6] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 9] => NetworkShuffleExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([o_year@0], 18), input_partitions=4 - │ 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] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, o_orderdate@3, n_name@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@0 as n_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, l_extendedprice@2, l_discount@3, o_orderdate@5, n_regionkey@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ 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] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@4)], projection=[n_regionkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ 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, c_nationkey@0 as c_nationkey] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@3)], projection=[c_nationkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@6] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ 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] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@0 as s_nationkey] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/region/1.parquet, /testdata/tpch/plan_sf1/region/10.parquet, /testdata/tpch/plan_sf1/region/11.parquet], [/testdata/tpch/plan_sf1/region/12.parquet, /testdata/tpch/plan_sf1/region/13.parquet, /testdata/tpch/plan_sf1/region/14.parquet], [/testdata/tpch/plan_sf1/region/15.parquet, /testdata/tpch/plan_sf1/region/16.parquet, /testdata/tpch/plan_sf1/region/2.parquet], [/testdata/tpch/plan_sf1/region/3.parquet, /testdata/tpch/plan_sf1/region/4.parquet, /testdata/tpch/plan_sf1/region/5.parquet], [/testdata/tpch/plan_sf1/region/6.parquet, /testdata/tpch/plan_sf1/region/7.parquet, /testdata/tpch/plan_sf1/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 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.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 2 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 2 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p23] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 24), input_partitions=6 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/customer/1.parquet:.., /testdata/tpch/plan_sf1/customer/10.parquet:.., /testdata/tpch/plan_sf1/customer/11.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:.., /testdata/tpch/plan_sf1/customer/12.parquet:.., /testdata/tpch/plan_sf1/customer/13.parquet:.., /testdata/tpch/plan_sf1/customer/14.parquet:..], [/testdata/tpch/plan_sf1/customer/14.parquet:.., /testdata/tpch/plan_sf1/customer/15.parquet:.., /testdata/tpch/plan_sf1/customer/16.parquet:..], [/testdata/tpch/plan_sf1/customer/16.parquet:.., /testdata/tpch/plan_sf1/customer/2.parquet:.., /testdata/tpch/plan_sf1/customer/3.parquet:.., /testdata/tpch/plan_sf1/customer/4.parquet:..], [/testdata/tpch/plan_sf1/customer/4.parquet:.., /testdata/tpch/plan_sf1/customer/5.parquet:.., /testdata/tpch/plan_sf1/customer/6.parquet:.., /testdata/tpch/plan_sf1/customer/7.parquet:..], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p23] t1:[p0..p23] + │ RepartitionExec: partitioning=Hash([o_custkey@3], 24), 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] + │ 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] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p0..p11] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 12), input_partitions=6 │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1995-01-01 AND o_orderdate@4 <= 1996-12-31 AND DynamicFilter [ empty ], 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=[] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,p5] + │ DataSourceExec: file_groups={12 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:..], [/testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1995-01-01 AND o_orderdate@4 <= 1996-12-31 AND DynamicFilter [ empty ], 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 6 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, predicate=p_type@4 = 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 8 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 12), 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] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/part/1.parquet:.., /testdata/tpch/plan_sf1/part/10.parquet:.., /testdata/tpch/plan_sf1/part/11.parquet:..], [/testdata/tpch/plan_sf1/part/11.parquet:.., /testdata/tpch/plan_sf1/part/12.parquet:.., /testdata/tpch/plan_sf1/part/13.parquet:.., /testdata/tpch/plan_sf1/part/14.parquet:..], [/testdata/tpch/plan_sf1/part/14.parquet:.., /testdata/tpch/plan_sf1/part/15.parquet:.., /testdata/tpch/plan_sf1/part/16.parquet:..], [/testdata/tpch/plan_sf1/part/16.parquet:.., /testdata/tpch/plan_sf1/part/2.parquet:.., /testdata/tpch/plan_sf1/part/3.parquet:.., /testdata/tpch/plan_sf1/part/4.parquet:..], [/testdata/tpch/plan_sf1/part/4.parquet:.., /testdata/tpch/plan_sf1/part/5.parquet:.., /testdata/tpch/plan_sf1/part/6.parquet:.., /testdata/tpch/plan_sf1/part/7.parquet:..], ...]}, projection=[p_partkey, p_type], file_type=parquet, predicate=p_type@4 = 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)] + └────────────────────────────────────────────────── "#); Ok(()) } @@ -462,65 +456,64 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC] - │ [Stage 7] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ 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)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] - │ 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)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 6), 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] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[n_name@1, l_quantity@2, l_extendedprice@3, l_discount@4, ps_supplycost@6, o_orderdate@7] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ 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] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 18), input_partitions=4 - │ 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] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[n_name@1, l_quantity@2, l_extendedprice@3, l_discount@4, ps_supplycost@6, o_orderdate@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ 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] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_quantity@2 as l_quantity, l_extendedprice@3 as l_extendedprice, l_discount@4 as l_discount, s_nationkey@5 as s_nationkey, ps_supplycost@0 as ps_supplycost] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_suppkey@1, l_suppkey@2), (ps_partkey@0, l_partkey@1)], projection=[ps_supplycost@2, l_orderkey@3, l_quantity@6, l_extendedprice@7, l_discount@8, s_nationkey@9] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ 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] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@2)], projection=[s_nationkey@1, l_orderkey@2, l_partkey@3, l_suppkey@4, l_quantity@5, l_extendedprice@6, l_discount@7] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet], [/testdata/tpch/plan_sf0.02/partsupp/10.parquet], [/testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet], [/testdata/tpch/plan_sf0.02/partsupp/13.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p23] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 24), input_partitions=6 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:.., /testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:.., /testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:.., /testdata/tpch/plan_sf1/orders/16.parquet:.., /testdata/tpch/plan_sf1/orders/2.parquet:..], [/testdata/tpch/plan_sf1/orders/2.parquet:.., /testdata/tpch/plan_sf1/orders/3.parquet:.., /testdata/tpch/plan_sf1/orders/4.parquet:..], [/testdata/tpch/plan_sf1/orders/4.parquet:.., /testdata/tpch/plan_sf1/orders/5.parquet:.., /testdata/tpch/plan_sf1/orders/6.parquet:.., /testdata/tpch/plan_sf1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=6 + │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_quantity@2 as l_quantity, l_extendedprice@3 as l_extendedprice, l_discount@4 as l_discount, s_nationkey@5 as s_nationkey, ps_supplycost@0 as ps_supplycost] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, l_suppkey@2), (ps_partkey@0, l_partkey@1)], projection=[ps_supplycost@2, l_orderkey@3, l_quantity@6, l_extendedprice@7, l_discount@8, s_nationkey@9] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] + │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 24), input_partitions=6 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,p5] + │ DataSourceExec: file_groups={12 groups: [[/testdata/tpch/plan_sf1/partsupp/1.parquet:.., /testdata/tpch/plan_sf1/partsupp/10.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:.., /testdata/tpch/plan_sf1/partsupp/12.parquet:..], [/testdata/tpch/plan_sf1/partsupp/13.parquet:.., /testdata/tpch/plan_sf1/partsupp/14.parquet:..], [/testdata/tpch/plan_sf1/partsupp/14.parquet:.., /testdata/tpch/plan_sf1/partsupp/15.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green% + ┌───── Stage 6 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 24), input_partitions=6 + │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount, s_nationkey@0 as s_nationkey] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@2)], projection=[s_nationkey@1, l_orderkey@2, l_partkey@3, l_suppkey@4, l_quantity@5, l_extendedprice@6, l_discount@7] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/part/1.parquet:.., /testdata/tpch/plan_sf1/part/10.parquet:.., /testdata/tpch/plan_sf1/part/11.parquet:..], [/testdata/tpch/plan_sf1/part/11.parquet:.., /testdata/tpch/plan_sf1/part/12.parquet:.., /testdata/tpch/plan_sf1/part/13.parquet:.., /testdata/tpch/plan_sf1/part/14.parquet:..], [/testdata/tpch/plan_sf1/part/14.parquet:.., /testdata/tpch/plan_sf1/part/15.parquet:.., /testdata/tpch/plan_sf1/part/16.parquet:..], [/testdata/tpch/plan_sf1/part/16.parquet:.., /testdata/tpch/plan_sf1/part/2.parquet:.., /testdata/tpch/plan_sf1/part/3.parquet:.., /testdata/tpch/plan_sf1/part/4.parquet:..], [/testdata/tpch/plan_sf1/part/4.parquet:.., /testdata/tpch/plan_sf1/part/5.parquet:.., /testdata/tpch/plan_sf1/part/6.parquet:.., /testdata/tpch/plan_sf1/part/7.parquet:..], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green% + └────────────────────────────────────────────────── "); Ok(()) } @@ -531,47 +524,52 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@2 DESC] - │ [Stage 5] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + ┌───── Stage 7 ── 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)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 18), input_partitions=4 + ┌───── Stage 6 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[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] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], projection=[n_name@1, c_custkey@2, c_name@3, c_address@4, c_phone@6, c_acctbal@7, c_comment@8, l_extendedprice@9, l_discount@10] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@8 = R AND DynamicFilter [ empty ], pruning_predicate=l_returnflag_null_count@2 != row_count@3 AND l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, required_guarantees=[l_returnflag in (R)] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ HashJoinExec: mode=Partitioned, 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] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ 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,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1993-10-01 AND o_orderdate@4 < 1994-01-01 AND DynamicFilter [ empty ], 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=[] + ┌───── Stage 4 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] + │ RepartitionExec: partitioning=Hash([o_orderkey@7], 24), input_partitions=6 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=6 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,p5] + │ DataSourceExec: file_groups={12 groups: [[/testdata/tpch/plan_sf1/customer/1.parquet:.., /testdata/tpch/plan_sf1/customer/10.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:.., /testdata/tpch/plan_sf1/customer/12.parquet:..], [/testdata/tpch/plan_sf1/customer/13.parquet:.., /testdata/tpch/plan_sf1/customer/14.parquet:..], [/testdata/tpch/plan_sf1/customer/14.parquet:.., /testdata/tpch/plan_sf1/customer/15.parquet:..], ...]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p17] t1:[p0..p17] + │ RepartitionExec: partitioning=Hash([o_custkey@1], 18), input_partitions=6 + │ 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,p2,p3,p4,p5,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,p5] + │ DataSourceExec: file_groups={12 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:..], [/testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1993-10-01 AND o_orderdate@4 < 1994-01-01 AND DynamicFilter [ empty ], 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=[] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=6 + │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@8 = R AND DynamicFilter [ empty ], pruning_predicate=l_returnflag_null_count@2 != row_count@3 AND l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, required_guarantees=[l_returnflag in (R)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -588,57 +586,53 @@ mod tests { │ ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] │ AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 │ ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as sum(partsupp.ps_supplycost * partsupp.ps_availqty), CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 AS Decimal128(38, 15)) as join_proj_push_down_1] │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=12, input_tasks=1 │ ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet], [/testdata/tpch/plan_sf0.02/partsupp/10.parquet], [/testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet], [/testdata/tpch/plan_sf0.02/partsupp/13.parquet], ...]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,p5] + │ DataSourceExec: file_groups={12 groups: [[/testdata/tpch/plan_sf1/partsupp/1.parquet:.., /testdata/tpch/plan_sf1/partsupp/10.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:.., /testdata/tpch/plan_sf1/partsupp/12.parquet:..], [/testdata/tpch/plan_sf1/partsupp/13.parquet:.., /testdata/tpch/plan_sf1/partsupp/14.parquet:..], [/testdata/tpch/plan_sf1/partsupp/14.parquet:.., /testdata/tpch/plan_sf1/partsupp/15.parquet:..], ...]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=6, consumer_tasks=2, output_partitions=12 │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, 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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/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..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=6, consumer_tasks=2, output_partitions=12 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([ps_partkey@0], 6), input_partitions=4 + ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 6), input_partitions=5 │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=18, input_tasks=1 │ 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] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_availqty@4, ps_supplycost@5] │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet], [/testdata/tpch/plan_sf0.02/partsupp/10.parquet], [/testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet], [/testdata/tpch/plan_sf0.02/partsupp/13.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=18, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={15 groups: [[/testdata/tpch/plan_sf1/partsupp/1.parquet:..], [/testdata/tpch/plan_sf1/partsupp/10.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:.., /testdata/tpch/plan_sf1/partsupp/12.parquet:..], [/testdata/tpch/plan_sf1/partsupp/13.parquet:.., /testdata/tpch/plan_sf1/partsupp/14.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 4 ── Tasks: t0:[p0..p17] + │ BroadcastExec: input_partitions=6, consumer_tasks=3, output_partitions=18 │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, 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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/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 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 5 ── Tasks: t0:[p0..p17] + │ BroadcastExec: input_partitions=6, consumer_tasks=3, output_partitions=18 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -650,28 +644,30 @@ mod tests { assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] - │ [Stage 3] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[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)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_shipmode@0], 18), input_partitions=4 + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([l_shipmode@0], 12), 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)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilter [ empty ] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=6 │ 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,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@14 = MAIL OR l_shipmode@14 = SHIP) AND l_receiptdate@12 > l_commitdate@11 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 1994-01-01 AND l_receiptdate@12 < 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)] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@14 = MAIL OR l_shipmode@14 = SHIP) AND l_receiptdate@12 > l_commitdate@11 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 1994-01-01 AND l_receiptdate@12 < 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..p23] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 24), input_partitions=6 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:.., /testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:.., /testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:.., /testdata/tpch/plan_sf1/orders/16.parquet:.., /testdata/tpch/plan_sf1/orders/2.parquet:..], [/testdata/tpch/plan_sf1/orders/2.parquet:.., /testdata/tpch/plan_sf1/orders/3.parquet:.., /testdata/tpch/plan_sf1/orders/4.parquet:..], [/testdata/tpch/plan_sf1/orders/4.parquet:.., /testdata/tpch/plan_sf1/orders/5.parquet:.., /testdata/tpch/plan_sf1/orders/6.parquet:.., /testdata/tpch/plan_sf1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "#); Ok(()) @@ -683,31 +679,32 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC] - │ [Stage 3] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[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))] - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] + ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] │ RepartitionExec: partitioning=Hash([c_count@0], 12), input_partitions=6 │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] - │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] + │ HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=4 - │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] - │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] - │ CoalescePartitionsExec - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey], file_type=parquet - │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@8 NOT LIKE %special%requests% + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 12), input_partitions=6 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/customer/1.parquet:.., /testdata/tpch/plan_sf1/customer/10.parquet:.., /testdata/tpch/plan_sf1/customer/11.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:.., /testdata/tpch/plan_sf1/customer/12.parquet:.., /testdata/tpch/plan_sf1/customer/13.parquet:.., /testdata/tpch/plan_sf1/customer/14.parquet:..], [/testdata/tpch/plan_sf1/customer/14.parquet:.., /testdata/tpch/plan_sf1/customer/15.parquet:.., /testdata/tpch/plan_sf1/customer/16.parquet:..], [/testdata/tpch/plan_sf1/customer/16.parquet:.., /testdata/tpch/plan_sf1/customer/2.parquet:.., /testdata/tpch/plan_sf1/customer/3.parquet:.., /testdata/tpch/plan_sf1/customer/4.parquet:..], [/testdata/tpch/plan_sf1/customer/4.parquet:.., /testdata/tpch/plan_sf1/customer/5.parquet:.., /testdata/tpch/plan_sf1/customer/6.parquet:.., /testdata/tpch/plan_sf1/customer/7.parquet:..], ...]}, projection=[c_custkey], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] + │ RepartitionExec: partitioning=Hash([o_custkey@1], 12), input_partitions=6 + │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:..], [/testdata/tpch/plan_sf1/orders/10.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/12.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@8 NOT LIKE %special%requests% └────────────────────────────────────────────────── "); Ok(()) @@ -721,22 +718,24 @@ mod tests { │ ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] │ AggregateExec: mode=Final, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + ┌───── 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] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1995-09-01 AND l_shipdate@10 < 1995-10-01 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p23] + │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=6 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/part/1.parquet:.., /testdata/tpch/plan_sf1/part/10.parquet:.., /testdata/tpch/plan_sf1/part/11.parquet:..], [/testdata/tpch/plan_sf1/part/11.parquet:.., /testdata/tpch/plan_sf1/part/12.parquet:.., /testdata/tpch/plan_sf1/part/13.parquet:.., /testdata/tpch/plan_sf1/part/14.parquet:..], [/testdata/tpch/plan_sf1/part/14.parquet:.., /testdata/tpch/plan_sf1/part/15.parquet:.., /testdata/tpch/plan_sf1/part/16.parquet:..], [/testdata/tpch/plan_sf1/part/16.parquet:.., /testdata/tpch/plan_sf1/part/2.parquet:.., /testdata/tpch/plan_sf1/part/3.parquet:.., /testdata/tpch/plan_sf1/part/4.parquet:..], [/testdata/tpch/plan_sf1/part/4.parquet:.., /testdata/tpch/plan_sf1/part/5.parquet:.., /testdata/tpch/plan_sf1/part/6.parquet:.., /testdata/tpch/plan_sf1/part/7.parquet:..], ...]}, projection=[p_partkey, p_type], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=6 + │ FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1995-09-01 AND l_shipdate@10 < 1995-10-01 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[] └────────────────────────────────────────────────── "#); Ok(()) @@ -757,7 +756,7 @@ mod tests { │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=1 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 │ ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 @@ -766,32 +765,31 @@ mod tests { │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 │ AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[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)] │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] - │ RepartitionExec: partitioning=Hash([l_suppkey@0], 12), input_partitions=4 + ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 18), input_partitions=6 │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1996-01-01 AND l_shipdate@10 < 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=[] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1996-01-01 AND l_shipdate@10 < 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 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=parquet + ┌───── Stage 4 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([l_suppkey@0], 24), input_partitions=4 + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1996-01-01 AND l_shipdate@10 < 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=[] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1996-01-01 AND l_shipdate@10 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -803,44 +801,42 @@ mod tests { 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] - │ [Stage 5] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..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)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] - │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 12), input_partitions=6 + ┌───── Stage 4 ── Tasks: t0:[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=[] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=4 + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 6), 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=[] │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=12, input_tasks=1 │ 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] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_brand@1, p_type@2, p_size@3, ps_suppkey@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet], [/testdata/tpch/plan_sf0.02/partsupp/10.parquet], [/testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet], [/testdata/tpch/plan_sf0.02/partsupp/13.parquet], ...]}, projection=[ps_partkey, ps_suppkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,p5] + │ DataSourceExec: file_groups={12 groups: [[/testdata/tpch/plan_sf1/partsupp/1.parquet:.., /testdata/tpch/plan_sf1/partsupp/10.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:.., /testdata/tpch/plan_sf1/partsupp/12.parquet:..], [/testdata/tpch/plan_sf1/partsupp/13.parquet:.., /testdata/tpch/plan_sf1/partsupp/14.parquet:..], [/testdata/tpch/plan_sf1/partsupp/14.parquet:.., /testdata/tpch/plan_sf1/partsupp/15.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=6, consumer_tasks=2, output_partitions=12 │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@6 LIKE %Customer%Complaints% + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@6 LIKE %Customer%Complaints% └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 2 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=6, consumer_tasks=2, output_partitions=12 │ FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND p_size@3 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]) - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@3 != Brand#45 AND p_type@4 NOT LIKE MEDIUM POLISHED% AND p_size@5 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]), pruning_predicate=p_brand_null_count@2 != row_count@3 AND (p_brand_min@0 != Brand#45 OR Brand#45 != p_brand_max@1) AND p_type_null_count@6 != row_count@3 AND (p_type_min@4 NOT LIKE MEDIUM POLISHED% OR p_type_max@5 NOT LIKE MEDIUM POLISHED%) AND (p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 49 AND 49 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 14 AND 14 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 23 AND 23 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 45 AND 45 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 19 AND 19 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 3 AND 3 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 36 AND 36 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 9 AND 9 <= p_size_max@8), required_guarantees=[p_brand not in (Brand#45), p_size in (14, 19, 23, 3, 36, 45, 49, 9)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/part/1.parquet:.., /testdata/tpch/plan_sf1/part/10.parquet:.., /testdata/tpch/plan_sf1/part/11.parquet:..], [/testdata/tpch/plan_sf1/part/11.parquet:.., /testdata/tpch/plan_sf1/part/12.parquet:.., /testdata/tpch/plan_sf1/part/13.parquet:.., /testdata/tpch/plan_sf1/part/14.parquet:..], [/testdata/tpch/plan_sf1/part/14.parquet:.., /testdata/tpch/plan_sf1/part/15.parquet:.., /testdata/tpch/plan_sf1/part/16.parquet:..], [/testdata/tpch/plan_sf1/part/16.parquet:.., /testdata/tpch/plan_sf1/part/2.parquet:.., /testdata/tpch/plan_sf1/part/3.parquet:.., /testdata/tpch/plan_sf1/part/4.parquet:..], [/testdata/tpch/plan_sf1/part/4.parquet:.., /testdata/tpch/plan_sf1/part/5.parquet:.., /testdata/tpch/plan_sf1/part/6.parquet:.., /testdata/tpch/plan_sf1/part/7.parquet:..], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@3 != Brand#45 AND p_type@4 NOT LIKE MEDIUM POLISHED% AND p_size@5 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]), 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -858,33 +854,31 @@ mod tests { └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ 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] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 │ ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 2 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[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] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 1 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@3 = Brand#23 AND p_container@6 = MED BOX, pruning_predicate=p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5, required_guarantees=[p_brand in (Brand#23), p_container in (MED BOX)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/part/1.parquet:.., /testdata/tpch/plan_sf1/part/10.parquet:.., /testdata/tpch/plan_sf1/part/11.parquet:..], [/testdata/tpch/plan_sf1/part/11.parquet:.., /testdata/tpch/plan_sf1/part/12.parquet:.., /testdata/tpch/plan_sf1/part/13.parquet:.., /testdata/tpch/plan_sf1/part/14.parquet:..], [/testdata/tpch/plan_sf1/part/14.parquet:.., /testdata/tpch/plan_sf1/part/15.parquet:.., /testdata/tpch/plan_sf1/part/16.parquet:..], [/testdata/tpch/plan_sf1/part/16.parquet:.., /testdata/tpch/plan_sf1/part/2.parquet:.., /testdata/tpch/plan_sf1/part/3.parquet:.., /testdata/tpch/plan_sf1/part/4.parquet:..], [/testdata/tpch/plan_sf1/part/4.parquet:.., /testdata/tpch/plan_sf1/part/5.parquet:.., /testdata/tpch/plan_sf1/part/6.parquet:.., /testdata/tpch/plan_sf1/part/7.parquet:..], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@3 = Brand#23 AND p_container@6 = 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..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=4 + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_partkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_partkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -896,50 +890,45 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] - │ [Stage 6] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + ┌───── Stage 6 ── 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)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ AggregateExec: mode=SinglePartitioned, 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)] + │ HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] + │ 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)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ 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] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 18), input_partitions=4 - │ 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)] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p23] t1:[p24..p47] t2:[p48..p71] - │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 - │ 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)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 4 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([o_orderkey@2], 24), input_partitions=6 + │ 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] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p23] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 24), input_partitions=6 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/customer/1.parquet:.., /testdata/tpch/plan_sf1/customer/10.parquet:.., /testdata/tpch/plan_sf1/customer/11.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:.., /testdata/tpch/plan_sf1/customer/12.parquet:.., /testdata/tpch/plan_sf1/customer/13.parquet:.., /testdata/tpch/plan_sf1/customer/14.parquet:..], [/testdata/tpch/plan_sf1/customer/14.parquet:.., /testdata/tpch/plan_sf1/customer/15.parquet:.., /testdata/tpch/plan_sf1/customer/16.parquet:..], [/testdata/tpch/plan_sf1/customer/16.parquet:.., /testdata/tpch/plan_sf1/customer/2.parquet:.., /testdata/tpch/plan_sf1/customer/3.parquet:.., /testdata/tpch/plan_sf1/customer/4.parquet:..], [/testdata/tpch/plan_sf1/customer/4.parquet:.., /testdata/tpch/plan_sf1/customer/5.parquet:.., /testdata/tpch/plan_sf1/customer/6.parquet:.., /testdata/tpch/plan_sf1/customer/7.parquet:..], ...]}, projection=[c_custkey, c_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=4 - │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] + │ RepartitionExec: partitioning=Hash([o_custkey@1], 24), input_partitions=5 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={15 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:..], [/testdata/tpch/plan_sf1/orders/10.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:..], [/testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet - └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=6 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── "); Ok(()) } @@ -952,22 +941,21 @@ mod tests { │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] t3:[p18..p23] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7] │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 │ 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] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=parquet, predicate=(l_quantity@4 >= Some(100),15,2 AND l_quantity@4 <= Some(1100),15,2 OR l_quantity@4 >= Some(1000),15,2 AND l_quantity@4 <= Some(2000),15,2 OR l_quantity@4 >= Some(2000),15,2 AND l_quantity@4 <= Some(3000),15,2) AND (l_shipmode@14 = AIR OR l_shipmode@14 = AIR REG) AND l_shipinstruct@13 = DELIVER IN PERSON AND DynamicFilter [ empty ], pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8, required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=parquet, predicate=(l_quantity@4 >= Some(100),15,2 AND l_quantity@4 <= Some(1100),15,2 OR l_quantity@4 >= Some(1000),15,2 AND l_quantity@4 <= Some(2000),15,2 OR l_quantity@4 >= Some(2000),15,2 AND l_quantity@4 <= Some(3000),15,2) AND (l_shipmode@14 = AIR OR l_shipmode@14 = AIR REG) AND l_shipinstruct@13 = DELIVER IN PERSON AND DynamicFilter [ empty ], pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8, required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 1 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 │ FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) AND p_size@2 >= 1 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@3 = Brand#12 AND p_container@6 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@5 <= 5 OR p_brand@3 = Brand#23 AND p_container@6 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@5 <= 10 OR p_brand@3 = Brand#34 AND p_container@6 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@5 <= 15) AND p_size@5 >= 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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/part/1.parquet:.., /testdata/tpch/plan_sf1/part/10.parquet:.., /testdata/tpch/plan_sf1/part/11.parquet:..], [/testdata/tpch/plan_sf1/part/11.parquet:.., /testdata/tpch/plan_sf1/part/12.parquet:.., /testdata/tpch/plan_sf1/part/13.parquet:.., /testdata/tpch/plan_sf1/part/14.parquet:..], [/testdata/tpch/plan_sf1/part/14.parquet:.., /testdata/tpch/plan_sf1/part/15.parquet:.., /testdata/tpch/plan_sf1/part/16.parquet:..], [/testdata/tpch/plan_sf1/part/16.parquet:.., /testdata/tpch/plan_sf1/part/2.parquet:.., /testdata/tpch/plan_sf1/part/3.parquet:.., /testdata/tpch/plan_sf1/part/4.parquet:..], [/testdata/tpch/plan_sf1/part/4.parquet:.., /testdata/tpch/plan_sf1/part/5.parquet:.., /testdata/tpch/plan_sf1/part/6.parquet:.., /testdata/tpch/plan_sf1/part/7.parquet:..], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@3 = Brand#12 AND p_container@6 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@5 <= 5 OR p_brand@3 = Brand#23 AND p_container@6 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@5 <= 10 OR p_brand@3 = Brand#34 AND p_container@6 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@5 <= 15) AND p_size@5 >= 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -982,43 +970,37 @@ mod tests { │ SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=6, consumer_tasks=1, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=4, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=parquet + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=6, consumer_tasks=1, output_partitions=6 + │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ HashJoinExec: mode=Partitioned, 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] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=2 │ ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] + │ RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 6), input_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=12, input_tasks=1 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,p5] + │ DataSourceExec: file_groups={12 groups: [[/testdata/tpch/plan_sf1/partsupp/1.parquet:.., /testdata/tpch/plan_sf1/partsupp/10.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:..], [/testdata/tpch/plan_sf1/partsupp/11.parquet:.., /testdata/tpch/plan_sf1/partsupp/12.parquet:..], [/testdata/tpch/plan_sf1/partsupp/13.parquet:.., /testdata/tpch/plan_sf1/partsupp/14.parquet:..], [/testdata/tpch/plan_sf1/partsupp/14.parquet:.., /testdata/tpch/plan_sf1/partsupp/15.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, 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 1 ── Tasks: t0:[p0..p11] + │ BroadcastExec: input_partitions=6, consumer_tasks=2, output_partitions=12 + │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/part/1.parquet:.., /testdata/tpch/plan_sf1/part/10.parquet:.., /testdata/tpch/plan_sf1/part/11.parquet:..], [/testdata/tpch/plan_sf1/part/11.parquet:.., /testdata/tpch/plan_sf1/part/12.parquet:.., /testdata/tpch/plan_sf1/part/13.parquet:.., /testdata/tpch/plan_sf1/part/14.parquet:..], [/testdata/tpch/plan_sf1/part/14.parquet:.., /testdata/tpch/plan_sf1/part/15.parquet:.., /testdata/tpch/plan_sf1/part/16.parquet:..], [/testdata/tpch/plan_sf1/part/16.parquet:.., /testdata/tpch/plan_sf1/part/2.parquet:.., /testdata/tpch/plan_sf1/part/3.parquet:.., /testdata/tpch/plan_sf1/part/4.parquet:..], [/testdata/tpch/plan_sf1/part/4.parquet:.., /testdata/tpch/plan_sf1/part/5.parquet:.., /testdata/tpch/plan_sf1/part/6.parquet:.., /testdata/tpch/plan_sf1/part/7.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..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ BroadcastExec: input_partitions=4, consumer_tasks=1, output_partitions=4 - │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, 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 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=4 + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=6 │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1994-01-01 AND l_shipdate@10 < 1995-01-01 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01, required_guarantees=[] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1994-01-01 AND l_shipdate@10 < 1995-01-01 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -1033,47 +1015,55 @@ mod tests { │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[s_name@0 as s_name, count(Int64(1))@1 as numwait] │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] - │ RepartitionExec: partitioning=Hash([s_name@0], 6), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] - │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] - │ CoalescePartitionsExec - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 - │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey], file_type=parquet - │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([s_name@0], 6), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] + │ HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] + │ HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 1 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, 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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/nation/1.parquet, /testdata/tpch/plan_sf1/nation/10.parquet, /testdata/tpch/plan_sf1/nation/11.parquet], [/testdata/tpch/plan_sf1/nation/12.parquet, /testdata/tpch/plan_sf1/nation/13.parquet, /testdata/tpch/plan_sf1/nation/14.parquet], [/testdata/tpch/plan_sf1/nation/15.parquet, /testdata/tpch/plan_sf1/nation/16.parquet, /testdata/tpch/plan_sf1/nation/2.parquet], [/testdata/tpch/plan_sf1/nation/3.parquet, /testdata/tpch/plan_sf1/nation/4.parquet, /testdata/tpch/plan_sf1/nation/5.parquet], [/testdata/tpch/plan_sf1/nation/6.parquet, /testdata/tpch/plan_sf1/nation/7.parquet, /testdata/tpch/plan_sf1/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..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 + ┌───── Stage 2 ── Tasks: t0:[p0..p23] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 24), input_partitions=6 │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_orderstatus], file_type=parquet, predicate=o_orderstatus@2 = 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)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:.., /testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:.., /testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:.., /testdata/tpch/plan_sf1/orders/16.parquet:.., /testdata/tpch/plan_sf1/orders/2.parquet:..], [/testdata/tpch/plan_sf1/orders/2.parquet:.., /testdata/tpch/plan_sf1/orders/3.parquet:.., /testdata/tpch/plan_sf1/orders/4.parquet:..], [/testdata/tpch/plan_sf1/orders/4.parquet:.., /testdata/tpch/plan_sf1/orders/5.parquet:.., /testdata/tpch/plan_sf1/orders/6.parquet:.., /testdata/tpch/plan_sf1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderstatus], file_type=parquet, predicate=o_orderstatus@2 = 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)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_name, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@2], 24), input_partitions=6 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=1 + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p23] + │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/supplier/1.parquet, /testdata/tpch/plan_sf1/supplier/10.parquet, /testdata/tpch/plan_sf1/supplier/11.parquet], [/testdata/tpch/plan_sf1/supplier/12.parquet, /testdata/tpch/plan_sf1/supplier/13.parquet, /testdata/tpch/plan_sf1/supplier/14.parquet], [/testdata/tpch/plan_sf1/supplier/15.parquet, /testdata/tpch/plan_sf1/supplier/16.parquet, /testdata/tpch/plan_sf1/supplier/2.parquet], [/testdata/tpch/plan_sf1/supplier/3.parquet, /testdata/tpch/plan_sf1/supplier/4.parquet, /testdata/tpch/plan_sf1/supplier/5.parquet], [/testdata/tpch/plan_sf1/supplier/6.parquet, /testdata/tpch/plan_sf1/supplier/7.parquet, /testdata/tpch/plan_sf1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=6 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_suppkey], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=6 + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,p4,p5,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4,__,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3,p4] + │ DataSourceExec: file_groups={21 groups: [[/testdata/tpch/plan_sf1/lineitem/1.parquet:..], [/testdata/tpch/plan_sf1/lineitem/10.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/11.parquet:..], [/testdata/tpch/plan_sf1/lineitem/12.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 └────────────────────────────────────────────────── "); Ok(()) @@ -1083,35 +1073,25 @@ mod tests { async fn test_tpch_22() -> Result<(), Box> { let plan = test_tpch_query("q22").await?; assert_snapshot!(plan, @r" - ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] - │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] - │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] - │ RepartitionExec: partitioning=Hash([cntrycode@0], 6), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] - │ ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal] - │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > avg(customer.c_acctbal)@0, projection=[avg(customer.c_acctbal)@0, c_phone@1, c_acctbal@2] - │ AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[c_phone@0 as c_phone, c_acctbal@1 as c_acctbal, CAST(c_acctbal@1 AS Decimal128(19, 6)) as join_proj_push_down_1] - │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_custkey], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] - │ FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@5 > Some(0),15,2 AND substr(c_phone@4, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), 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..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@4, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) - └────────────────────────────────────────────────── + SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] + SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] + ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] + AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] + RepartitionExec: partitioning=Hash([cntrycode@0], 6), input_partitions=6 + AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] + ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal] + NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > avg(customer.c_acctbal)@0, projection=[avg(customer.c_acctbal)@0, c_phone@1, c_acctbal@2] + AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] + CoalescePartitionsExec + AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] + FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] + DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/customer/1.parquet:.., /testdata/tpch/plan_sf1/customer/10.parquet:.., /testdata/tpch/plan_sf1/customer/11.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:.., /testdata/tpch/plan_sf1/customer/12.parquet:.., /testdata/tpch/plan_sf1/customer/13.parquet:.., /testdata/tpch/plan_sf1/customer/14.parquet:..], [/testdata/tpch/plan_sf1/customer/14.parquet:.., /testdata/tpch/plan_sf1/customer/15.parquet:.., /testdata/tpch/plan_sf1/customer/16.parquet:..], [/testdata/tpch/plan_sf1/customer/16.parquet:.., /testdata/tpch/plan_sf1/customer/2.parquet:.., /testdata/tpch/plan_sf1/customer/3.parquet:.., /testdata/tpch/plan_sf1/customer/4.parquet:..], [/testdata/tpch/plan_sf1/customer/4.parquet:.., /testdata/tpch/plan_sf1/customer/5.parquet:.., /testdata/tpch/plan_sf1/customer/6.parquet:.., /testdata/tpch/plan_sf1/customer/7.parquet:..], [/testdata/tpch/plan_sf1/customer/7.parquet:.., /testdata/tpch/plan_sf1/customer/8.parquet:.., /testdata/tpch/plan_sf1/customer/9.parquet:..]]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@5 > Some(0),15,2 AND substr(c_phone@4, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), pruning_predicate=c_acctbal_null_count@1 != row_count@2 AND c_acctbal_max@0 > Some(0),15,2, required_guarantees=[] + ProjectionExec: expr=[c_phone@0 as c_phone, c_acctbal@1 as c_acctbal, CAST(c_acctbal@1 AS Decimal128(19, 6)) as join_proj_push_down_1] + HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] + CoalescePartitionsExec + FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) + DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/customer/1.parquet:.., /testdata/tpch/plan_sf1/customer/10.parquet:.., /testdata/tpch/plan_sf1/customer/11.parquet:..], [/testdata/tpch/plan_sf1/customer/11.parquet:.., /testdata/tpch/plan_sf1/customer/12.parquet:.., /testdata/tpch/plan_sf1/customer/13.parquet:.., /testdata/tpch/plan_sf1/customer/14.parquet:..], [/testdata/tpch/plan_sf1/customer/14.parquet:.., /testdata/tpch/plan_sf1/customer/15.parquet:.., /testdata/tpch/plan_sf1/customer/16.parquet:..], [/testdata/tpch/plan_sf1/customer/16.parquet:.., /testdata/tpch/plan_sf1/customer/2.parquet:.., /testdata/tpch/plan_sf1/customer/3.parquet:.., /testdata/tpch/plan_sf1/customer/4.parquet:..], [/testdata/tpch/plan_sf1/customer/4.parquet:.., /testdata/tpch/plan_sf1/customer/5.parquet:.., /testdata/tpch/plan_sf1/customer/6.parquet:.., /testdata/tpch/plan_sf1/customer/7.parquet:..], [/testdata/tpch/plan_sf1/customer/7.parquet:.., /testdata/tpch/plan_sf1/customer/8.parquet:.., /testdata/tpch/plan_sf1/customer/9.parquet:..]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@4, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) + DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf1/orders/1.parquet:.., /testdata/tpch/plan_sf1/orders/10.parquet:.., /testdata/tpch/plan_sf1/orders/11.parquet:..], [/testdata/tpch/plan_sf1/orders/11.parquet:.., /testdata/tpch/plan_sf1/orders/12.parquet:.., /testdata/tpch/plan_sf1/orders/13.parquet:.., /testdata/tpch/plan_sf1/orders/14.parquet:..], [/testdata/tpch/plan_sf1/orders/14.parquet:.., /testdata/tpch/plan_sf1/orders/15.parquet:.., /testdata/tpch/plan_sf1/orders/16.parquet:.., /testdata/tpch/plan_sf1/orders/2.parquet:..], [/testdata/tpch/plan_sf1/orders/2.parquet:.., /testdata/tpch/plan_sf1/orders/3.parquet:.., /testdata/tpch/plan_sf1/orders/4.parquet:..], [/testdata/tpch/plan_sf1/orders/4.parquet:.., /testdata/tpch/plan_sf1/orders/5.parquet:.., /testdata/tpch/plan_sf1/orders/6.parquet:.., /testdata/tpch/plan_sf1/orders/7.parquet:..], [/testdata/tpch/plan_sf1/orders/7.parquet:.., /testdata/tpch/plan_sf1/orders/8.parquet:.., /testdata/tpch/plan_sf1/orders/9.parquet:..]]}, projection=[o_custkey], file_type=parquet "); Ok(()) } @@ -1119,7 +1099,9 @@ mod tests { // test_tpch_query generates and displays a distributed plan for each TPC-H query. async fn test_tpch_query(query_id: &str) -> Result> { let (d_ctx, _guard, _) = start_localhost_context(4, DefaultSessionBuilder).await; - let d_ctx = d_ctx.with_distributed_broadcast_joins(true)?; + let d_ctx = d_ctx + .with_distributed_broadcast_joins(true)? + .with_distributed_bytes_processed_per_partition(BYTES_PROCESSED_PER_PARTITION)?; let data_dir = ensure_tpch_data(TPCH_SCALE_FACTOR, TPCH_DATA_PARTS).await; let sql = tpch::get_query(query_id)?; d_ctx diff --git a/tests/tpch_stats.rs b/tests/tpch_stats.rs new file mode 100644 index 00000000..58035eb4 --- /dev/null +++ b/tests/tpch_stats.rs @@ -0,0 +1,696 @@ +#[cfg(all(feature = "integration", feature = "tpch", test))] +mod tests { + use datafusion::physical_plan::collect; + use datafusion::prelude::SessionContext; + use datafusion_distributed::assert_snapshot; + use datafusion_distributed::test_utils::stats_vs_metrics::{ + StatsVsMetricsDisplayOptions, stats_vs_metrics_display, + }; + use datafusion_distributed::test_utils::{benchmarks_common, tpch}; + use std::error::Error; + use std::fs; + use std::path::Path; + use tokio::sync::OnceCell; + + const TPCH_SCALE_FACTOR: f64 = 1.0; + const TPCH_DATA_PARTS: i32 = 16; + + #[tokio::test] + async fn test_tpch_1() -> Result<(), Box> { + let display = stats_vs_metrics("q1").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=1200243 vs 4 (99%) + SortExec: output_rows=1200243 vs 4 (99%) + ProjectionExec: output_rows=1200243 vs 4 (99%) + AggregateExec: output_rows=1200243 vs 4 (99%) + RepartitionExec: output_rows=1200243 vs 48 (99%) + AggregateExec: output_rows=1200243 vs 48 (99%) + ProjectionExec: output_rows=1200243 vs 5916591 (79%) + FilterExec: output_rows=1200243 vs 5916591 (79%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_2() -> Result<(), Box> { + let display = stats_vs_metrics("q2").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=0 vs 117760 (100%) + SortExec: output_rows=0 vs 117760 (100%) + ProjectionExec: output_rows=0 vs 117760 (100%) + HashJoinExec: output_rows=0 vs 117760 (100%) + RepartitionExec: output_rows=0 vs 164352 (100%) + HashJoinExec: output_rows=0 vs 164352 (100%) + CoalescePartitionsExec: output_rows=16 vs 16 (0%) + FilterExec: output_rows=16 vs 16 (0%) + DataSourceExec: output_rows=80 vs 80 (0%) + ProjectionExec: output_rows=20 vs 47808 (99%) + HashJoinExec: output_rows=20 vs 47808 (99%) + CoalescePartitionsExec: output_rows=400 vs 400 (0%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=500 vs 2988 (83%) + HashJoinExec: output_rows=500 vs 2988 (83%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + HashJoinExec: output_rows=40000 vs 2988 (92%) + CoalescePartitionsExec: output_rows=40000 vs 747 (98%) + FilterExec: output_rows=40000 vs 747 (98%) + DataSourceExec: output_rows=200000 vs 200000 (0%) + DataSourceExec: output_rows=800000 vs 800000 (0%) + RepartitionExec: output_rows=16 vs 117422 (99%) + ProjectionExec: output_rows=16 vs 117422 (99%) + AggregateExec: output_rows=16 vs 117422 (99%) + RepartitionExec: output_rows=16 vs 117422 (99%) + AggregateExec: output_rows=16 vs 117422 (99%) + HashJoinExec: output_rows=16 vs 40693760 (99%) + CoalescePartitionsExec: output_rows=16 vs 16 (0%) + FilterExec: output_rows=16 vs 16 (0%) + DataSourceExec: output_rows=80 vs 80 (0%) + ProjectionExec: output_rows=400 vs 12800000 (99%) + HashJoinExec: output_rows=400 vs 12800000 (99%) + CoalescePartitionsExec: output_rows=400 vs 400 (0%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=10000 vs 800000 (98%) + HashJoinExec: output_rows=10000 vs 800000 (98%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=800000 vs 800000 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_3() -> Result<(), Box> { + let display = stats_vs_metrics("q3").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=1200 vs 11620 (89%) + SortExec: output_rows=1200 vs 11620 (89%) + ProjectionExec: output_rows=1200 vs 11620 (89%) + AggregateExec: output_rows=1200 vs 11620 (89%) + RepartitionExec: output_rows=1200 vs 11620 (89%) + AggregateExec: output_rows=1200 vs 11620 (89%) + HashJoinExec: output_rows=1200 vs 30519 (96%) + CoalescePartitionsExec: output_rows=6000 vs 147126 (95%) + HashJoinExec: output_rows=6000 vs 147126 (95%) + CoalescePartitionsExec: output_rows=30000 vs 30142 (0%) + FilterExec: output_rows=30000 vs 30142 (0%) + DataSourceExec: output_rows=150000 vs 150000 (0%) + FilterExec: output_rows=300000 vs 727305 (58%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + FilterExec: output_rows=1200243 vs 3241776 (62%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_4() -> Result<(), Box> { + let display = stats_vs_metrics("q4").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=300000 vs 5 (99%) + SortExec: output_rows=300000 vs 5 (99%) + ProjectionExec: output_rows=300000 vs 5 (99%) + AggregateExec: output_rows=300000 vs 5 (99%) + RepartitionExec: output_rows=300000 vs 80 (99%) + AggregateExec: output_rows=300000 vs 80 (99%) + HashJoinExec: output_rows=300000 vs 52523 (82%) + RepartitionExec: output_rows=300000 vs 57218 (80%) + FilterExec: output_rows=300000 vs 57218 (80%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + RepartitionExec: output_rows=1200243 vs 3793296 (68%) + FilterExec: output_rows=1200243 vs 3793296 (68%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_5() -> Result<(), Box> { + let display = stats_vs_metrics("q5").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=3 vs 5 (40%) + SortExec: output_rows=3 vs 5 (40%) + ProjectionExec: output_rows=3 vs 5 (40%) + AggregateExec: output_rows=3 vs 5 (40%) + RepartitionExec: output_rows=3 vs 80 (96%) + AggregateExec: output_rows=3 vs 80 (96%) + HashJoinExec: output_rows=3 vs 1854208 (99%) + CoalescePartitionsExec: output_rows=16 vs 16 (0%) + FilterExec: output_rows=16 vs 16 (0%) + DataSourceExec: output_rows=80 vs 80 (0%) + ProjectionExec: output_rows=80 vs 579776 (99%) + HashJoinExec: output_rows=80 vs 579776 (99%) + CoalescePartitionsExec: output_rows=400 vs 400 (0%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=2000 vs 36236 (94%) + HashJoinExec: output_rows=2000 vs 36236 (94%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + HashJoinExec: output_rows=30000 vs 910519 (96%) + RepartitionExec: output_rows=30000 vs 227597 (86%) + HashJoinExec: output_rows=30000 vs 227597 (86%) + RepartitionExec: output_rows=150000 vs 150000 (0%) + DataSourceExec: output_rows=150000 vs 150000 (0%) + RepartitionExec: output_rows=300000 vs 227597 (24%) + FilterExec: output_rows=300000 vs 227597 (24%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + RepartitionExec: output_rows=6001215 vs 6001215 (0%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_6() -> Result<(), Box> { + let display = stats_vs_metrics("q6").await?; + assert_snapshot!(display, @r" + ProjectionExec: output_rows=1 vs 1 (0%) + AggregateExec: output_rows=1 vs 1 (0%) + CoalescePartitionsExec: output_rows=1200243 vs 16 (99%) + AggregateExec: output_rows=1200243 vs 16 (99%) + FilterExec: output_rows=1200243 vs 114160 (90%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_7() -> Result<(), Box> { + let display = stats_vs_metrics("q7").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=0 vs 4 (100%) + SortExec: output_rows=0 vs 4 (100%) + ProjectionExec: output_rows=0 vs 4 (100%) + AggregateExec: output_rows=0 vs 4 (100%) + RepartitionExec: output_rows=0 vs 64 (100%) + AggregateExec: output_rows=0 vs 64 (100%) + ProjectionExec: output_rows=0 vs 1516544 (100%) + HashJoinExec: output_rows=0 vs 1516544 (100%) + CoalescePartitionsExec: output_rows=80 vs 32 (60%) + FilterExec: output_rows=80 vs 32 (60%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=0 vs 2331248 (100%) + HashJoinExec: output_rows=0 vs 2331248 (100%) + CoalescePartitionsExec: output_rows=80 vs 32 (60%) + FilterExec: output_rows=80 vs 32 (60%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=49 vs 1828450 (99%) + HashJoinExec: output_rows=49 vs 1828450 (99%) + RepartitionExec: output_rows=150000 vs 150000 (0%) + DataSourceExec: output_rows=150000 vs 150000 (0%) + RepartitionExec: output_rows=499 vs 1828450 (99%) + ProjectionExec: output_rows=499 vs 1828450 (99%) + HashJoinExec: output_rows=499 vs 1828450 (99%) + RepartitionExec: output_rows=1500000 vs 1500000 (0%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + RepartitionExec: output_rows=2000 vs 1828450 (99%) + HashJoinExec: output_rows=2000 vs 1828450 (99%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + FilterExec: output_rows=1200243 vs 1828450 (34%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_8() -> Result<(), Box> { + let display = stats_vs_metrics("q8").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=0 vs 2 (100%) + SortExec: output_rows=0 vs 2 (100%) + ProjectionExec: output_rows=0 vs 2 (100%) + AggregateExec: output_rows=0 vs 2 (100%) + RepartitionExec: output_rows=0 vs 32 (100%) + AggregateExec: output_rows=0 vs 32 (100%) + ProjectionExec: output_rows=0 vs 10661888 (100%) + HashJoinExec: output_rows=0 vs 10661888 (100%) + CoalescePartitionsExec: output_rows=16 vs 16 (0%) + FilterExec: output_rows=16 vs 16 (0%) + DataSourceExec: output_rows=80 vs 80 (0%) + ProjectionExec: output_rows=0 vs 3427584 (100%) + HashJoinExec: output_rows=0 vs 3427584 (100%) + CoalescePartitionsExec: output_rows=400 vs 400 (0%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=0 vs 214224 (100%) + HashJoinExec: output_rows=0 vs 214224 (100%) + CoalescePartitionsExec: output_rows=400 vs 400 (0%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=0 vs 13389 (100%) + HashJoinExec: output_rows=0 vs 13389 (100%) + RepartitionExec: output_rows=150000 vs 150000 (0%) + DataSourceExec: output_rows=150000 vs 150000 (0%) + RepartitionExec: output_rows=3 vs 13389 (99%) + ProjectionExec: output_rows=3 vs 13389 (99%) + HashJoinExec: output_rows=3 vs 13389 (99%) + RepartitionExec: output_rows=300000 vs 457263 (34%) + FilterExec: output_rows=300000 vs 457263 (34%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + RepartitionExec: output_rows=66 vs 43693 (99%) + ProjectionExec: output_rows=66 vs 43693 (99%) + HashJoinExec: output_rows=66 vs 43693 (99%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + HashJoinExec: output_rows=40000 vs 43693 (8%) + CoalescePartitionsExec: output_rows=40000 vs 1451 (96%) + FilterExec: output_rows=40000 vs 1451 (96%) + DataSourceExec: output_rows=200000 vs 200000 (0%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_9() -> Result<(), Box> { + let display = stats_vs_metrics("q9").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=0 vs 175 (100%) + SortExec: output_rows=0 vs 175 (100%) + ProjectionExec: output_rows=0 vs 175 (100%) + AggregateExec: output_rows=0 vs 175 (100%) + RepartitionExec: output_rows=0 vs 2800 (100%) + AggregateExec: output_rows=0 vs 2800 (100%) + ProjectionExec: output_rows=0 vs 5110464 (100%) + HashJoinExec: output_rows=0 vs 5110464 (100%) + CoalescePartitionsExec: output_rows=400 vs 400 (0%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=1 vs 319404 (99%) + HashJoinExec: output_rows=1 vs 319404 (99%) + RepartitionExec: output_rows=1500000 vs 1500000 (0%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + RepartitionExec: output_rows=8 vs 319404 (99%) + ProjectionExec: output_rows=8 vs 319404 (99%) + HashJoinExec: output_rows=8 vs 319404 (99%) + RepartitionExec: output_rows=800000 vs 800000 (0%) + DataSourceExec: output_rows=800000 vs 800000 (0%) + RepartitionExec: output_rows=66 vs 319404 (99%) + ProjectionExec: output_rows=66 vs 319404 (99%) + HashJoinExec: output_rows=66 vs 319404 (99%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + HashJoinExec: output_rows=40000 vs 319404 (87%) + CoalescePartitionsExec: output_rows=40000 vs 10664 (73%) + FilterExec: output_rows=40000 vs 10664 (73%) + DataSourceExec: output_rows=200000 vs 200000 (0%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_10() -> Result<(), Box> { + let display = stats_vs_metrics("q10").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=16 vs 37967 (99%) + SortExec: output_rows=16 vs 37967 (99%) + ProjectionExec: output_rows=16 vs 37967 (99%) + AggregateExec: output_rows=16 vs 37967 (99%) + RepartitionExec: output_rows=16 vs 48177 (99%) + AggregateExec: output_rows=16 vs 48177 (99%) + ProjectionExec: output_rows=16 vs 1835280 (99%) + HashJoinExec: output_rows=16 vs 1835280 (99%) + CoalescePartitionsExec: output_rows=400 vs 400 (0%) + DataSourceExec: output_rows=400 vs 400 (0%) + HashJoinExec: output_rows=6000 vs 114705 (94%) + RepartitionExec: output_rows=30000 vs 57069 (47%) + HashJoinExec: output_rows=30000 vs 57069 (47%) + RepartitionExec: output_rows=150000 vs 150000 (0%) + DataSourceExec: output_rows=150000 vs 150000 (0%) + RepartitionExec: output_rows=300000 vs 57069 (80%) + FilterExec: output_rows=300000 vs 57069 (80%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + RepartitionExec: output_rows=1200243 vs 1478870 (18%) + FilterExec: output_rows=1200243 vs 1478870 (18%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_11() -> Result<(), Box> { + let display = stats_vs_metrics("q11").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=81 vs 1048 (92%) + SortExec: output_rows=81 vs 1048 (92%) + ProjectionExec: output_rows=81 vs 1048 (92%) + NestedLoopJoinExec: output_rows=81 vs 1048 (92%) + ProjectionExec: output_rows=1 vs 1 (0%) + AggregateExec: output_rows=1 vs 1 (0%) + CoalescePartitionsExec: output_rows=80 vs 16 (80%) + AggregateExec: output_rows=80 vs 16 (80%) + HashJoinExec: output_rows=80 vs 506880 (99%) + CoalescePartitionsExec: output_rows=80 vs 16 (80%) + FilterExec: output_rows=80 vs 16 (80%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=10000 vs 800000 (98%) + HashJoinExec: output_rows=10000 vs 800000 (98%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=800000 vs 800000 (0%) + ProjectionExec: output_rows=80 vs 29818 (99%) + AggregateExec: output_rows=80 vs 29818 (99%) + RepartitionExec: output_rows=80 vs 29818 (99%) + AggregateExec: output_rows=80 vs 29818 (99%) + HashJoinExec: output_rows=80 vs 506880 (99%) + CoalescePartitionsExec: output_rows=80 vs 16 (80%) + FilterExec: output_rows=80 vs 16 (80%) + DataSourceExec: output_rows=400 vs 400 (0%) + ProjectionExec: output_rows=10000 vs 800000 (98%) + HashJoinExec: output_rows=10000 vs 800000 (98%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=800000 vs 800000 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_12() -> Result<(), Box> { + let display = stats_vs_metrics("q12").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=300000 vs 2 (99%) + SortExec: output_rows=300000 vs 2 (99%) + ProjectionExec: output_rows=300000 vs 2 (99%) + AggregateExec: output_rows=300000 vs 2 (99%) + RepartitionExec: output_rows=300000 vs 32 (99%) + AggregateExec: output_rows=300000 vs 32 (99%) + HashJoinExec: output_rows=300000 vs 30988 (89%) + RepartitionExec: output_rows=1200243 vs 30988 (97%) + FilterExec: output_rows=1200243 vs 30988 (97%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + RepartitionExec: output_rows=1500000 vs 1500000 (0%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_13() -> Result<(), Box> { + let display = stats_vs_metrics("q13").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=150000 vs 42 (99%) + SortExec: output_rows=150000 vs 42 (99%) + ProjectionExec: output_rows=150000 vs 42 (99%) + AggregateExec: output_rows=150000 vs 42 (99%) + RepartitionExec: output_rows=150000 vs 596 (99%) + AggregateExec: output_rows=150000 vs 596 (99%) + ProjectionExec: output_rows=150000 vs 150000 (0%) + AggregateExec: output_rows=150000 vs 150000 (0%) + HashJoinExec: output_rows=150000 vs 1533923 (90%) + RepartitionExec: output_rows=150000 vs 150000 (0%) + DataSourceExec: output_rows=150000 vs 150000 (0%) + RepartitionExec: output_rows=300000 vs 1483918 (79%) + FilterExec: output_rows=300000 vs 1483918 (79%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_14() -> Result<(), Box> { + let display = stats_vs_metrics("q14").await?; + assert_snapshot!(display, @r" + ProjectionExec: output_rows=1 vs 1 (0%) + AggregateExec: output_rows=1 vs 1 (0%) + CoalescePartitionsExec: output_rows=40000 vs 16 (99%) + AggregateExec: output_rows=40000 vs 16 (99%) + ProjectionExec: output_rows=40000 vs 75983 (47%) + HashJoinExec: output_rows=40000 vs 75983 (47%) + RepartitionExec: output_rows=200000 vs 200000 (0%) + DataSourceExec: output_rows=200000 vs 200000 (0%) + RepartitionExec: output_rows=1200243 vs 75983 (93%) + FilterExec: output_rows=1200243 vs 75983 (93%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_15() -> Result<(), Box> { + let display = stats_vs_metrics("q15").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=0 vs 1 (100%) + SortExec: output_rows=0 vs 1 (100%) + HashJoinExec: output_rows=0 vs 1 (100%) + AggregateExec: output_rows=1 vs 1 (0%) + CoalescePartitionsExec: output_rows=1200243 vs 16 (99%) + AggregateExec: output_rows=1200243 vs 16 (99%) + ProjectionExec: output_rows=1200243 vs 10000 (99%) + AggregateExec: output_rows=1200243 vs 10000 (99%) + RepartitionExec: output_rows=1200243 vs 98336 (91%) + AggregateExec: output_rows=1200243 vs 98336 (91%) + FilterExec: output_rows=1200243 vs 225954 (81%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + HashJoinExec: output_rows=10000 vs 10000 (0%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + ProjectionExec: output_rows=1200243 vs 10000 (99%) + AggregateExec: output_rows=1200243 vs 10000 (99%) + RepartitionExec: output_rows=1200243 vs 98336 (91%) + AggregateExec: output_rows=1200243 vs 98336 (91%) + FilterExec: output_rows=1200243 vs 225954 (81%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_16() -> Result<(), Box> { + let display = stats_vs_metrics("q16").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=40000 vs 18314 (54%) + SortExec: output_rows=40000 vs 18314 (54%) + ProjectionExec: output_rows=40000 vs 18314 (54%) + AggregateExec: output_rows=40000 vs 18314 (54%) + RepartitionExec: output_rows=40000 vs 95811 (58%) + AggregateExec: output_rows=40000 vs 95811 (58%) + AggregateExec: output_rows=40000 vs 118250 (66%) + RepartitionExec: output_rows=40000 vs 118271 (66%) + AggregateExec: output_rows=40000 vs 118271 (66%) + HashJoinExec: output_rows=40000 vs 118274 (66%) + CoalescePartitionsExec: output_rows=2000 vs 4 (99%) + FilterExec: output_rows=2000 vs 4 (99%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + ProjectionExec: output_rows=40000 vs 118324 (66%) + HashJoinExec: output_rows=40000 vs 118324 (66%) + CoalescePartitionsExec: output_rows=40000 vs 29581 (26%) + FilterExec: output_rows=40000 vs 29581 (26%) + DataSourceExec: output_rows=200000 vs 200000 (0%) + DataSourceExec: output_rows=800000 vs 800000 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_17() -> Result<(), Box> { + let display = stats_vs_metrics("q17").await?; + assert_snapshot!(display, @r" + ProjectionExec: output_rows=1 vs 1 (0%) + AggregateExec: output_rows=1 vs 1 (0%) + CoalescePartitionsExec: output_rows=40000 vs 16 (99%) + AggregateExec: output_rows=40000 vs 16 (99%) + HashJoinExec: output_rows=40000 vs 587 (98%) + RepartitionExec: output_rows=40000 vs 6088 (84%) + ProjectionExec: output_rows=40000 vs 6088 (84%) + HashJoinExec: output_rows=40000 vs 6088 (84%) + CoalescePartitionsExec: output_rows=40000 vs 204 (99%) + FilterExec: output_rows=40000 vs 204 (99%) + DataSourceExec: output_rows=200000 vs 200000 (0%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + ProjectionExec: output_rows=6001215 vs 200000 (96%) + AggregateExec: output_rows=6001215 vs 200000 (96%) + RepartitionExec: output_rows=6001215 vs 2136476 (64%) + AggregateExec: output_rows=6001215 vs 2136476 (64%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_18() -> Result<(), Box> { + let display = stats_vs_metrics("q18").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=150000 vs 57 (99%) + SortExec: output_rows=150000 vs 57 (99%) + AggregateExec: output_rows=150000 vs 57 (99%) + HashJoinExec: output_rows=150000 vs 399 (99%) + FilterExec: output_rows=1200243 vs 57 (99%) + AggregateExec: output_rows=6001215 vs 1500000 (75%) + RepartitionExec: output_rows=6001215 vs 1500000 (75%) + AggregateExec: output_rows=6001215 vs 1500000 (75%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + HashJoinExec: output_rows=150000 vs 6001215 (97%) + RepartitionExec: output_rows=150000 vs 1500000 (90%) + HashJoinExec: output_rows=150000 vs 1500000 (90%) + RepartitionExec: output_rows=150000 vs 150000 (0%) + DataSourceExec: output_rows=150000 vs 150000 (0%) + RepartitionExec: output_rows=1500000 vs 1500000 (0%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + RepartitionExec: output_rows=6001215 vs 6001215 (0%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_19() -> Result<(), Box> { + let display = stats_vs_metrics("q19").await?; + assert_snapshot!(display, @r" + ProjectionExec: output_rows=1 vs 1 (0%) + AggregateExec: output_rows=1 vs 1 (0%) + CoalescePartitionsExec: output_rows=8000 vs 16 (99%) + AggregateExec: output_rows=8000 vs 16 (99%) + HashJoinExec: output_rows=8000 vs 121 (98%) + CoalescePartitionsExec: output_rows=40000 vs 485 (98%) + FilterExec: output_rows=40000 vs 485 (98%) + DataSourceExec: output_rows=200000 vs 200000 (0%) + FilterExec: output_rows=1200243 vs 128371 (89%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_20() -> Result<(), Box> { + let display = stats_vs_metrics("q20").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=80 vs 2976 (97%) + SortExec: output_rows=80 vs 2976 (97%) + HashJoinExec: output_rows=80 vs 2976 (97%) + CoalescePartitionsExec: output_rows=80 vs 6592 (98%) + HashJoinExec: output_rows=80 vs 6592 (98%) + CoalescePartitionsExec: output_rows=80 vs 16 (80%) + FilterExec: output_rows=80 vs 16 (80%) + DataSourceExec: output_rows=400 vs 400 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + HashJoinExec: output_rows=800000 vs 5833 (99%) + RepartitionExec: output_rows=800000 vs 8508 (98%) + HashJoinExec: output_rows=800000 vs 8508 (98%) + CoalescePartitionsExec: output_rows=40000 vs 2127 (94%) + FilterExec: output_rows=40000 vs 2127 (94%) + DataSourceExec: output_rows=200000 vs 200000 (0%) + DataSourceExec: output_rows=800000 vs 800000 (0%) + ProjectionExec: output_rows=1200243 vs 543210 (54%) + AggregateExec: output_rows=1200243 vs 543210 (54%) + RepartitionExec: output_rows=1200243 vs 866468 (27%) + AggregateExec: output_rows=1200243 vs 866468 (27%) + FilterExec: output_rows=1200243 vs 909455 (24%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_21() -> Result<(), Box> { + let display = stats_vs_metrics("q21").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=0 vs 411 (100%) + SortExec: output_rows=0 vs 411 (100%) + ProjectionExec: output_rows=0 vs 411 (100%) + AggregateExec: output_rows=0 vs 411 (100%) + RepartitionExec: output_rows=0 vs 3032 (100%) + AggregateExec: output_rows=0 vs 3032 (100%) + HashJoinExec: output_rows=0 vs 66256 (100%) + HashJoinExec: output_rows=0 vs 1169424 (100%) + HashJoinExec: output_rows=0 vs 1213936 (100%) + CoalescePartitionsExec: output_rows=80 vs 16 (80%) + FilterExec: output_rows=80 vs 16 (80%) + DataSourceExec: output_rows=400 vs 400 (0%) + HashJoinExec: output_rows=99 vs 1828911 (99%) + RepartitionExec: output_rows=300000 vs 729413 (58%) + FilterExec: output_rows=300000 vs 729413 (58%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + RepartitionExec: output_rows=2000 vs 3793296 (99%) + HashJoinExec: output_rows=2000 vs 3793296 (99%) + CoalescePartitionsExec: output_rows=10000 vs 10000 (0%) + DataSourceExec: output_rows=10000 vs 10000 (0%) + FilterExec: output_rows=1200243 vs 3793296 (68%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + RepartitionExec: output_rows=6001215 vs 6001215 (0%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + RepartitionExec: output_rows=1200243 vs 3793296 (68%) + FilterExec: output_rows=1200243 vs 3793296 (68%) + DataSourceExec: output_rows=6001215 vs 6001215 (0%) + "); + Ok(()) + } + + #[tokio::test] + async fn test_tpch_22() -> Result<(), Box> { + let display = stats_vs_metrics("q22").await?; + assert_snapshot!(display, @r" + SortPreservingMergeExec: output_rows=30001 vs 7 (99%) + SortExec: output_rows=30001 vs 7 (99%) + ProjectionExec: output_rows=30001 vs 7 (99%) + AggregateExec: output_rows=30001 vs 7 (99%) + RepartitionExec: output_rows=30001 vs 7 (99%) + AggregateExec: output_rows=30001 vs 7 (99%) + ProjectionExec: output_rows=30001 vs 6384 (78%) + NestedLoopJoinExec: output_rows=30001 vs 6384 (78%) + AggregateExec: output_rows=1 vs 1 (0%) + CoalescePartitionsExec: output_rows=30000 vs 16 (99%) + AggregateExec: output_rows=30000 vs 16 (99%) + FilterExec: output_rows=30000 vs 38120 (21%) + DataSourceExec: output_rows=150000 vs 150000 (0%) + ProjectionExec: output_rows=30000 vs 14086 (53%) + HashJoinExec: output_rows=30000 vs 14086 (53%) + CoalescePartitionsExec: output_rows=30000 vs 42015 (28%) + FilterExec: output_rows=30000 vs 42015 (28%) + DataSourceExec: output_rows=150000 vs 150000 (0%) + DataSourceExec: output_rows=1500000 vs 1500000 (0%) + "); + Ok(()) + } + + async fn stats_vs_metrics(query_id: &str) -> Result> { + let ctx = SessionContext::new(); + let data_dir = ensure_tpch_data(TPCH_SCALE_FACTOR, TPCH_DATA_PARTS).await; + let sql = tpch::get_query(query_id)?; + + benchmarks_common::register_tables(&ctx, &data_dir).await?; + + // Query 15 has three queries in it, one creating the view, the second + // executing, which we want to capture the output of, and the third + // tearing down the view + let plan = if query_id == "q15" { + let queries: Vec<&str> = sql + .split(';') + .map(str::trim) + .filter(|s| !s.is_empty()) + .collect(); + + ctx.sql(queries[0]).await?.collect().await?; + let df = ctx.sql(queries[1]).await?; + let plan = df.create_physical_plan().await?; + ctx.sql(queries[2]).await?.collect().await?; + plan + } else { + let df = ctx.sql(&sql).await?; + df.create_physical_plan().await? + }; + + collect(plan.clone(), ctx.task_ctx()).await?; + Ok(stats_vs_metrics_display( + plan, + StatsVsMetricsDisplayOptions::default().with_display_output_rows(), + )?) + } + + // OnceCell to ensure TPCH tables are generated only once for tests + static INIT_TEST_TPCH_TABLES: OnceCell<()> = OnceCell::const_new(); + + // ensure_tpch_data initializes the TPCH data on disk. + pub async fn ensure_tpch_data(sf: f64, parts: i32) -> std::path::PathBuf { + let data_dir = + Path::new(env!("CARGO_MANIFEST_DIR")).join(format!("testdata/tpch/plan_sf{sf}")); + INIT_TEST_TPCH_TABLES + .get_or_init(|| async { + if !fs::exists(&data_dir).unwrap() { + tpch::generate_tpch_data(&data_dir, sf, parts); + } + }) + .await; + data_dir + } +} diff --git a/tests/udfs.rs b/tests/udfs.rs index 3df06e75..874902f8 100644 --- a/tests/udfs.rs +++ b/tests/udfs.rs @@ -1,23 +1,16 @@ #[cfg(all(feature = "integration", test))] mod tests { - use arrow::datatypes::{Field, Schema}; - use arrow::util::pretty::pretty_format_batches; use datafusion::arrow::datatypes::DataType; use datafusion::error::DataFusionError; use datafusion::execution::{SessionState, SessionStateBuilder}; use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, Volatility, }; - use datafusion::physical_expr::expressions::lit; - use datafusion::physical_expr::{Partitioning, ScalarFunctionExpr}; - use datafusion::physical_optimizer::PhysicalOptimizerRule; - use datafusion::physical_plan::empty::EmptyExec; - use datafusion::physical_plan::repartition::RepartitionExec; - use datafusion::physical_plan::{ExecutionPlan, execute_stream}; + use datafusion::physical_plan::execute_stream; use datafusion_distributed::test_utils::localhost::start_localhost_context; + use datafusion_distributed::test_utils::parquet::register_parquet_tables; use datafusion_distributed::{ - DistributedExt, DistributedPhysicalOptimizerRule, WorkerQueryContext, assert_snapshot, - display_plan_ascii, + DistributedExt, WorkerQueryContext, assert_snapshot, display_plan_ascii, }; use futures::TryStreamExt; use std::any::Any; @@ -32,59 +25,36 @@ mod tests { let (mut ctx, _guard, _) = start_localhost_context(3, build_state).await; ctx = SessionStateBuilder::from(ctx.state()) - .with_distributed_task_estimator(2) + .with_distributed_bytes_processed_per_partition(100)? .with_scalar_functions(vec![udf()]) .build() .into(); - let wrap = |input: Arc| -> Arc { - Arc::new( - RepartitionExec::try_new( - input, - Partitioning::Hash( - vec![Arc::new(ScalarFunctionExpr::new( - "test_udf", - udf(), - vec![lit(1)], - Arc::new(Field::new("return", DataType::Int32, false)), - Default::default(), - ))], - 1, - ), - ) - .unwrap(), - ) - }; + let query = r#"SELECT "MinTemp" FROM weather WHERE test_udf("MinTemp") > 20.0"#; + register_parquet_tables(&ctx).await?; + let df = ctx.sql(query).await?; + let plan = df.create_physical_plan().await?; - let node = wrap(wrap(Arc::new(EmptyExec::new(Arc::new(Schema::empty()))))); + let stream = execute_stream(Arc::clone(&plan), ctx.task_ctx())?; + // It should not fail. + stream.try_collect::>().await?; - let physical_distributed = - DistributedPhysicalOptimizerRule.optimize(node, ctx.copied_config().options())?; - - let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref(), false); + let physical_distributed_str = display_plan_ascii(plan.as_ref(), false); assert_snapshot!(physical_distributed_str, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ [Stage 2] => NetworkShuffleExec: output_partitions=1, input_tasks=2 + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p0..p1] - │ RepartitionExec: partitioning=Hash([test_udf(1)], 2), input_partitions=1 - │ EmptyExec + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ FilterExec: test_udf(MinTemp@0) > 20 + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp], file_type=parquet, predicate=test_udf(MinTemp@0) > 20 └────────────────────────────────────────────────── ", ); - let batches = pretty_format_batches( - &execute_stream(physical_distributed, ctx.task_ctx())? - .try_collect::>() - .await?, - )?; - - assert_snapshot!(batches, @r" - ++ - ++ - "); Ok(()) }