diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index aab51efa1e34..d13102c7619e 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -766,9 +766,8 @@ async fn test_aggregate_with_pk2() -> Result<()> { physical_plan_to_string(&df).await, @r" AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[], ordering_mode=Sorted - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 = 1 AND name@1 = a - DataSourceExec: partitions=1, partition_sizes=[1] + FilterExec: id@0 = 1 AND name@1 = a + DataSourceExec: partitions=1, partition_sizes=[1] " ); @@ -815,9 +814,8 @@ async fn test_aggregate_with_pk3() -> Result<()> { physical_plan_to_string(&df).await, @r" AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[], ordering_mode=PartiallySorted([0]) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 = 1 - DataSourceExec: partitions=1, partition_sizes=[1] + FilterExec: id@0 = 1 + DataSourceExec: partitions=1, partition_sizes=[1] " ); @@ -866,9 +864,8 @@ async fn test_aggregate_with_pk4() -> Result<()> { physical_plan_to_string(&df).await, @r" AggregateExec: mode=Single, gby=[id@0 as id], aggr=[], ordering_mode=Sorted - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 = 1 - DataSourceExec: partitions=1, partition_sizes=[1] + FilterExec: id@0 = 1 + DataSourceExec: partitions=1, partition_sizes=[1] " ); @@ -3329,31 +3326,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[a] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ + +---------------+-------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-------------------------------------------------------------------------------------------------------------------------+ " ); @@ -3385,31 +3381,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[a] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ + +---------------+-------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-------------------------------------------------------------------------------------------------------------------------+ " ); diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index 56d48901f284..9d172db246ad 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -199,9 +199,8 @@ fn pushes_global_limit_exec_through_projection_exec() -> Result<()> { let expected = [ "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", - " GlobalLimitExec: skip=0, fetch=5", - " FilterExec: c3@2 > 0", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + " FilterExec: c3@2 > 0, fetch=5", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -310,7 +309,7 @@ fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions() -> R let expected = [ "CoalescePartitionsExec: fetch=5", - " FilterExec: c3@2 > 0", + " FilterExec: c3@2 > 0, fetch=5", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 929de7a5304d..532c15728e24 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -766,7 +766,7 @@ async fn test_physical_plan_display_indent() { assert_snapshot!( actual, - @r###" + @r" SortPreservingMergeExec: [the_min@2 DESC], fetch=10 SortExec: TopK(fetch=10), expr=[the_min@2 DESC], preserve_partitioning=[true] ProjectionExec: expr=[c1@0 as c1, max(aggregate_test_100.c12)@1 as max(aggregate_test_100.c12), min(aggregate_test_100.c12)@2 as the_min] @@ -774,11 +774,10 @@ async fn test_physical_plan_display_indent() { CoalesceBatchesExec: target_batch_size=4096 RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000 AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)] - CoalesceBatchesExec: target_batch_size=4096 - FilterExec: c12@1 < 10 - RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1 - DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true - "### + FilterExec: c12@1 < 10 + RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1 + DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true + " ); } @@ -1013,16 +1012,14 @@ async fn parquet_recursive_projection_pushdown() -> Result<()> { RecursiveQueryExec: name=number_series, is_distinct=false CoalescePartitionsExec ProjectionExec: expr=[id@0 as id, 1 as level] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 = 1 - RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1 - DataSourceExec: file_groups={1 group: [[TMP_DIR/hierarchy.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 = 1, pruning_predicate=id_null_count@2 != row_count@3 AND id_min@0 <= 1 AND 1 <= id_max@1, required_guarantees=[id in (1)] + FilterExec: id@0 = 1 + RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1 + DataSourceExec: file_groups={1 group: [[TMP_DIR/hierarchy.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 = 1, pruning_predicate=id_null_count@2 != row_count@3 AND id_min@0 <= 1 AND 1 <= id_max@1, required_guarantees=[id in (1)] CoalescePartitionsExec ProjectionExec: expr=[id@0 + 1 as ns.id + Int64(1), level@1 + 1 as ns.level + Int64(1)] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: id@0 < 10 - RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1 - WorkTableExec: name=number_series + FilterExec: id@0 < 10 + RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1 + WorkTableExec: name=number_series " ); diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index b19d8d9518b3..bde49e71305b 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -26,7 +26,7 @@ use datafusion_common::error::Result; use datafusion_common::{config::ConfigOptions, internal_err}; use datafusion_physical_expr::Partitioning; use datafusion_physical_plan::{ - async_func::AsyncFuncExec, coalesce_batches::CoalesceBatchesExec, filter::FilterExec, + async_func::AsyncFuncExec, coalesce_batches::CoalesceBatchesExec, joins::HashJoinExec, repartition::RepartitionExec, ExecutionPlan, }; @@ -56,12 +56,7 @@ impl PhysicalOptimizerRule for CoalesceBatches { let target_batch_size = config.execution.batch_size; plan.transform_up(|plan| { let plan_any = plan.as_any(); - // The goal here is to detect operators that could produce small batches and only - // wrap those ones with a CoalesceBatchesExec operator. An alternate approach here - // would be to build the coalescing logic directly into the operators - // See https://github.com/apache/datafusion/issues/139 - let wrap_in_coalesce = plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() + let wrap_in_coalesce = plan_any.downcast_ref::().is_some() // Don't need to add CoalesceBatchesExec after a round robin RepartitionExec || plan_any .downcast_ref::() diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 0c583e1fb973..8a05bfcee3ab 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -26,6 +26,8 @@ use super::{ ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +use crate::coalesce::LimitedBatchCoalescer; +use crate::coalesce::PushBatchStatus::LimitReached; use crate::common::can_project; use crate::execution_plan::CardinalityEffect; use crate::filter_pushdown::{ @@ -42,7 +44,7 @@ use crate::{ DisplayFormatType, ExecutionPlan, }; -use arrow::compute::{filter_record_batch, BatchCoalescer}; +use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; @@ -87,6 +89,8 @@ pub struct FilterExec { projection: Option>, /// Target batch size for output batches batch_size: usize, + /// Number of rows to fetch + fetch: Option, } impl FilterExec { @@ -112,6 +116,7 @@ impl FilterExec { cache, projection: None, batch_size: FILTER_EXEC_DEFAULT_BATCH_SIZE, + fetch: None, }) } other => { @@ -160,6 +165,7 @@ impl FilterExec { cache, projection, batch_size: self.batch_size, + fetch: self.fetch, }) } @@ -172,6 +178,7 @@ impl FilterExec { cache: self.cache.clone(), projection: self.projection.clone(), batch_size, + fetch: self.fetch, }) } @@ -351,7 +358,14 @@ impl DisplayAs for FilterExec { } else { "".to_string() }; - write!(f, "FilterExec: {}{}", self.predicate, display_projections) + let fetch = self + .fetch + .map_or_else(|| "".to_string(), |f| format!(", fetch={f}")); + write!( + f, + "FilterExec: {}{}{}", + self.predicate, display_projections, fetch + ) } DisplayFormatType::TreeRender => { write!(f, "predicate={}", fmt_sql(self.predicate.as_ref())) @@ -393,7 +407,7 @@ impl ExecutionPlan for FilterExec { e.with_default_selectivity(selectivity) }) .and_then(|e| e.with_projection(self.projection().cloned())) - .map(|e| Arc::new(e) as _) + .map(|e| e.with_fetch(self.fetch).unwrap()) } fn execute( @@ -409,8 +423,11 @@ impl ExecutionPlan for FilterExec { input: self.input.execute(partition, context)?, metrics, projection: self.projection.clone(), - batch_coalescer: BatchCoalescer::new(self.schema(), self.batch_size) - .with_biggest_coalesce_batch_size(Some(self.batch_size / 2)), + batch_coalescer: LimitedBatchCoalescer::new( + self.schema(), + self.batch_size, + self.fetch, + ), })) } @@ -569,6 +586,7 @@ impl ExecutionPlan for FilterExec { )?, projection: None, batch_size: self.batch_size, + fetch: self.fetch, }; Some(Arc::new(new) as _) }; @@ -578,6 +596,19 @@ impl ExecutionPlan for FilterExec { updated_node, }) } + + fn with_fetch(&self, fetch: Option) -> Option> { + Some(Arc::new(Self { + predicate: Arc::clone(&self.predicate), + input: Arc::clone(&self.input), + metrics: self.metrics.clone(), + default_selectivity: self.default_selectivity, + cache: self.cache.clone(), + projection: self.projection.clone(), + batch_size: self.batch_size, + fetch, + })) + } } impl EmbeddedProjection for FilterExec { @@ -648,7 +679,7 @@ struct FilterExecStream { /// The projection indices of the columns in the input schema projection: Option>, /// Batch coalescer to combine small batches - batch_coalescer: BatchCoalescer, + batch_coalescer: LimitedBatchCoalescer, } /// The metrics for `FilterExec` @@ -670,6 +701,23 @@ impl FilterExecMetrics { } } +impl FilterExecStream { + fn flush_remaining_batches( + &mut self, + ) -> Poll>> { + // Flush any remaining buffered batch + match self.batch_coalescer.finish() { + Ok(()) => { + Poll::Ready(self.batch_coalescer.next_completed_batch().map(|batch| { + self.metrics.selectivity.add_part(batch.num_rows()); + Ok(batch) + })) + } + Err(e) => Poll::Ready(Some(Err(e))), + } + } +} + pub fn batch_filter( batch: &RecordBatch, predicate: &Arc, @@ -715,7 +763,7 @@ impl Stream for FilterExecStream { match ready!(self.input.poll_next_unpin(cx)) { Some(Ok(batch)) => { let timer = elapsed_compute.timer(); - self.predicate.as_ref() + let status = self.predicate.as_ref() .evaluate(&batch) .and_then(|v| v.into_array(batch.num_rows())) .and_then(|array| { @@ -729,11 +777,11 @@ impl Stream for FilterExecStream { }).and_then(|(array, batch)| { match as_boolean_array(&array) { Ok(filter_array) => { - self.metrics.selectivity.add_part(filter_array.true_count()); self.metrics.selectivity.add_total(batch.num_rows()); - - self.batch_coalescer.push_batch_with_filter(batch.clone(), filter_array)?; - Ok(()) + // TODO: support push_batch_with_filter in LimitedBatchCoalescer + let batch = filter_record_batch(&batch, filter_array)?; + let state = self.batch_coalescer.push_batch(batch)?; + Ok(state) } Err(_) => { internal_err!( @@ -742,28 +790,28 @@ impl Stream for FilterExecStream { } } })?; - timer.done(); - if self.batch_coalescer.has_completed_batch() { - poll = Poll::Ready(Some(Ok(self - .batch_coalescer - .next_completed_batch() - .expect("has_completed_batch is true")))); + if let LimitReached = status { + poll = self.flush_remaining_batches(); + break; + } + + if let Some(batch) = self.batch_coalescer.next_completed_batch() { + self.metrics.selectivity.add_part(batch.num_rows()); + poll = Poll::Ready(Some(Ok(batch))); break; } continue; } None => { // Flush any remaining buffered batch - match self.batch_coalescer.finish_buffered_batch() { + match self.batch_coalescer.finish() { Ok(()) => { - poll = Poll::Ready( - self.batch_coalescer.next_completed_batch().map(Ok), - ); + poll = self.flush_remaining_batches(); } Err(e) => { - poll = Poll::Ready(Some(Err(e.into()))); + poll = Poll::Ready(Some(Err(e))); } } break; @@ -782,7 +830,6 @@ impl Stream for FilterExecStream { self.input.size_hint() } } - impl RecordBatchStream for FilterExecStream { fn schema(&self) -> SchemaRef { Arc::clone(&self.schema) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 692204b7b9cc..f20fabcd444c 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -6022,10 +6022,9 @@ physical_plan 07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 08)--------------CoalescePartitionsExec 09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c2, c3], file_type=csv, has_header=true +10)------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query I SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c3 order by c3 limit 4; @@ -7165,13 +7164,12 @@ logical_plan 03)----Aggregate: groupBy=[[having_test.v1, having_test.v2]], aggr=[[max(having_test.v1)]] 04)------TableScan: having_test projection=[v1, v2] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] -03)----AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] +02)--AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query error diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 7aa267a4dc6d..c69e7a19e4f7 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6436,10 +6436,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6465,10 +6464,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6494,10 +6492,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6523,10 +6520,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6552,10 +6548,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IN ([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]) +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I with test AS (SELECT substr(md5(i::text)::text, 1, 32) as needle FROM generate_series(1, 100000) t(i)) @@ -6583,10 +6578,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IS NOT NULL OR NULL -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] +06)----------FilterExec: substr(md5(CAST(value@0 AS Utf8View)), 1, 32) IS NOT NULL OR NULL +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # any operator query ? diff --git a/datafusion/sqllogictest/test_files/async_udf.slt b/datafusion/sqllogictest/test_files/async_udf.slt index c61d02cfecfd..9bca72a2fdfb 100644 --- a/datafusion/sqllogictest/test_files/async_udf.slt +++ b/datafusion/sqllogictest/test_files/async_udf.slt @@ -80,12 +80,11 @@ logical_plan 01)Filter: async_abs(data.x) < Int32(5) 02)--TableScan: data projection=[x] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: __async_fn_0@1 < 5, projection=[x@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------AsyncFuncExec: async_expr=[async_expr(name=__async_fn_0, expr=async_abs(x@0))] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: __async_fn_0@1 < 5, projection=[x@0] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----AsyncFuncExec: async_expr=[async_expr(name=__async_fn_0, expr=async_abs(x@0))] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] # Async udf can be used in projection query I rowsort diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 19d9ddecc9ff..38676baddd99 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -64,13 +64,12 @@ logical_plan 04)------TableScan: t1 projection=[a] physical_plan 01)ProjectionExec: expr=[a@0 as a, count(Int64(1))@1 as cnt] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: count(Int64(1))@1 > 0 -04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: count(Int64(1))@1 > 0 +03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 1; diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index e7ca7a5ae1d8..03900a608e6a 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -117,10 +117,9 @@ physical_plan 03)----PlaceholderRowExec 04)--CoalescePartitionsExec 05)----ProjectionExec: expr=[id@0 + 1 as id] -06)------CoalesceBatchesExec: target_batch_size=8192 -07)--------FilterExec: id@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------------WorkTableExec: name=nodes +06)------FilterExec: id@0 < 10 +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------WorkTableExec: name=nodes # setup statement ok @@ -162,10 +161,9 @@ physical_plan 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], file_type=csv, has_header=true 04)----CoalescePartitionsExec 05)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] -06)--------CoalesceBatchesExec: target_batch_size=2 -07)----------FilterExec: time@0 < 10 -08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------------WorkTableExec: name=balances +06)--------FilterExec: time@0 < 10 +07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------------WorkTableExec: name=balances # recursive CTE with static term derived from table works # note that this is run with batch size set to 2. This should produce multiple batches per iteration since the input @@ -734,12 +732,11 @@ physical_plan 04)--ProjectionExec: expr=[2 as val] 05)----CrossJoinExec 06)------CoalescePartitionsExec -07)--------CoalesceBatchesExec: target_batch_size=8182 -08)----------FilterExec: val@0 < 2 -09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)--------------WorkTableExec: name=recursive_cte -11)------ProjectionExec: expr=[2 as val] -12)--------PlaceholderRowExec +07)--------FilterExec: val@0 < 2 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------WorkTableExec: name=recursive_cte +10)------ProjectionExec: expr=[2 as val] +11)--------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/9794 # Non-recursive term and recursive term have different types @@ -964,10 +961,9 @@ physical_plan 03)----PlaceholderRowExec 04)--CoalescePartitionsExec 05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------FilterExec: n@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------------WorkTableExec: name=numbers +06)------FilterExec: n@0 < 10 +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------WorkTableExec: name=numbers query TT explain WITH RECURSIVE numbers AS ( @@ -990,10 +986,9 @@ physical_plan 03)----PlaceholderRowExec 04)--CoalescePartitionsExec 05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] -06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------FilterExec: n@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------------WorkTableExec: name=numbers +06)------FilterExec: n@0 < 10 +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------WorkTableExec: name=numbers # Test for issue #16998: SortExec shares DynamicFilterPhysicalExpr across multiple executions query II diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index fd9a7fb9ce44..b6098758a9e6 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -410,9 +410,8 @@ logical_plan 01)Filter: test.column2 = Dictionary(Int32, Utf8("1")) 02)--TableScan: test projection=[column1, column2] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column2@1 = 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: column2@1 = 1 +02)--DataSourceExec: partitions=1, partition_sizes=[1] # try literal = col to verify order doesn't matter # filter should not cast column2 @@ -423,9 +422,8 @@ logical_plan 01)Filter: test.column2 = Dictionary(Int32, Utf8("1")) 02)--TableScan: test projection=[column1, column2] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column2@1 = 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: column2@1 = 1 +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Now query using an integer which must be coerced into a dictionary string @@ -441,9 +439,8 @@ logical_plan 01)Filter: test.column2 = Dictionary(Int32, Utf8("1")) 02)--TableScan: test projection=[column1, column2] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column2@1 = 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: column2@1 = 1 +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Window Functions query I diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index ec3d9f746577..d7af5ff4b9c2 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -43,10 +43,9 @@ logical_plan 02)--Filter: aggregate_test_100.c2 > Int8(10) 03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: c2@1 > 10, projection=[c1@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], file_type=csv, has_header=true +01)FilterExec: c2@1 > 10, projection=[c1@0] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], file_type=csv, has_header=true # explain_csv_exec_scan_config diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 5f3c778fc961..6ac28997a990 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -166,32 +166,26 @@ explain SELECT int_col FROM table1 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: csv │ +21)└───────────────────────────┘ # Aggregate query TT @@ -411,36 +405,30 @@ WHERE string_col != 'foo' AND string_col != 'bar' AND string_col != 'a really lo ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ FilterExec │ -09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo AND │ -12)│ string_col != bar │ -13)│ AND string_col != a │ -14)│ really long string │ -15)│ constant │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ RepartitionExec │ -19)│ -------------------- │ -20)│ partition_count(in->out): │ -21)│ 1 -> 4 │ -22)│ │ -23)│ partitioning_scheme: │ -24)│ RoundRobinBatch(4) │ -25)└─────────────┬─────────────┘ -26)┌─────────────┴─────────────┐ -27)│ DataSourceExec │ -28)│ -------------------- │ -29)│ files: 1 │ -30)│ format: csv │ -31)└───────────────────────────┘ +04)│ predicate: │ +05)│ string_col != foo AND │ +06)│ string_col != bar │ +07)│ AND string_col != a │ +08)│ really long string │ +09)│ constant │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ partition_count(in->out): │ +15)│ 1 -> 4 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ DataSourceExec │ +22)│ -------------------- │ +23)│ files: 1 │ +24)│ format: csv │ +25)└───────────────────────────┘ # Check maximum line limit. query TT @@ -449,17 +437,17 @@ WHERE string_col != 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ FilterExec │ -09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != │ -12)│ aaaaaaaaaaaa │ +04)│ predicate: │ +05)│ string_col != │ +06)│ aaaaaaaaaaaa │ +07)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +08)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +09)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +10)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +11)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +12)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 13)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 14)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 15)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ @@ -480,29 +468,23 @@ physical_plan 30)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 31)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 32)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -33)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -34)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -35)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -36)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -37)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -38)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -39)│ ... │ -40)└─────────────┬─────────────┘ -41)┌─────────────┴─────────────┐ -42)│ RepartitionExec │ -43)│ -------------------- │ -44)│ partition_count(in->out): │ -45)│ 1 -> 4 │ -46)│ │ -47)│ partitioning_scheme: │ -48)│ RoundRobinBatch(4) │ -49)└─────────────┬─────────────┘ -50)┌─────────────┴─────────────┐ -51)│ DataSourceExec │ -52)│ -------------------- │ -53)│ files: 1 │ -54)│ format: csv │ -55)└───────────────────────────┘ +33)│ ... │ +34)└─────────────┬─────────────┘ +35)┌─────────────┴─────────────┐ +36)│ RepartitionExec │ +37)│ -------------------- │ +38)│ partition_count(in->out): │ +39)│ 1 -> 4 │ +40)│ │ +41)│ partitioning_scheme: │ +42)│ RoundRobinBatch(4) │ +43)└─────────────┬─────────────┘ +44)┌─────────────┴─────────────┐ +45)│ DataSourceExec │ +46)│ -------------------- │ +47)│ files: 1 │ +48)│ format: csv │ +49)└───────────────────────────┘ # Check exactly the render width. query TT @@ -511,32 +493,26 @@ WHERE string_col != 'aaaaaaaaaaaaa'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│string_col != aaaaaaaaaaaaa│ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│string_col != aaaaaaaaaaaaa│ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: csv │ +21)└───────────────────────────┘ # Check with the render witdth + 1. query TT @@ -545,34 +521,28 @@ WHERE string_col != 'aaaaaaaaaaaaaaa'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ FilterExec │ -09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != │ -12)│ aaaaaaaaaaaa │ -13)│ aaa │ -14)└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐ -16)│ RepartitionExec │ -17)│ -------------------- │ -18)│ partition_count(in->out): │ -19)│ 1 -> 4 │ -20)│ │ -21)│ partitioning_scheme: │ -22)│ RoundRobinBatch(4) │ -23)└─────────────┬─────────────┘ -24)┌─────────────┴─────────────┐ -25)│ DataSourceExec │ -26)│ -------------------- │ -27)│ files: 1 │ -28)│ format: csv │ -29)└───────────────────────────┘ +04)│ predicate: │ +05)│ string_col != │ +06)│ aaaaaaaaaaaa │ +07)│ aaa │ +08)└─────────────┬─────────────┘ +09)┌─────────────┴─────────────┐ +10)│ RepartitionExec │ +11)│ -------------------- │ +12)│ partition_count(in->out): │ +13)│ 1 -> 4 │ +14)│ │ +15)│ partitioning_scheme: │ +16)│ RoundRobinBatch(4) │ +17)└─────────────┬─────────────┘ +18)┌─────────────┴─────────────┐ +19)│ DataSourceExec │ +20)│ -------------------- │ +21)│ files: 1 │ +22)│ format: csv │ +23)└───────────────────────────┘ # Query with filter on csv query TT @@ -580,32 +550,26 @@ explain SELECT int_col FROM table1 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: csv │ +21)└───────────────────────────┘ # Query with filter on parquet @@ -614,35 +578,29 @@ explain SELECT int_col FROM table2 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: parquet │ -27)│ │ -28)│ predicate: │ -29)│ string_col != foo │ -30)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: parquet │ +21)│ │ +22)│ predicate: │ +23)│ string_col != foo │ +24)└───────────────────────────┘ # Query with filter on memory query TT @@ -650,24 +608,18 @@ explain SELECT int_col FROM table3 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ DataSourceExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ DataSourceExec │ -15)│ -------------------- │ -16)│ bytes: 520 │ -17)│ format: memory │ -18)│ rows: 1 │ -19)└───────────────────────────┘ +10)│ bytes: 520 │ +11)│ format: memory │ +12)│ rows: 1 │ +13)└───────────────────────────┘ # Query with filter on json query TT @@ -675,32 +627,26 @@ explain SELECT int_col FROM table4 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: json │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: json │ +21)└───────────────────────────┘ # Query with filter on arrow query TT @@ -708,32 +654,26 @@ explain SELECT int_col FROM table5 WHERE string_col != 'foo'; ---- physical_plan 01)┌───────────────────────────┐ -02)│ CoalesceBatchesExec │ +02)│ FilterExec │ 03)│ -------------------- │ -04)│ target_batch_size: │ -05)│ 8192 │ +04)│ predicate: │ +05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: arrow │ -27)└───────────────────────────┘ +10)│ partition_count(in->out): │ +11)│ 1 -> 4 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: arrow │ +21)└───────────────────────────┘ # Query with window agg. @@ -1483,31 +1423,25 @@ physical_plan 05)│ ASC NULLS LAST │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ CoalesceBatchesExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ target_batch_size: │ -11)│ 8192 │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ FilterExec │ -15)│ -------------------- │ -16)│ predicate: ticker = A │ -17)└─────────────┬─────────────┘ -18)┌─────────────┴─────────────┐ -19)│ RepartitionExec │ -20)│ -------------------- │ -21)│ partition_count(in->out): │ -22)│ 1 -> 4 │ -23)│ │ -24)│ partitioning_scheme: │ -25)│ RoundRobinBatch(4) │ -26)└─────────────┬─────────────┘ -27)┌─────────────┴─────────────┐ -28)│ StreamingTableExec │ -29)│ -------------------- │ -30)│ infinite: true │ -31)│ limit: None │ -32)└───────────────────────────┘ +10)│ predicate: ticker = A │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ RepartitionExec │ +14)│ -------------------- │ +15)│ partition_count(in->out): │ +16)│ 1 -> 4 │ +17)│ │ +18)│ partitioning_scheme: │ +19)│ RoundRobinBatch(4) │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ StreamingTableExec │ +23)│ -------------------- │ +24)│ infinite: true │ +25)│ limit: None │ +26)└───────────────────────────┘ # constant ticker, CAST(time AS DATE) = time, order by time @@ -1523,33 +1457,27 @@ physical_plan 04)│ time ASC NULLS LAST │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ CoalesceBatchesExec │ +07)│ FilterExec │ 08)│ -------------------- │ -09)│ target_batch_size: │ -10)│ 8192 │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ FilterExec │ -14)│ -------------------- │ -15)│ predicate: │ -16)│ ticker = A AND CAST(time │ -17)│ AS Date32) = date │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ partition_count(in->out): │ -23)│ 1 -> 4 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ -30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +09)│ predicate: │ +10)│ ticker = A AND CAST(time │ +11)│ AS Date32) = date │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ StreamingTableExec │ +24)│ -------------------- │ +25)│ infinite: true │ +26)│ limit: None │ +27)└───────────────────────────┘ # same thing but order by date query TT @@ -1564,33 +1492,27 @@ physical_plan 04)│ date ASC NULLS LAST │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ CoalesceBatchesExec │ +07)│ FilterExec │ 08)│ -------------------- │ -09)│ target_batch_size: │ -10)│ 8192 │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ FilterExec │ -14)│ -------------------- │ -15)│ predicate: │ -16)│ ticker = A AND CAST(time │ -17)│ AS Date32) = date │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ partition_count(in->out): │ -23)│ 1 -> 4 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ -30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +09)│ predicate: │ +10)│ ticker = A AND CAST(time │ +11)│ AS Date32) = date │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ StreamingTableExec │ +24)│ -------------------- │ +25)│ infinite: true │ +26)│ limit: None │ +27)└───────────────────────────┘ # same thing but order by ticker query TT @@ -1603,33 +1525,27 @@ physical_plan 02)│ CoalescePartitionsExec │ 03)└─────────────┬─────────────┘ 04)┌─────────────┴─────────────┐ -05)│ CoalesceBatchesExec │ +05)│ FilterExec │ 06)│ -------------------- │ -07)│ target_batch_size: │ -08)│ 8192 │ -09)└─────────────┬─────────────┘ -10)┌─────────────┴─────────────┐ -11)│ FilterExec │ -12)│ -------------------- │ -13)│ predicate: │ -14)│ ticker = A AND CAST(time │ -15)│ AS Date32) = date │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ RepartitionExec │ -19)│ -------------------- │ -20)│ partition_count(in->out): │ -21)│ 1 -> 4 │ -22)│ │ -23)│ partitioning_scheme: │ -24)│ RoundRobinBatch(4) │ -25)└─────────────┬─────────────┘ -26)┌─────────────┴─────────────┐ -27)│ StreamingTableExec │ -28)│ -------------------- │ -29)│ infinite: true │ -30)│ limit: None │ -31)└───────────────────────────┘ +07)│ predicate: │ +08)│ ticker = A AND CAST(time │ +09)│ AS Date32) = date │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ partition_count(in->out): │ +15)│ 1 -> 4 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ StreamingTableExec │ +22)│ -------------------- │ +23)│ infinite: true │ +24)│ limit: None │ +25)└───────────────────────────┘ # same thing but order by time, date @@ -1646,33 +1562,27 @@ physical_plan 05)│ ASC NULLS LAST │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ CoalesceBatchesExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ target_batch_size: │ -11)│ 8192 │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ FilterExec │ -15)│ -------------------- │ -16)│ predicate: │ -17)│ ticker = A AND CAST(time │ -18)│ AS Date32) = date │ -19)└─────────────┬─────────────┘ -20)┌─────────────┴─────────────┐ -21)│ RepartitionExec │ -22)│ -------------------- │ -23)│ partition_count(in->out): │ -24)│ 1 -> 4 │ -25)│ │ -26)│ partitioning_scheme: │ -27)│ RoundRobinBatch(4) │ -28)└─────────────┬─────────────┘ -29)┌─────────────┴─────────────┐ -30)│ StreamingTableExec │ -31)│ -------------------- │ -32)│ infinite: true │ -33)│ limit: None │ -34)└───────────────────────────┘ +10)│ predicate: │ +11)│ ticker = A AND CAST(time │ +12)│ AS Date32) = date │ +13)└─────────────┬─────────────┘ +14)┌─────────────┴─────────────┐ +15)│ RepartitionExec │ +16)│ -------------------- │ +17)│ partition_count(in->out): │ +18)│ 1 -> 4 │ +19)│ │ +20)│ partitioning_scheme: │ +21)│ RoundRobinBatch(4) │ +22)└─────────────┬─────────────┘ +23)┌─────────────┴─────────────┐ +24)│ StreamingTableExec │ +25)│ -------------------- │ +26)│ infinite: true │ +27)│ limit: None │ +28)└───────────────────────────┘ @@ -1693,34 +1603,26 @@ physical_plan 07)│ time ASC NULLS LAST │ 08)└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐ -10)│ CoalesceBatchesExec │ +10)│ FilterExec │ 11)│ -------------------- │ -12)│ limit: 5 │ -13)│ │ -14)│ target_batch_size: │ -15)│ 8192 │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ FilterExec │ -19)│ -------------------- │ -20)│ predicate: │ -21)│ date = 2006-01-02 │ -22)└─────────────┬─────────────┘ -23)┌─────────────┴─────────────┐ -24)│ RepartitionExec │ -25)│ -------------------- │ -26)│ partition_count(in->out): │ -27)│ 1 -> 4 │ -28)│ │ -29)│ partitioning_scheme: │ -30)│ RoundRobinBatch(4) │ -31)└─────────────┬─────────────┘ -32)┌─────────────┴─────────────┐ -33)│ StreamingTableExec │ -34)│ -------------------- │ -35)│ infinite: true │ -36)│ limit: None │ -37)└───────────────────────────┘ +12)│ predicate: │ +13)│ date = 2006-01-02 │ +14)└─────────────┬─────────────┘ +15)┌─────────────┴─────────────┐ +16)│ RepartitionExec │ +17)│ -------------------- │ +18)│ partition_count(in->out): │ +19)│ 1 -> 4 │ +20)│ │ +21)│ partitioning_scheme: │ +22)│ RoundRobinBatch(4) │ +23)└─────────────┬─────────────┘ +24)┌─────────────┴─────────────┐ +25)│ StreamingTableExec │ +26)│ -------------------- │ +27)│ infinite: true │ +28)│ limit: None │ +29)└───────────────────────────┘ @@ -1739,32 +1641,26 @@ physical_plan 05)│ time ASC NULLS LAST │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ CoalesceBatchesExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ target_batch_size: │ -11)│ 8192 │ +10)│ predicate: │ +11)│ date = 2006-01-02 │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ FilterExec │ +14)│ RepartitionExec │ 15)│ -------------------- │ -16)│ predicate: │ -17)│ date = 2006-01-02 │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ partition_count(in->out): │ -23)│ 1 -> 4 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ -30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ StreamingTableExec │ +24)│ -------------------- │ +25)│ infinite: true │ +26)│ limit: None │ +27)└───────────────────────────┘ @@ -1794,30 +1690,24 @@ physical_plan 12)│ ││ id: id + 1 │ 13)└───────────────────────────┘└─────────────┬─────────────┘ 14)-----------------------------┌─────────────┴─────────────┐ -15)-----------------------------│ CoalesceBatchesExec │ +15)-----------------------------│ FilterExec │ 16)-----------------------------│ -------------------- │ -17)-----------------------------│ target_batch_size: │ -18)-----------------------------│ 8192 │ -19)-----------------------------└─────────────┬─────────────┘ -20)-----------------------------┌─────────────┴─────────────┐ -21)-----------------------------│ FilterExec │ -22)-----------------------------│ -------------------- │ -23)-----------------------------│ predicate: id < 10 │ -24)-----------------------------└─────────────┬─────────────┘ -25)-----------------------------┌─────────────┴─────────────┐ -26)-----------------------------│ RepartitionExec │ -27)-----------------------------│ -------------------- │ -28)-----------------------------│ partition_count(in->out): │ -29)-----------------------------│ 1 -> 4 │ -30)-----------------------------│ │ -31)-----------------------------│ partitioning_scheme: │ -32)-----------------------------│ RoundRobinBatch(4) │ -33)-----------------------------└─────────────┬─────────────┘ -34)-----------------------------┌─────────────┴─────────────┐ -35)-----------------------------│ WorkTableExec │ -36)-----------------------------│ -------------------- │ -37)-----------------------------│ name: nodes │ -38)-----------------------------└───────────────────────────┘ +17)-----------------------------│ predicate: id < 10 │ +18)-----------------------------└─────────────┬─────────────┘ +19)-----------------------------┌─────────────┴─────────────┐ +20)-----------------------------│ RepartitionExec │ +21)-----------------------------│ -------------------- │ +22)-----------------------------│ partition_count(in->out): │ +23)-----------------------------│ 1 -> 4 │ +24)-----------------------------│ │ +25)-----------------------------│ partitioning_scheme: │ +26)-----------------------------│ RoundRobinBatch(4) │ +27)-----------------------------└─────────────┬─────────────┘ +28)-----------------------------┌─────────────┴─────────────┐ +29)-----------------------------│ WorkTableExec │ +30)-----------------------------│ -------------------- │ +31)-----------------------------│ name: nodes │ +32)-----------------------------└───────────────────────────┘ query TT explain COPY (VALUES (1, 'foo', 1, '2023-01-01'), (2, 'bar', 2, '2023-01-02'), (3, 'baz', 3, '2023-01-03')) @@ -1937,25 +1827,17 @@ physical_plan 38)│ skip: 6 │ 39)└─────────────┬─────────────┘ 40)┌─────────────┴─────────────┐ -41)│ CoalesceBatchesExec │ +41)│ FilterExec │ 42)│ -------------------- │ -43)│ limit: 9 │ -44)│ │ -45)│ target_batch_size: │ -46)│ 8192 │ -47)└─────────────┬─────────────┘ -48)┌─────────────┴─────────────┐ -49)│ FilterExec │ -50)│ -------------------- │ -51)│ predicate: a > 3 │ -52)└─────────────┬─────────────┘ -53)┌─────────────┴─────────────┐ -54)│ DataSourceExec │ -55)│ -------------------- │ -56)│ bytes: 160 │ -57)│ format: memory │ -58)│ rows: 1 │ -59)└───────────────────────────┘ +43)│ predicate: a > 3 │ +44)└─────────────┬─────────────┘ +45)┌─────────────┴─────────────┐ +46)│ DataSourceExec │ +47)│ -------------------- │ +48)│ bytes: 160 │ +49)│ format: memory │ +50)│ rows: 1 │ +51)└───────────────────────────┘ # clean up statement ok @@ -1985,33 +1867,25 @@ physical_plan 04)│ limit: 5 │ 05)└─────────────┬─────────────┘ 06)┌─────────────┴─────────────┐ -07)│ CoalesceBatchesExec │ +07)│ FilterExec │ 08)│ -------------------- │ -09)│ limit: 5 │ -10)│ │ -11)│ target_batch_size: │ -12)│ 8192 │ -13)└─────────────┬─────────────┘ -14)┌─────────────┴─────────────┐ -15)│ FilterExec │ -16)│ -------------------- │ -17)│ predicate: c3 > 0 │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ partition_count(in->out): │ -23)│ 1 -> 4 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ StreamingTableExec │ -30)│ -------------------- │ -31)│ infinite: true │ -32)│ limit: None │ -33)└───────────────────────────┘ +09)│ predicate: c3 > 0 │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ partition_count(in->out): │ +15)│ 1 -> 4 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ StreamingTableExec │ +22)│ -------------------- │ +23)│ infinite: true │ +24)│ limit: None │ +25)└───────────────────────────┘ # Test explain tree for PlaceholderRowExec query TT diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index a09d8ce26ddf..be79f1423859 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -38,10 +38,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST, time@2 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # constant ticker, CAST(time AS DATE) = time, order by time query TT @@ -55,10 +54,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by date query TT @@ -72,10 +70,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by ticker query TT @@ -89,10 +86,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)CoalescePartitionsExec -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by time, date query TT @@ -106,10 +102,9 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST, date@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # CAST(time AS DATE) <> date (should require a sort) # no physical plan due to sort breaking pipeline @@ -147,7 +142,6 @@ logical_plan 03)----TableScan: data projection=[date, ticker, time] physical_plan 01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: date@0 = 2006-01-02 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +02)--FilterExec: date@0 = 2006-01-02 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 87373af1472a..ea998f742d30 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -936,10 +936,9 @@ logical_plan 06)----TableScan: department projection=[dept_name] physical_plan 01)CrossJoinExec -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: name@1 = Alice OR name@1 = Bob -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)--DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: name@1 = Alice OR name@1 = Bob +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)--DataSourceExec: partitions=1, partition_sizes=[1] # expect no row for Carol query ITT @@ -984,15 +983,13 @@ logical_plan 07)------SubqueryAlias: d 08)--------TableScan: department projection=[emp_id, dept_name] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)----------DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] +05)--------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -1386,9 +1383,8 @@ physical_plan 10)--------------DataSourceExec: partitions=1, partition_sizes=[1] 11)------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] 12)--------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 -13)----------------CoalesceBatchesExec: target_batch_size=8192 -14)------------------FilterExec: y@1 = x@0 -15)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +13)----------------FilterExec: y@1 = x@0 +14)------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table pairs; diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index a1efc1317b4a..e78169734fe5 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -99,10 +99,9 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true -06)--------CoalesceBatchesExec: target_batch_size=8192 -07)----------FilterExec: d@3 = 3 -08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true +06)--------FilterExec: d@3 = 3 +07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt index 0336cfc2d331..166d1a028ecb 100644 --- a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt +++ b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt @@ -153,11 +153,10 @@ physical_plan 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t1.val + Int64(1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------FilterExec: CAST(val@1 AS Int64) + 1 IS NOT DISTINCT FROM 11 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] -11)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------FilterExec: CAST(val@1 AS Int64) + 1 IS NOT DISTINCT FROM 11 +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] +10)--------DataSourceExec: partitions=1, partition_sizes=[1] # Mixed join predicate with `IS DISTINCT FROM` and `IS NOT DISTINCT FROM` query IIII rowsort diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 62804ad76bd6..4a243258a519 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2052,14 +2052,12 @@ physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id] 02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 03)----CoalescePartitionsExec -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)----CoalesceBatchesExec: target_batch_size=2 -09)------FilterExec: t1_id@0 > 10 -10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------FilterExec: t2_int@1 > 1, projection=[t2_id@0] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----FilterExec: t1_id@0 > 10 +08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2092,14 +2090,12 @@ logical_plan physical_plan 01)NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 02)--CoalescePartitionsExec -03)----CoalesceBatchesExec: target_batch_size=2 -04)------FilterExec: t1_id@0 > 22 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)--CoalesceBatchesExec: target_batch_size=2 -08)----FilterExec: t2_id@0 > 11 -09)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)--------DataSourceExec: partitions=1, partition_sizes=[1] +03)----FilterExec: t1_id@0 > 22 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--FilterExec: t2_id@0 > 11 +07)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -4108,9 +4104,8 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=3 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] 06)----------DataSourceExec: partitions=1, partition_sizes=[0] -07)----------CoalesceBatchesExec: target_batch_size=3 -08)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] -09)--------------DataSourceExec: partitions=1, partition_sizes=[0] +07)----------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] +08)------------DataSourceExec: partitions=1, partition_sizes=[0] statement ok DROP TABLE sales_global; @@ -4459,11 +4454,10 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalescePartitionsExec -05)--------CoalesceBatchesExec: target_batch_size=3 -06)----------FilterExec: b@1 > 3, projection=[a@0] -07)------------DataSourceExec: partitions=2, partition_sizes=[1, 1] -08)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] -09)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] +05)--------FilterExec: b@1 > 3, projection=[a@0] +06)----------DataSourceExec: partitions=2, partition_sizes=[1, 1] +07)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] +08)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] query TT explain select * from test where a in (select a from test where b > 3) order by c desc nulls last; @@ -4481,11 +4475,10 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalescePartitionsExec -05)--------CoalesceBatchesExec: target_batch_size=3 -06)----------FilterExec: b@1 > 3, projection=[a@0] -07)------------DataSourceExec: partitions=2, partition_sizes=[1, 1] -08)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] -09)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] +05)--------FilterExec: b@1 > 3, projection=[a@0] +06)----------DataSourceExec: partitions=2, partition_sizes=[1, 1] +07)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] +08)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] query III select * from test where a in (select a from test where b > 3) order by c desc nulls first; @@ -5085,12 +5078,11 @@ WHERE k2 > 0 ) ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--FilterExec: k2@0 > 0 OR mark@2, projection=[k2@0, v2@1] -03)----CoalesceBatchesExec: target_batch_size=3 -04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(k2@0, k1@0)] -05)--------DataSourceExec: partitions=1, partition_sizes=[0] -06)--------DataSourceExec: partitions=1, partition_sizes=[10000] +01)FilterExec: k2@0 > 0 OR mark@2, projection=[k2@0, v2@1] +02)--CoalesceBatchesExec: target_batch_size=3 +03)----HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(k2@0, k1@0)] +04)------DataSourceExec: partitions=1, partition_sizes=[0] +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * @@ -5136,9 +5128,8 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----CoalesceBatchesExec: target_batch_size=3 -05)------FilterExec: k1@0 < 0 -06)--------DataSourceExec: partitions=1, partition_sizes=[10000] +04)----FilterExec: k1@0 < 0 +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * @@ -5205,12 +5196,10 @@ physical_plan 01)SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--PiecewiseMergeJoin: operator=Gt, join_type=Inner, on=(t1_id > t2_id) 03)----SortExec: expr=[t1_id@0 ASC], preserve_partitioning=[false] -04)------CoalesceBatchesExec: target_batch_size=3 -05)--------FilterExec: t1_id@0 > 10 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----CoalesceBatchesExec: target_batch_size=3 -08)------FilterExec: t2_int@1 > 1, projection=[t2_id@0] -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------FilterExec: t1_id@0 > 10 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)----FilterExec: t2_int@1 > 1, projection=[t2_id@0] +07)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok DROP TABLE t1; diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 25b741b025a7..48781e46f11c 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -377,9 +377,8 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------ProjectionExec: expr=[] 07)------------GlobalLimitExec: skip=6, fetch=3 -08)--------------CoalesceBatchesExec: target_batch_size=8192, fetch=9 -09)----------------FilterExec: a@0 > 3 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------FilterExec: a@0 > 3, fetch=9 +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query I SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 45f8c5d25fbe..75acd68432c1 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -113,9 +113,8 @@ logical_plan 01)Filter: table_with_map.int_field > Int64(0) 02)--TableScan: table_with_map projection=[int_field, map_field] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: int_field@0 > 0 -03)----DataSourceExec: partitions=1, partition_sizes=[0] +01)FilterExec: int_field@0 > 0 +02)--DataSourceExec: partitions=1, partition_sizes=[0] statement ok drop table table_with_map; diff --git a/datafusion/sqllogictest/test_files/operator.slt b/datafusion/sqllogictest/test_files/operator.slt index 6f3c40188172..e50fa721c885 100644 --- a/datafusion/sqllogictest/test_files/operator.slt +++ b/datafusion/sqllogictest/test_files/operator.slt @@ -287,9 +287,8 @@ EXPLAIN SELECT * FROM numeric_types WHERE int64 < 5 AND uint64 < 5 AND float64 < 5 AND decimal < 5; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: int64@3 < 5 AND uint64@7 < 5 AND float64@9 < 5 AND decimal@10 < Some(500),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: int64@3 < 5 AND uint64@7 < 5 AND float64@9 < 5 AND decimal@10 < Some(500),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## < negative integer (expect no casts) query TT @@ -297,9 +296,8 @@ EXPLAIN SELECT * FROM numeric_types WHERE int64 < -5 AND uint64 < -5 AND float64 < -5 AND decimal < -5; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: int64@3 < -5 AND CAST(uint64@7 AS Decimal128(20, 0)) < Some(-5),20,0 AND float64@9 < -5 AND decimal@10 < Some(-500),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: int64@3 < -5 AND CAST(uint64@7 AS Decimal128(20, 0)) < Some(-5),20,0 AND float64@9 < -5 AND decimal@10 < Some(-500),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## < decimal (expect casts for integers to float) query TT @@ -307,9 +305,8 @@ EXPLAIN SELECT * FROM numeric_types WHERE int64 < 5.1 AND uint64 < 5.1 AND float64 < 5.1 AND decimal < 5.1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: CAST(int64@3 AS Float64) < 5.1 AND CAST(uint64@7 AS Float64) < 5.1 AND float64@9 < 5.1 AND decimal@10 < Some(510),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: CAST(int64@3 AS Float64) < 5.1 AND CAST(uint64@7 AS Float64) < 5.1 AND float64@9 < 5.1 AND decimal@10 < Some(510),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## < negative decimal (expect casts for integers to float) query TT @@ -317,9 +314,8 @@ EXPLAIN SELECT * FROM numeric_types WHERE int64 < -5.1 AND uint64 < -5.1 AND float64 < -5.1 AND decimal < -5.1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: CAST(int64@3 AS Float64) < -5.1 AND CAST(uint64@7 AS Float64) < -5.1 AND float64@9 < -5.1 AND decimal@10 < Some(-510),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: CAST(int64@3 AS Float64) < -5.1 AND CAST(uint64@7 AS Float64) < -5.1 AND float64@9 < -5.1 AND decimal@10 < Some(-510),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ############### Equality ############### @@ -330,9 +326,8 @@ EXPLAIN SELECT * FROM numeric_types WHERE int64 = 5 AND uint64 = 5 AND float64 = 5 AND decimal = 5; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: int64@3 = 5 AND uint64@7 = 5 AND float64@9 = 5 AND decimal@10 = Some(500),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: int64@3 = 5 AND uint64@7 = 5 AND float64@9 = 5 AND decimal@10 = Some(500),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## = negative integer (expect no casts) query TT @@ -340,9 +335,8 @@ EXPLAIN SELECT * FROM numeric_types WHERE int64 = -5 AND uint64 = -5 AND float64 = -5 AND decimal = -5; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: int64@3 = -5 AND CAST(uint64@7 AS Decimal128(20, 0)) = Some(-5),20,0 AND float64@9 = -5 AND decimal@10 = Some(-500),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: int64@3 = -5 AND CAST(uint64@7 AS Decimal128(20, 0)) = Some(-5),20,0 AND float64@9 = -5 AND decimal@10 = Some(-500),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## = decimal (expect casts for integers to float) query TT @@ -350,9 +344,8 @@ EXPLAIN SELECT * FROM numeric_types WHERE int64 = 5.1 AND uint64 = 5.1 AND float64 = 5.1 AND decimal = 5.1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: CAST(int64@3 AS Float64) = 5.1 AND CAST(uint64@7 AS Float64) = 5.1 AND float64@9 = 5.1 AND decimal@10 = Some(510),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: CAST(int64@3 AS Float64) = 5.1 AND CAST(uint64@7 AS Float64) = 5.1 AND float64@9 = 5.1 AND decimal@10 = Some(510),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## = negative decimal (expect casts for integers to float) query TT @@ -360,9 +353,8 @@ EXPLAIN SELECT * FROM numeric_types WHERE int64 = -5.1 AND uint64 = -5.1 AND float64 = -5.1 AND decimal = -5.1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: CAST(int64@3 AS Float64) = -5.1 AND CAST(uint64@7 AS Float64) = -5.1 AND float64@9 = -5.1 AND decimal@10 = Some(-510),5,2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: CAST(int64@3 AS Float64) = -5.1 AND CAST(uint64@7 AS Float64) = -5.1 AND float64@9 = -5.1 AND decimal@10 = Some(-510),5,2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/options.slt b/datafusion/sqllogictest/test_files/options.slt index 71ff12e8cc50..0d1583dbc008 100644 --- a/datafusion/sqllogictest/test_files/options.slt +++ b/datafusion/sqllogictest/test_files/options.slt @@ -23,7 +23,6 @@ statement ok create table a(c0 int) as values (1), (2); -# Expect coalesce and default batch size query TT explain SELECT * FROM a WHERE c0 < 1; ---- @@ -31,9 +30,8 @@ logical_plan 01)Filter: a.c0 < Int32(1) 02)--TableScan: a projection=[c0] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: c0@0 < 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: c0@0 < 1 +02)--DataSourceExec: partitions=1, partition_sizes=[1] ## # test_disable_coalesce @@ -72,9 +70,8 @@ logical_plan 01)Filter: a.c0 < Int32(1) 02)--TableScan: a projection=[c0] physical_plan -01)CoalesceBatchesExec: target_batch_size=1234 -02)--FilterExec: c0@0 < 1 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: c0@0 < 1 +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index c21f3129d4ee..c786f7bdc77c 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -454,10 +454,9 @@ logical_plan 01)Filter: CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%") 02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +01)FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% statement ok @@ -502,10 +501,9 @@ logical_plan 01)Filter: binary_as_string_option.binary_col LIKE Utf8View("%a%") AND binary_as_string_option.largebinary_col LIKE Utf8View("%a%") AND binary_as_string_option.binaryview_col LIKE Utf8View("%a%") 02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8View("%a%"), binary_as_string_option.largebinary_col LIKE Utf8View("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -553,10 +551,9 @@ logical_plan 01)Filter: binary_as_string_both.binary_col LIKE Utf8View("%a%") AND binary_as_string_both.largebinary_col LIKE Utf8View("%a%") AND binary_as_string_both.binaryview_col LIKE Utf8View("%a%") 02)--TableScan: binary_as_string_both projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_both.binary_col LIKE Utf8View("%a%"), binary_as_string_both.largebinary_col LIKE Utf8View("%a%"), binary_as_string_both.binaryview_col LIKE Utf8View("%a%")] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -668,10 +665,9 @@ logical_plan 01)Filter: foo.column1 LIKE Utf8View("f%") 02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column1@0 LIKE f% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] +01)FilterExec: column1@0 LIKE f% +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] statement ok drop table foo diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 0166cd2572ce..8bb79d576990 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -95,10 +95,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: b@1 > 2, projection=[a@0] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -06)----------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----FilterExec: b@1 > 2, projection=[a@0] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -133,11 +132,9 @@ logical_plan 04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--ProjectionExec: expr=[a@0 as a] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: b@1 = 2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -06)----------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--FilterExec: b@1 = 2, projection=[a@0] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -266,10 +263,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: b@1 > 2, projection=[a@0] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -06)----------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----FilterExec: b@1 > 2, projection=[a@0] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -304,11 +300,9 @@ logical_plan 04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--ProjectionExec: expr=[a@0 as a] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: b@1 = 2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -06)----------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--FilterExec: b@1 = 2, projection=[a@0] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -344,10 +338,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: b@1 > 2, projection=[a@0] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -06)----------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----FilterExec: b@1 > 2, projection=[a@0] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t_pushdown where b = 2 ORDER BY b; @@ -416,9 +409,8 @@ logical_plan 02)--Filter: CAST(t_pushdown.b AS Float64) > random() 03)----TableScan: t_pushdown projection=[a, b] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: CAST(b@1 AS Float64) > random(), projection=[a@0] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet +01)FilterExec: CAST(b@1 AS Float64) > random(), projection=[a@0] +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet ## cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/parquet_statistics.slt b/datafusion/sqllogictest/test_files/parquet_statistics.slt index c04235ef4ee6..14cf4b280247 100644 --- a/datafusion/sqllogictest/test_files/parquet_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_statistics.slt @@ -59,10 +59,9 @@ query TT EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] -02)--FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] # cleanup statement ok @@ -85,10 +84,9 @@ query TT EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] -02)--FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(31), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0))]] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(121), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0))]] # cleanup statement ok @@ -112,10 +110,9 @@ query TT EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(1)))]] -02)--FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +01)FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] # cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index c10e67a22535..769a42108b14 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -672,16 +672,14 @@ physical_plan 02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 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 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 l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------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 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=true -09)----CoalesceBatchesExec: target_batch_size=8192 -10)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -11)--------CoalesceBatchesExec: target_batch_size=8192 -12)----------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 -13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true +05)--------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 +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=true +08)----CoalesceBatchesExec: target_batch_size=8192 +09)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +10)--------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 +11)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true ######## # TPCH Q19 - Pull predicates to inner join (simplified) @@ -771,10 +769,9 @@ physical_plan 10)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true 11)------------CoalesceBatchesExec: target_batch_size=8192 12)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -13)----------------CoalesceBatchesExec: target_batch_size=8192 -14)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true +13)----------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true # Simplification of a binary operator with a NULL value @@ -804,9 +801,8 @@ logical_plan 01)Filter: t.x < Int32(5) AND Boolean(NULL) 02)--TableScan: t projection=[x] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: x@0 < 5 AND NULL -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: x@0 < 5 AND NULL +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN FORMAT INDENT SELECT * FROM t WHERE x < 5 OR (10 * NULL < x); @@ -815,9 +811,8 @@ logical_plan 01)Filter: t.x < Int32(5) OR Boolean(NULL) 02)--TableScan: t projection=[x] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: x@0 < 5 OR NULL -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: x@0 < 5 OR NULL +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; @@ -846,9 +841,8 @@ logical_plan 01)Filter: t.x = Int32(5) 02)--TableScan: t projection=[x] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: x@0 = 5 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: x@0 = 5 +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select x from t where x NOT IN (1,2,3,4,5) AND x IN (1,2,3); diff --git a/datafusion/sqllogictest/test_files/projection.slt b/datafusion/sqllogictest/test_files/projection.slt index 9f840e7bdc2f..5a4411233424 100644 --- a/datafusion/sqllogictest/test_files/projection.slt +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -276,7 +276,6 @@ logical_plan 03)----TableScan: t1 projection=[a], partial_filters=[t1.a > Int64(1)] physical_plan 01)ProjectionExec: expr=[] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@0 > 1 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] +02)--FilterExec: a@0 > 1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index 47095d92d937..cedc3f9f76a8 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -40,9 +40,8 @@ physical_plan 02)--UnnestExec 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------ProjectionExec: expr=[column2@0 as __unnest_placeholder(v.column2)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: column1@0 = 2, projection=[column2@1] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: column1@0 = 2, projection=[column2@1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query I select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; @@ -56,12 +55,11 @@ explain select uc2 from (select unnest(column2) as uc2, column1 from v) where uc ---- physical_plan 01)ProjectionExec: expr=[__unnest_placeholder(v.column2,depth=1)@0 as uc2] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------UnnestExec -06)----------ProjectionExec: expr=[column2@0 as __unnest_placeholder(v.column2)] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------UnnestExec +05)--------ProjectionExec: expr=[column2@0 as __unnest_placeholder(v.column2)] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query II select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 AND column1 = 2; @@ -75,14 +73,12 @@ explain select uc2, column1 from (select unnest(column2) as uc2, column1 from v ---- physical_plan 01)ProjectionExec: expr=[__unnest_placeholder(v.column2,depth=1)@0 as uc2, column1@1 as column1] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 -04)------UnnestExec -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------FilterExec: column1@0 = 2 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 +03)----UnnestExec +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] +06)----------FilterExec: column1@0 = 2 +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query II select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 OR column1 = 2; @@ -97,12 +93,11 @@ explain select uc2, column1 from (select unnest(column2) as uc2, column1 from v ---- physical_plan 01)ProjectionExec: expr=[__unnest_placeholder(v.column2,depth=1)@0 as uc2, column1@1 as column1] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 OR column1@1 = 2 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------UnnestExec -06)----------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: __unnest_placeholder(v.column2,depth=1)@0 > 3 OR column1@1 = 2 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------UnnestExec +05)--------ProjectionExec: expr=[column2@1 as __unnest_placeholder(v.column2), column1@0 as column1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table v; @@ -121,12 +116,11 @@ explain select * from (select column1, unnest(column2) as o from d) where o['a'] ---- physical_plan 01)ProjectionExec: expr=[column1@0 as column1, __unnest_placeholder(d.column2,depth=1)@1 as o] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: get_field(__unnest_placeholder(d.column2,depth=1)@1, a) = 1 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------UnnestExec -06)----------ProjectionExec: expr=[column1@0 as column1, column2@1 as __unnest_placeholder(d.column2)] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: get_field(__unnest_placeholder(d.column2,depth=1)@1, a) = 1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------UnnestExec +05)--------ProjectionExec: expr=[column1@0 as column1, column2@1 as __unnest_placeholder(d.column2)] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table d; @@ -143,12 +137,11 @@ query TT explain select * from (select unnest(column1) from d) where "__unnest_placeholder(d.column1).b" > 5; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: __unnest_placeholder(d.column1).b@1 > 5 -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------UnnestExec -05)--------ProjectionExec: expr=[column1@0 as __unnest_placeholder(d.column1)] -06)----------DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: __unnest_placeholder(d.column1).b@1 > 5 +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----UnnestExec +04)------ProjectionExec: expr=[column1@0 as __unnest_placeholder(d.column1)] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table d; diff --git a/datafusion/sqllogictest/test_files/pwmj.slt b/datafusion/sqllogictest/test_files/pwmj.slt index eafa4d0ba394..295eb94318ee 100644 --- a/datafusion/sqllogictest/test_files/pwmj.slt +++ b/datafusion/sqllogictest/test_files/pwmj.slt @@ -87,13 +87,11 @@ physical_plan 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----PiecewiseMergeJoin: operator=Gt, join_type=Inner, on=(t1_id > t2_id) 04)------SortExec: expr=[t1_id@0 ASC], preserve_partitioning=[false] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: t1_id@0 > 10 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------CoalesceBatchesExec: target_batch_size=8192 -10)----------FilterExec: t2_int@1 > 1, projection=[t2_id@0] -11)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: t1_id@0 > 10 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] +09)----------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.t1_id, t2.t2_id @@ -134,13 +132,11 @@ physical_plan 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----PiecewiseMergeJoin: operator=GtEq, join_type=Inner, on=(t1_id >= t2_id) 04)------SortExec: expr=[t1_id@0 ASC], preserve_partitioning=[false] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: t1_id@0 >= 22 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------CoalesceBatchesExec: target_batch_size=8192 -10)----------FilterExec: t2_int@1 = 3, projection=[t2_id@0] -11)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: t1_id@0 >= 22 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------FilterExec: t2_int@1 = 3, projection=[t2_id@0] +09)----------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.t1_id, t2.t2_id @@ -183,9 +179,8 @@ physical_plan 04)------SortExec: expr=[t1_id@0 DESC], preserve_partitioning=[false] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)--------CoalesceBatchesExec: target_batch_size=8192 -08)----------FilterExec: t2_int@1 >= 3, projection=[t2_id@0] -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------FilterExec: t2_int@1 >= 3, projection=[t2_id@0] +08)----------DataSourceExec: partitions=1, partition_sizes=[1] query II @@ -231,9 +226,8 @@ physical_plan 04)------SortExec: expr=[CAST(t1_id@0 AS Int64) DESC], preserve_partitioning=[false] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)--------CoalesceBatchesExec: target_batch_size=8192 -08)----------FilterExec: t2_int@1 >= 3, projection=[t2_id@0] -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------FilterExec: t2_int@1 >= 3, projection=[t2_id@0] +08)----------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.t1_id, t2.t2_id @@ -275,13 +269,11 @@ physical_plan 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----PiecewiseMergeJoin: operator=LtEq, join_type=Inner, on=(t1_id <= t2_id) 04)------SortExec: expr=[t1_id@0 DESC], preserve_partitioning=[false] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: t1_id@0 = 11 OR t1_id@0 = 44 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------CoalesceBatchesExec: target_batch_size=8192 -10)----------FilterExec: t2_name@1 != y, projection=[t2_id@0] -11)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: t1_id@0 = 11 OR t1_id@0 = 44 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------FilterExec: t2_name@1 != y, projection=[t2_id@0] +09)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok CREATE TABLE null_join_t1 (id INT); diff --git a/datafusion/sqllogictest/test_files/qualify.slt b/datafusion/sqllogictest/test_files/qualify.slt index 524f6baad2be..d4de5f9a9a61 100644 --- a/datafusion/sqllogictest/test_files/qualify.slt +++ b/datafusion/sqllogictest/test_files/qualify.slt @@ -273,10 +273,9 @@ logical_plan 05)--------TableScan: users projection=[id, name] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 > 1, projection=[id@0, name@1] -04)------WindowAggExec: wdw=[count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64 }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 > 1, projection=[id@0, name@1] +03)----WindowAggExec: wdw=[count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64 }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +04)------DataSourceExec: partitions=1, partition_sizes=[1] # plan row_number() query TT @@ -290,12 +289,11 @@ logical_plan 05)--------TableScan: users projection=[dept] physical_plan 01)ProjectionExec: expr=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@0 as rk] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@0 > 1 -04)------ProjectionExec: expr=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] -05)--------BoundedWindowAggExec: wdw=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -06)----------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[false] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@0 > 1 +03)----ProjectionExec: expr=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +04)------BoundedWindowAggExec: wdw=[row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +05)--------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[false] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] # plan with window function and group by query TT @@ -318,23 +316,20 @@ logical_plan 09)----------------TableScan: users projection=[salary, dept] physical_plan 01)ProjectionExec: expr=[dept@0 as dept, avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as r] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 > Some(60000000000),14,6 -04)------ProjectionExec: expr=[dept@0 as dept, avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] -05)--------WindowAggExec: wdw=[avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(14, 6), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -06)----------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[true] -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=4 -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------FilterExec: sum(users.salary)@2 > Some(2000000),20,2, projection=[dept@0, salary@1] -11)--------------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept, salary@1 as salary], aggr=[sum(users.salary)] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([dept@0, salary@1], 4), input_partitions=4 -14)--------------------------AggregateExec: mode=Partial, gby=[dept@1 as dept, salary@0 as salary], aggr=[sum(users.salary)] -15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------FilterExec: salary@0 > Some(500000),10,2 -18)----------------------------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--FilterExec: avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 > Some(60000000000),14,6 +03)----ProjectionExec: expr=[dept@0 as dept, avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +04)------WindowAggExec: wdw=[avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Decimal128(14, 6), nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +05)--------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[true] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=4 +08)--------------FilterExec: sum(users.salary)@2 > Some(2000000),20,2, projection=[dept@0, salary@1] +09)----------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept, salary@1 as salary], aggr=[sum(users.salary)] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([dept@0, salary@1], 4), input_partitions=4 +12)----------------------AggregateExec: mode=Partial, gby=[dept@1 as dept, salary@0 as salary], aggr=[sum(users.salary)] +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------------------FilterExec: salary@0 > Some(500000),10,2 +15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] # plan with aggregate function query TT @@ -355,17 +350,16 @@ physical_plan 01)SortPreservingMergeExec: [dept@0 ASC NULLS LAST] 02)--SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[dept@0 as dept, sum(users.salary)@1 as s] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------FilterExec: rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 = 1, projection=[dept@0, sum(users.salary)@1] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------BoundedWindowAggExec: wdw=[rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -08)--------------SortPreservingMergeExec: [sum(users.salary)@1 DESC] -09)----------------SortExec: expr=[sum(users.salary)@1 DESC], preserve_partitioning=[true] -10)------------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept], aggr=[sum(users.salary)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=1 -13)------------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)] -14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------FilterExec: rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 = 1, projection=[dept@0, sum(users.salary)@1] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------BoundedWindowAggExec: wdw=[rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +07)------------SortPreservingMergeExec: [sum(users.salary)@1 DESC] +08)--------------SortExec: expr=[sum(users.salary)@1 DESC], preserve_partitioning=[true] +09)----------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept], aggr=[sum(users.salary)] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=1 +12)----------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)] +13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] # Clean up statement ok diff --git a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt index dd42511eade9..6f2d5a873c1b 100644 --- a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt +++ b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt @@ -251,9 +251,8 @@ logical_plan 01)Filter: dict_table.column1 LIKE Utf8("%oo%") 02)--TableScan: dict_table projection=[column1] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column1@0 LIKE %oo% -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: column1@0 LIKE %oo% +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Ensure casting / coercion works for all operators # (there should be no casts to Utf8) diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index a3b6b380c57f..f6061b61a249 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -121,10 +121,9 @@ logical_plan 03)----TableScan: sink_table projection=[c1, c2, c3] physical_plan 01)CoalescePartitionsExec: fetch=5 -02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 -03)----FilterExec: c3@2 > 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -05)--------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true +02)--FilterExec: c3@2 > 0, fetch=5 +03)----RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 +04)------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true # Start repratition on empty column test. # See https://github.com/apache/datafusion/issues/12057 diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 41718b3aebc2..06ea22761d92 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -59,9 +59,8 @@ logical_plan 01)Filter: parquet_table.column1 != Int32(42) 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column1@0 != 42 -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +01)FilterExec: column1@0 != 42 +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok @@ -75,9 +74,8 @@ logical_plan 01)Filter: parquet_table.column1 != Int32(42) 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column1@0 != 42 -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +01)FilterExec: column1@0 != 42 +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -100,9 +98,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: column1@0 != 42 -05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..266], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:266..526, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:272..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +03)----FilterExec: column1@0 != 42 +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..266], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:266..526, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:272..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered @@ -136,9 +133,8 @@ logical_plan 03)----TableScan: parquet_table_with_order projection=[column1], partial_filters=[parquet_table_with_order.column1 != Int32(42)] physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: column1@0 != 42 -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..263], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..268], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:268..537], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:263..526]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +02)--FilterExec: column1@0 != 42 +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..263], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..268], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:268..537], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:263..526]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # Cleanup statement ok @@ -183,9 +179,8 @@ logical_plan 01)Filter: csv_table.column1 != Int32(42) 02)--TableScan: csv_table projection=[column1], partial_filters=[csv_table.column1 != Int32(42)] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column1@0 != 42 -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:5..10], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:10..15], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:15..18]]}, projection=[column1], file_type=csv, has_header=true +01)FilterExec: column1@0 != 42 +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:5..10], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:10..15], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:15..18]]}, projection=[column1], file_type=csv, has_header=true # Cleanup statement ok @@ -226,9 +221,8 @@ logical_plan 01)Filter: json_table.column1 != Int32(42) 02)--TableScan: json_table projection=[column1], partial_filters=[json_table.column1 != Int32(42)] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column1@0 != 42 -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:0..18], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:18..36], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:36..54], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:54..70]]}, projection=[column1], file_type=json +01)FilterExec: column1@0 != 42 +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:0..18], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:18..36], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:36..54], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:54..70]]}, projection=[column1], file_type=json # Cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 5c684eb83d1a..21878d36db02 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1440,10 +1440,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +02)--FilterExec: a@1 = 0 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1461,10 +1460,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +02)--FilterExec: a@1 = 0 AND b@2 = 0 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1482,10 +1480,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +02)--FilterExec: a@1 = 0 AND b@2 = 0 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1503,10 +1500,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +02)--FilterExec: a@1 = 0 AND b@2 = 0 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is when filter contains or @@ -1525,10 +1521,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] 02)--SortExec: expr=[c@3 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: a@1 = 0 OR b@2 = 0 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +03)----FilterExec: a@1 = 0 OR b@2 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # When ordering lost during projection, we shouldn't keep the SortExec. # in the final physical plan. @@ -1573,10 +1568,9 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) = annotated_data_finite2.a] physical_plan 01)SortPreservingMergeExec: [CAST(round(CAST(b@2 AS Float64)) AS Int32) ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +02)--FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true statement ok diff --git a/datafusion/sqllogictest/test_files/simplify_expr.slt b/datafusion/sqllogictest/test_files/simplify_expr.slt index 2387385369cb..d8c25ab25e8e 100644 --- a/datafusion/sqllogictest/test_files/simplify_expr.slt +++ b/datafusion/sqllogictest/test_files/simplify_expr.slt @@ -26,9 +26,8 @@ logical_plan 01)Filter: t.a = Int32(3) 02)--TableScan: t projection=[a] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: a@0 = 3 -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: a@0 = 3 +02)--DataSourceExec: partitions=1, partition_sizes=[1] # test regex exprs query TT @@ -38,9 +37,8 @@ logical_plan 01)Filter: t.b ~ Utf8View(".*") 02)--TableScan: t projection=[b] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: b@0 ~ .* -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: b@0 ~ .* +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select b from t where b !~ '.*' @@ -49,9 +47,8 @@ logical_plan 01)Filter: t.b !~ Utf8View(".*") 02)--TableScan: t projection=[b] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: b@0 !~ .* -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: b@0 !~ .* +02)--DataSourceExec: partitions=1, partition_sizes=[1] query T select b from t where b ~ '.*' @@ -70,9 +67,8 @@ logical_plan 01)Filter: t.a IS NOT NULL OR Boolean(NULL) 02)--TableScan: t projection=[a, b] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: a@0 IS NOT NULL OR NULL -03)----DataSourceExec: partitions=1, partition_sizes=[1] +01)FilterExec: a@0 IS NOT NULL OR NULL +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; @@ -117,4 +113,3 @@ logical_plan physical_plan 01)ProjectionExec: expr=[[{x:100}] as a] 02)--PlaceholderRowExec - diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index dec935749535..27325d4e5e84 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -311,16 +311,15 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -06)----------CoalesceBatchesExec: target_batch_size=2 -07)------------FilterExec: sum(t2.t2_int)@1 < 3 -08)--------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -09)----------------CoalesceBatchesExec: target_batch_size=2 -10)------------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -11)--------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------DataSourceExec: partitions=1, partition_sizes=[2] -14)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)--------DataSourceExec: partitions=1, partition_sizes=[2] +06)----------FilterExec: sum(t2.t2_int)@1 < 3 +07)------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +08)--------------CoalesceBatchesExec: target_batch_size=2 +09)----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +10)------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------DataSourceExec: partitions=1, partition_sizes=[2] +13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------DataSourceExec: partitions=1, partition_sizes=[2] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id having sum(t2_int) < 3) as t2_sum from t1 @@ -1189,13 +1188,12 @@ logical_plan 05)------SubqueryAlias: __correlated_sq_1 06)--------TableScan: t2 projection=[t2_id] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(t2_id@0, t1_id@0)] -05)--------DataSourceExec: partitions=1, partition_sizes=[2] -06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[2] +01)FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(t2_id@0, t1_id@0)] +04)------DataSourceExec: partitions=1, partition_sizes=[2] +05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[2] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index 4a6ad5eddfb7..6d76be67c206 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -55,6 +55,5 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 07)------------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))] 08)--------------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] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------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] -11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=csv, has_header=false +09)----------------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] +10)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 4cfd69bbc24f..f848a330b276 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -91,14 +91,12 @@ physical_plan 20)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=csv, has_header=false 21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 22)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -23)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -25)------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false -26)--------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -28)------------------------------CoalesceBatchesExec: target_batch_size=8192 -29)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -30)----------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=csv, has_header=false -31)------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -33)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +23)--------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +24)----------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false +25)--------------------------CoalesceBatchesExec: target_batch_size=8192 +26)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +27)------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +28)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=csv, has_header=false +29)------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +31)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index 52bbd7b63afb..6d3831cfaeaf 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -96,29 +96,27 @@ physical_plan 21)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false 22)----------------------CoalesceBatchesExec: target_batch_size=8192 23)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -24)--------------------------CoalesceBatchesExec: target_batch_size=8192 -25)----------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -26)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -28)------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)] -29)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -30)----------CoalescePartitionsExec -31)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -32)--------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -34)------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -36)----------------------CoalesceBatchesExec: target_batch_size=8192 -37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -38)--------------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -40)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false -41)--------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -43)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -44)------------------CoalesceBatchesExec: target_batch_size=8192 -45)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -46)----------------------CoalesceBatchesExec: target_batch_size=8192 -47)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -48)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -49)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +24)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +25)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +26)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +27)------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)] +28)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +29)----------CoalescePartitionsExec +30)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +31)--------------CoalesceBatchesExec: target_batch_size=8192 +32)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +33)------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +35)----------------------CoalesceBatchesExec: target_batch_size=8192 +36)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +37)--------------------------CoalesceBatchesExec: target_batch_size=8192 +38)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +39)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false +40)--------------------------CoalesceBatchesExec: target_batch_size=8192 +41)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +42)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +43)------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +45)----------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +46)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +47)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index f7344daed8c7..c2bc4d8511ef 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -70,9 +70,8 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------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] -14)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false -15)------------------CoalesceBatchesExec: target_batch_size=8192 -16)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -17)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false +12)----------------------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] +13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false +14)------------------CoalesceBatchesExec: target_batch_size=8192 +15)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +16)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 17d827cebb82..65531be833fd 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -69,6 +69,5 @@ physical_plan 14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false 15)----------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -17)--------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -19)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], file_type=csv, has_header=false +17)--------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +18)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index 71dea1a5e12a..32e05fa583d6 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -50,9 +50,8 @@ physical_plan 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] 08)--------------CoalesceBatchesExec: target_batch_size=8192 09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] -12)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false -13)--------------CoalesceBatchesExec: target_batch_size=8192 -14)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 -15)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false +10)------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] +11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +12)--------------CoalesceBatchesExec: target_batch_size=8192 +13)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 +14)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index a3284b484122..d7275347bd1e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -83,19 +83,17 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 12)----------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -15)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false -16)--------CoalesceBatchesExec: target_batch_size=8192 -17)----------HashJoinExec: mode=Partitioned, 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] -18)------------CoalesceBatchesExec: target_batch_size=8192 -19)--------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -20)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false -21)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -22)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -23)----------------CoalesceBatchesExec: target_batch_size=8192 -24)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -25)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -26)----------------------CoalesceBatchesExec: target_batch_size=8192 -27)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -28)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +13)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +14)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +15)--------CoalesceBatchesExec: target_batch_size=8192 +16)----------HashJoinExec: mode=Partitioned, 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] +17)------------CoalesceBatchesExec: target_batch_size=8192 +18)--------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +19)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false +20)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +21)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +22)----------------CoalesceBatchesExec: target_batch_size=8192 +23)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +24)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +25)----------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +26)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index 53d637ea3f51..fca427dd8632 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -87,13 +87,11 @@ physical_plan 20)--------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey], file_type=csv, has_header=false 21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 22)------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -23)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------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]) -25)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -26)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=csv, has_header=false -27)--------------------------CoalesceBatchesExec: target_batch_size=8192 -28)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -29)------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] -31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_header=false +23)--------------------------------------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]) +24)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=csv, has_header=false +26)--------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +28)------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] +29)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +30)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 51a0d096428c..e2123cfc7e56 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -64,13 +64,12 @@ physical_plan 11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=csv, has_header=false 12)----------------CoalesceBatchesExec: target_batch_size=8192 13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)--------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] -16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false -18)------------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] -19)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -20)----------------CoalesceBatchesExec: target_batch_size=8192 -21)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -22)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -23)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=false +14)--------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] +15)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false +17)------------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] +18)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +19)----------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +21)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +22)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 16a5b7eb39b5..9225f21a61e5 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -90,10 +90,9 @@ physical_plan 21)--------------------CoalesceBatchesExec: target_batch_size=8192 22)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 23)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], file_type=csv, has_header=false -24)----------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -26)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -27)----------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -29)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -30)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], file_type=csv, has_header=false +24)----------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +25)------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +26)--------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +28)------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +29)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index 4960ad1f4a91..12efc64555b2 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -72,12 +72,10 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------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] -11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=csv, has_header=false -12)------------CoalesceBatchesExec: target_batch_size=8192 -13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)----------------CoalesceBatchesExec: target_batch_size=8192 -15)------------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) AND p_size@2 >= 1 -16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=csv, has_header=false +09)----------------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] +10)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=csv, has_header=false +11)------------CoalesceBatchesExec: target_batch_size=8192 +12)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +13)----------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) AND p_size@2 >= 1 +14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index c299fa88a9c4..bf412998b6f4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -123,54 +123,51 @@ physical_plan 22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] 23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] -27)----------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false -29)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -31)------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false -32)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -34)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=csv, has_header=false -35)--------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -37)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -38)------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -40)----------------------CoalesceBatchesExec: target_batch_size=8192 -41)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -42)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -43)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false -44)----------CoalesceBatchesExec: target_batch_size=8192 -45)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -46)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -47)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -48)------------------CoalesceBatchesExec: target_batch_size=8192 -49)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -50)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -51)------------------------CoalesceBatchesExec: target_batch_size=8192 -52)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -53)----------------------------CoalesceBatchesExec: target_batch_size=8192 -54)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -55)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -56)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -59)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -60)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -61)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -62)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -63)------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false -64)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -65)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -66)------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -67)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -68)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -69)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -70)----------------------------CoalesceBatchesExec: target_batch_size=8192 -71)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -72)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -73)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -74)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -75)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +25)------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] +26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false +28)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +30)------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false +31)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +33)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=csv, has_header=false +34)--------------------------CoalesceBatchesExec: target_batch_size=8192 +35)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +36)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +37)------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +39)----------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +40)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +42)----------CoalesceBatchesExec: target_batch_size=8192 +43)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +44)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +45)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +46)------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +48)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +49)------------------------CoalesceBatchesExec: target_batch_size=8192 +50)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +51)----------------------------CoalesceBatchesExec: target_batch_size=8192 +52)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +53)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +54)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +55)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +56)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +57)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +59)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +60)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +61)------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false +62)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +63)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +64)------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +65)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +66)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +67)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +68)----------------------------CoalesceBatchesExec: target_batch_size=8192 +69)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +70)--------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +71)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +72)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 492c68d6aaa0..9dc0f6822a1a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -94,32 +94,29 @@ physical_plan 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false 12)----------------CoalesceBatchesExec: target_batch_size=8192 13)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -14)--------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] -16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -18)--------CoalesceBatchesExec: target_batch_size=8192 -19)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -20)------------CoalesceBatchesExec: target_batch_size=8192 -21)--------------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] -22)----------------CoalesceBatchesExec: target_batch_size=8192 -23)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -24)--------------------CoalesceBatchesExec: target_batch_size=8192 -25)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -26)------------------------CoalesceBatchesExec: target_batch_size=8192 -27)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -28)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=csv, has_header=false -29)------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -31)----------------------------CoalesceBatchesExec: target_batch_size=8192 -32)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] -33)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -34)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false -35)----------------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] -36)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -37)--------------------CoalesceBatchesExec: target_batch_size=8192 -38)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -39)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -40)--------------------------CoalesceBatchesExec: target_batch_size=8192 -41)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -42)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=csv, has_header=false +14)--------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +15)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +17)--------CoalesceBatchesExec: target_batch_size=8192 +18)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +19)------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------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] +21)----------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +23)--------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +25)------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +27)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=csv, has_header=false +28)------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +30)----------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] +31)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +32)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false +33)----------------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] +34)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +35)--------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +37)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +38)--------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] +39)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index 96341ba32311..adb1b9bd10cc 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -118,25 +118,21 @@ physical_plan 26)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false 27)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -29)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -31)------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false -32)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -34)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -36)----------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], file_type=csv, has_header=false -37)------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] -41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -42)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -43)----------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -45)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], file_type=csv, has_header=false -46)------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -48)----------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -50)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false +29)--------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +30)----------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false +31)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +33)------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +34)--------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], file_type=csv, has_header=false +35)------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +37)----------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] +38)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +40)----------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +42)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], file_type=csv, has_header=false +43)------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +45)----------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +46)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index 22476156b80d..818c7bc98965 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -90,17 +90,15 @@ physical_plan 14)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------FilterExec: substr(c_phone@1, 1, 2) IN ([13, 31, 23, 29, 30, 18, 17]) -19)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -20)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false -21)----------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -23)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_custkey], file_type=csv, has_header=false -24)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] -25)----------------------CoalescePartitionsExec -26)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -27)--------------------------CoalesceBatchesExec: target_batch_size=8192 -28)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] -29)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false +17)--------------------------------FilterExec: substr(c_phone@1, 1, 2) IN ([13, 31, 23, 29, 30, 18, 17]) +18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false +20)----------------------------CoalesceBatchesExec: target_batch_size=8192 +21)------------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +22)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_custkey], file_type=csv, has_header=false +23)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] +24)----------------------CoalescePartitionsExec +25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] +26)--------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] +27)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index d982ec32e954..27fd404644fa 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -73,17 +73,14 @@ physical_plan 13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] -18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], file_type=csv, has_header=false -20)--------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -22)------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 -24)----------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=csv, has_header=false -25)------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -27)----------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -29)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +16)------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] +17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], file_type=csv, has_header=false +19)--------------------------CoalesceBatchesExec: target_batch_size=8192 +20)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +21)------------------------------FilterExec: o_orderdate@2 < 1995-03-15 +22)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=csv, has_header=false +23)------------------CoalesceBatchesExec: target_batch_size=8192 +24)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +25)----------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +26)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index f7de3cd3c967..9f53448304a6 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -64,11 +64,9 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] -14)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=csv, has_header=false -15)------------------CoalesceBatchesExec: target_batch_size=8192 -16)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -17)----------------------CoalesceBatchesExec: target_batch_size=8192 -18)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] -19)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false +12)----------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] +13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=csv, has_header=false +14)------------------CoalesceBatchesExec: target_batch_size=8192 +15)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +16)----------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] +17)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index dcf462915899..5c2fa1a40898 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -97,21 +97,19 @@ physical_plan 28)------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false 29)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 30)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -31)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -33)----------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false -34)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -36)----------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false -37)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 -39)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -40)--------------------------CoalesceBatchesExec: target_batch_size=8192 -41)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -42)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -43)------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -45)----------------------CoalesceBatchesExec: target_batch_size=8192 -46)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -47)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -48)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +31)------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +32)--------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false +33)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +35)----------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false +36)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 +38)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +39)--------------------------CoalesceBatchesExec: target_batch_size=8192 +40)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +41)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +42)------------------CoalesceBatchesExec: target_batch_size=8192 +43)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +44)----------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +45)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part index b1e5d2869a8c..eb9063d69171 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part @@ -38,6 +38,5 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------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] -07)------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +05)--------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] +06)----------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index 53ab43ba491b..e19ae5c350e9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -115,24 +115,21 @@ physical_plan 29)--------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false 30)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 31)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -32)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 -34)------------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false -35)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -37)------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey], file_type=csv, has_header=false -38)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -40)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -41)----------------------------CoalesceBatchesExec: target_batch_size=8192 -42)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -43)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -44)----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY -45)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -46)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -47)--------------------CoalesceBatchesExec: target_batch_size=8192 -48)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -49)------------------------CoalesceBatchesExec: target_batch_size=8192 -50)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -51)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -52)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +32)--------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 +33)----------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +34)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +36)------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey], file_type=csv, has_header=false +37)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +39)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +40)----------------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +42)--------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +43)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +44)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +45)--------------------CoalesceBatchesExec: target_batch_size=8192 +46)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +47)------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +48)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +49)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 6b0b05f2f636..5d3ce5e56611 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -126,33 +126,30 @@ physical_plan 34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] 35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -37)------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] -39)----------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)------------------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false -41)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -43)------------------------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false -44)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -45)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -46)----------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -47)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -49)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -50)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -51)------------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false -52)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -53)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -54)------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -55)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -56)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -57)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -58)----------------------------CoalesceBatchesExec: target_batch_size=8192 -59)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -60)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -61)--------------------CoalesceBatchesExec: target_batch_size=8192 -62)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -63)------------------------CoalesceBatchesExec: target_batch_size=8192 -64)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] -65)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -66)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +37)------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] +38)--------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)----------------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false +40)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +42)------------------------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false +43)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +45)----------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +46)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +47)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +48)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +49)----------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false +50)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +51)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +52)------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +53)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +54)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +55)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +56)----------------------------CoalesceBatchesExec: target_batch_size=8192 +57)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +58)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +59)--------------------CoalesceBatchesExec: target_batch_size=8192 +60)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +61)------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] +62)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +63)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index dcee31dfecd3..e4aa46dc15d1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -103,22 +103,21 @@ physical_plan 26)--------------------------------------------------HashJoinExec: mode=Partitioned, 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] 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] -31)------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)--------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false -33)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -35)--------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=csv, has_header=false -36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -38)------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -39)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -41)----------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false -42)----------------------------CoalesceBatchesExec: target_batch_size=8192 -43)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -44)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], file_type=csv, has_header=false -45)--------------------CoalesceBatchesExec: target_batch_size=8192 -46)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -47)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +29)--------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] +30)----------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false +32)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +34)--------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=csv, has_header=false +35)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +37)------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +38)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +40)----------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false +41)----------------------------CoalesceBatchesExec: target_batch_size=8192 +42)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +43)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], file_type=csv, has_header=false +44)--------------------CoalesceBatchesExec: target_batch_size=8192 +45)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +46)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index f7ab6a0c9281..9e63f79f4545 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -513,16 +513,15 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=2 10)------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -12)----------------------CoalesceBatchesExec: target_batch_size=2 -13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] -14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true -16)----ProjectionExec: expr=[1 as cnt] -17)------PlaceholderRowExec -18)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] -19)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": nullable Int64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -20)--------ProjectionExec: expr=[1 as c1] -21)----------PlaceholderRowExec +12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true +15)----ProjectionExec: expr=[1 as cnt] +16)------PlaceholderRowExec +17)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +18)------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": nullable Int64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +19)--------ProjectionExec: expr=[1 as c1] +20)----------PlaceholderRowExec ######## @@ -834,14 +833,12 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--UnionExec -03)----CoalesceBatchesExec: target_batch_size=2 -04)------FilterExec: c1@0 = a -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true -07)----CoalesceBatchesExec: target_batch_size=2 -08)------FilterExec: c1@0 = a -09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true +03)----FilterExec: c1@0 = a +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true +06)----FilterExec: c1@0 = a +07)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true # Clean up after the test statement ok diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 26cb71acbdfe..2e228472d68c 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1885,10 +1885,9 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=4096 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -10)------------------CoalesceBatchesExec: target_batch_size=4096 -11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true +10)------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true query I @@ -3219,10 +3218,9 @@ logical_plan 06)----------TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan 01)ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as rn1] -02)--CoalesceBatchesExec: target_batch_size=4096, fetch=5 -03)----FilterExec: row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 < 50 -04)------BoundedWindowAggExec: wdw=[row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -05)--------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] +02)--FilterExec: row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 < 50, fetch=5 +03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +04)------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] # Top level sort is pushed down through BoundedWindowAggExec as its SUM result does already satisfy the required # global order. The existing sort is for the second-term lexicographical ordering requirement, which is being @@ -3525,9 +3523,8 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] physical_plan 01)BoundedWindowAggExec: wdw=[sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -02)--CoalesceBatchesExec: target_batch_size=4096 -03)----FilterExec: b@2 = 0 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], file_type=csv, has_header=true +02)--FilterExec: b@2 = 0 +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], file_type=csv, has_header=true # Since column b is constant after filter b=0, # window requirement b ASC, d ASC can be satisfied @@ -3544,9 +3541,8 @@ logical_plan physical_plan 01)BoundedWindowAggExec: wdw=[sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 02)--SortExec: expr=[d@4 ASC NULLS LAST], preserve_partitioning=[false] -03)----CoalesceBatchesExec: target_batch_size=4096 -04)------FilterExec: b@2 = 0 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], file_type=csv, has_header=true +03)----FilterExec: b@2 = 0 +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], file_type=csv, has_header=true # Create an unbounded source where there is multiple orderings. @@ -3599,9 +3595,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] 02)--BoundedWindowAggExec: wdw=[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Int32 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -03)----CoalesceBatchesExec: target_batch_size=4096 -04)------FilterExec: d@1 = 0 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true +03)----FilterExec: d@1 = 0 +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) @@ -4216,9 +4211,8 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 -05)--------CoalesceBatchesExec: target_batch_size=4096 -06)----------FilterExec: a@0 = 1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: a@0 = 1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4238,9 +4232,8 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 -05)--------CoalesceBatchesExec: target_batch_size=4096 -06)----------FilterExec: a@0 = 1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------FilterExec: a@0 = 1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT @@ -5307,10 +5300,9 @@ physical_plan 04)------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=1 06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------FilterExec: c1@0 = 2 OR c1@0 = 3 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------FilterExec: c1@0 = 2 OR c1@0 = 3 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5345,13 +5337,12 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rank] -03)----CoalesceBatchesExec: target_batch_size=1 -04)------FilterExec: c2@1 >= 10 -05)--------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] -07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +03)----FilterExec: c2@1 >= 10 +04)------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] +06)----------CoalesceBatchesExec: target_batch_size=1 +07)------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5386,16 +5377,14 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rank] -03)----CoalesceBatchesExec: target_batch_size=1 -04)------FilterExec: c2@1 = 10 -05)--------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -06)----------SortExec: expr=[c2@1 ASC NULLS LAST], preserve_partitioning=[true] -07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -09)----------------CoalesceBatchesExec: target_batch_size=1 -10)------------------FilterExec: c1@0 = 1 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +03)----FilterExec: c2@1 = 10 +04)------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +05)--------SortExec: expr=[c2@1 ASC NULLS LAST], preserve_partitioning=[true] +06)----------CoalesceBatchesExec: target_batch_size=1 +07)------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +08)--------------FilterExec: c1@0 = 1 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5427,13 +5416,12 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rank] -03)----CoalesceBatchesExec: target_batch_size=1 -04)------FilterExec: c1@0 = 1 OR c2@1 = 10 -05)--------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] -07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +03)----FilterExec: c1@0 = 1 OR c2@1 = 10 +04)------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] +06)----------CoalesceBatchesExec: target_batch_size=1 +07)------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5479,10 +5467,9 @@ physical_plan 09)----------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 10)------------------CoalesceBatchesExec: target_batch_size=1 11)--------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -12)----------------------CoalesceBatchesExec: target_batch_size=1 -13)------------------------FilterExec: c1@0 > 1 -14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)----------------------FilterExec: c1@0 > 1 +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 @@ -5526,13 +5513,12 @@ physical_plan 05)--------SortExec: expr=[c2@1 ASC NULLS LAST, c1@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=1 07)------------RepartitionExec: partitioning=Hash([c2@1, c1@0], 2), input_partitions=2 -08)--------------CoalesceBatchesExec: target_batch_size=1 -09)----------------FilterExec: c2@1 > 1 -10)------------------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -11)--------------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] -12)----------------------CoalesceBatchesExec: target_batch_size=1 -13)------------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 -14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------FilterExec: c2@1 > 1 +09)----------------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +10)------------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] +11)--------------------CoalesceBatchesExec: target_batch_size=1 +12)----------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2