diff --git a/benchmarks/src/tpcds/run.rs b/benchmarks/src/tpcds/run.rs index 586ee754d2114..55a7c6f6889d5 100644 --- a/benchmarks/src/tpcds/run.rs +++ b/benchmarks/src/tpcds/run.rs @@ -144,6 +144,10 @@ pub struct RunOpt { /// The tables should have been created with the `--sort` option for this to have any effect. #[arg(short = 't', long = "sorted")] sorted: bool, + + /// How many bytes to buffer on the probe side of hash joins. + #[arg(long, default_value = "1048576")] + hash_join_buffering_capacity: usize, } impl RunOpt { @@ -162,6 +166,8 @@ impl RunOpt { config.options_mut().optimizer.prefer_hash_join = self.prefer_hash_join; config.options_mut().optimizer.enable_piecewise_merge_join = self.enable_piecewise_merge_join; + config.options_mut().execution.hash_join_buffering_capacity = + self.hash_join_buffering_capacity; let rt_builder = self.common.runtime_env_builder()?; let ctx = SessionContext::new_with_config_rt(config, rt_builder.build_arc()?); // register tables diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 2b8385ac2d89c..abf068db0cca8 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -269,7 +269,7 @@ impl StatementExecutor { let options = task_ctx.session_config().options(); // Track memory usage for the query result if it's bounded - let mut reservation = + let reservation = MemoryConsumer::new("DataFusion-Cli").register(task_ctx.memory_pool()); if physical_plan.boundedness().is_unbounded() { diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 87344914d2f7e..508a4d2a20744 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -669,6 +669,17 @@ config_namespace! { /// # Default /// `false` — ANSI SQL mode is disabled by default. pub enable_ansi_mode: bool, default = false + + /// How many bytes to buffer in the probe side of hash joins while the build side is + /// concurrently being built. + /// + /// Without this, hash joins will wait until the full materialization of the build side + /// before polling the probe side. This is useful in scenarios where the query is not + /// completely CPU bounded, allowing to do some early work concurrently and reducing the + /// latency of the query. + /// + /// 1Mb by default. Set to 0 for disabling it. + pub hash_join_buffering_capacity: usize, default = 1024 * 1024 } } diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 712f4389f5852..b6c606ff467f9 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -2186,7 +2186,7 @@ mod tests { // configure with same memory / disk manager let memory_pool = ctx1.runtime_env().memory_pool.clone(); - let mut reservation = MemoryConsumer::new("test").register(&memory_pool); + let reservation = MemoryConsumer::new("test").register(&memory_pool); reservation.grow(100); let disk_manager = ctx1.runtime_env().disk_manager.clone(); diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index c09db371912b0..6749ba4492b41 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2967,8 +2967,9 @@ async fn test_count_wildcard_on_where_in() -> Result<()> { | physical_plan | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | | | ProjectionExec: expr=[4 as count(*)] | | | PlaceholderRowExec | - | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | BufferExec: capacity=1048576 | + | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+----------------------------------------------------------------------------------------------------------------------+ " @@ -3011,8 +3012,9 @@ async fn test_count_wildcard_on_where_in() -> Result<()> { | physical_plan | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | | | ProjectionExec: expr=[4 as count(*)] | | | PlaceholderRowExec | - | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | BufferExec: capacity=1048576 | + | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+----------------------------------------------------------------------------------------------------------------------+ " @@ -3351,7 +3353,8 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | 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] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | BufferExec: capacity=1048576 | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+----------------------------------------------------------------------------------------------------------------------------+ " @@ -3407,7 +3410,8 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | 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] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | BufferExec: capacity=1048576 | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+----------------------------------------------------------------------------------------------------------------------------+ " diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index fa248c448683b..3a7b8cc81b0d5 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -804,8 +804,9 @@ async fn test_physical_plan_display_indent_multi_children() { HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)], projection=[c1@0] RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=1 DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true - RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=1 - DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1@0 as c2], file_type=csv, has_header=true + BufferExec: capacity=1048576 + RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=1 + DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1@0 as c2], file_type=csv, has_header=true " ); } diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 6635c9072dd97..d59b42ed15d15 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -1360,7 +1360,7 @@ impl FileSink for ParquetSink { parquet_props.clone(), ) .await?; - let mut reservation = MemoryConsumer::new(format!("ParquetSink[{path}]")) + let reservation = MemoryConsumer::new(format!("ParquetSink[{path}]")) .register(context.memory_pool()); file_write_tasks.spawn(async move { while let Some(batch) = rx.recv().await { @@ -1465,7 +1465,7 @@ impl DataSink for ParquetSink { async fn column_serializer_task( mut rx: Receiver, mut writer: ArrowColumnWriter, - mut reservation: MemoryReservation, + reservation: MemoryReservation, ) -> Result<(ArrowColumnWriter, MemoryReservation)> { while let Some(col) = rx.recv().await { writer.write(&col)?; @@ -1550,7 +1550,7 @@ fn spawn_rg_join_and_finalize_task( rg_rows: usize, pool: &Arc, ) -> SpawnedTask { - let mut rg_reservation = + let rg_reservation = MemoryConsumer::new("ParquetSink(SerializedRowGroupWriter)").register(pool); SpawnedTask::spawn(async move { @@ -1682,12 +1682,12 @@ async fn concatenate_parallel_row_groups( mut object_store_writer: Box, pool: Arc, ) -> Result { - let mut file_reservation = + let file_reservation = MemoryConsumer::new("ParquetSink(SerializedFileWriter)").register(&pool); while let Some(task) = serialize_rx.recv().await { let result = task.join_unwind().await; - let (serialized_columns, mut rg_reservation, _cnt) = + let (serialized_columns, rg_reservation, _cnt) = result.map_err(|e| DataFusionError::ExecutionJoin(Box::new(e)))??; let mut rg_out = parquet_writer.next_row_group()?; diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index de18b6be2235f..0eada1fc037dd 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -37,8 +37,10 @@ use itertools::Itertools; use crate::file_scan_config::FileScanConfig; use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{Constraints, Result, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::expressions::DynamicFilterPhysicalExpr; use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::SortOrderPushdownResult; @@ -410,6 +412,30 @@ impl ExecutionPlan for DataSourceExec { as Arc }) } + + fn dynamic_filters(&self) -> Vec> { + let Some(node) = self.data_source.as_any().downcast_ref::() + else { + return vec![]; + }; + + let Some(filter) = node.file_source.filter() else { + return vec![]; + }; + + let mut filters = vec![]; + let _ = filter.transform_down(|expr| { + if let Ok(dynamic_filter) = + Arc::downcast::(Arc::clone(&expr) as _) + { + filters.push(dynamic_filter); + }; + + Ok(Transformed::no(expr)) + }); + + filters + } } impl DataSourceExec { diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index fbf9ce41da8fe..b23eede2a054e 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -18,7 +18,7 @@ //! [`MemoryPool`] for memory management during query execution, [`proxy`] for //! help with allocation accounting. -use datafusion_common::{Result, internal_err}; +use datafusion_common::{Result, internal_datafusion_err}; use std::hash::{Hash, Hasher}; use std::{cmp::Ordering, sync::Arc, sync::atomic}; @@ -322,7 +322,7 @@ impl MemoryConsumer { pool: Arc::clone(pool), consumer: self, }), - size: 0, + size: atomic::AtomicUsize::new(0), } } } @@ -351,13 +351,13 @@ impl Drop for SharedRegistration { #[derive(Debug)] pub struct MemoryReservation { registration: Arc, - size: usize, + size: atomic::AtomicUsize, } impl MemoryReservation { /// Returns the size of this reservation in bytes pub fn size(&self) -> usize { - self.size + self.size.load(atomic::Ordering::Relaxed) } /// Returns [MemoryConsumer] for this [MemoryReservation] @@ -367,8 +367,8 @@ impl MemoryReservation { /// Frees all bytes from this reservation back to the underlying /// pool, returning the number of bytes freed. - pub fn free(&mut self) -> usize { - let size = self.size; + pub fn free(&self) -> usize { + let size = self.size.load(atomic::Ordering::Relaxed); if size != 0 { self.shrink(size) } @@ -380,60 +380,62 @@ impl MemoryReservation { /// # Panics /// /// Panics if `capacity` exceeds [`Self::size`] - pub fn shrink(&mut self, capacity: usize) { - let new_size = self.size.checked_sub(capacity).unwrap(); + pub fn shrink(&self, capacity: usize) { + self.size.fetch_sub(capacity, atomic::Ordering::Relaxed); self.registration.pool.shrink(self, capacity); - self.size = new_size } /// Tries to free `capacity` bytes from this reservation /// if `capacity` does not exceed [`Self::size`] /// Returns new reservation size /// or error if shrinking capacity is more than allocated size - pub fn try_shrink(&mut self, capacity: usize) -> Result { - if let Some(new_size) = self.size.checked_sub(capacity) { - self.registration.pool.shrink(self, capacity); - self.size = new_size; - Ok(new_size) - } else { - internal_err!( - "Cannot free the capacity {capacity} out of allocated size {}", - self.size + pub fn try_shrink(&self, capacity: usize) -> Result { + let updated = self.size.fetch_update( + atomic::Ordering::Relaxed, + atomic::Ordering::Relaxed, + |prev| prev.checked_sub(capacity), + ); + updated.map_err(|_| { + let prev = self.size.load(atomic::Ordering::Relaxed); + internal_datafusion_err!( + "Cannot free the capacity {capacity} out of allocated size {prev}" ) - } + }) } /// Sets the size of this reservation to `capacity` - pub fn resize(&mut self, capacity: usize) { - match capacity.cmp(&self.size) { - Ordering::Greater => self.grow(capacity - self.size), - Ordering::Less => self.shrink(self.size - capacity), + pub fn resize(&self, capacity: usize) { + let size = self.size.load(atomic::Ordering::Relaxed); + match capacity.cmp(&size) { + Ordering::Greater => self.grow(capacity - size), + Ordering::Less => self.shrink(size - capacity), _ => {} } } /// Try to set the size of this reservation to `capacity` - pub fn try_resize(&mut self, capacity: usize) -> Result<()> { - match capacity.cmp(&self.size) { - Ordering::Greater => self.try_grow(capacity - self.size)?, - Ordering::Less => self.shrink(self.size - capacity), + pub fn try_resize(&self, capacity: usize) -> Result<()> { + let size = self.size.load(atomic::Ordering::Relaxed); + match capacity.cmp(&size) { + Ordering::Greater => self.try_grow(capacity - size)?, + Ordering::Less => self.shrink(size - capacity), _ => {} }; Ok(()) } /// Increase the size of this reservation by `capacity` bytes - pub fn grow(&mut self, capacity: usize) { + pub fn grow(&self, capacity: usize) { self.registration.pool.grow(self, capacity); - self.size += capacity; + self.size.fetch_add(capacity, atomic::Ordering::Relaxed); } /// Try to increase the size of this reservation by `capacity` /// bytes, returning error if there is insufficient capacity left /// in the pool. - pub fn try_grow(&mut self, capacity: usize) -> Result<()> { + pub fn try_grow(&self, capacity: usize) -> Result<()> { self.registration.pool.try_grow(self, capacity)?; - self.size += capacity; + self.size.fetch_add(capacity, atomic::Ordering::Relaxed); Ok(()) } @@ -447,10 +449,16 @@ impl MemoryReservation { /// # Panics /// /// Panics if `capacity` exceeds [`Self::size`] - pub fn split(&mut self, capacity: usize) -> MemoryReservation { - self.size = self.size.checked_sub(capacity).unwrap(); + pub fn split(&self, capacity: usize) -> MemoryReservation { + self.size + .fetch_update( + atomic::Ordering::Relaxed, + atomic::Ordering::Relaxed, + |prev| prev.checked_sub(capacity), + ) + .unwrap(); Self { - size: capacity, + size: atomic::AtomicUsize::new(capacity), registration: Arc::clone(&self.registration), } } @@ -458,7 +466,7 @@ impl MemoryReservation { /// Returns a new empty [`MemoryReservation`] with the same [`MemoryConsumer`] pub fn new_empty(&self) -> Self { Self { - size: 0, + size: atomic::AtomicUsize::new(0), registration: Arc::clone(&self.registration), } } @@ -466,7 +474,7 @@ impl MemoryReservation { /// Splits off all the bytes from this [`MemoryReservation`] into /// a new [`MemoryReservation`] with the same [`MemoryConsumer`] pub fn take(&mut self) -> MemoryReservation { - self.split(self.size) + self.split(self.size.load(atomic::Ordering::Relaxed)) } } @@ -492,7 +500,7 @@ mod tests { #[test] fn test_memory_pool_underflow() { let pool = Arc::new(GreedyMemoryPool::new(50)) as _; - let mut a1 = MemoryConsumer::new("a1").register(&pool); + let a1 = MemoryConsumer::new("a1").register(&pool); assert_eq!(pool.reserved(), 0); a1.grow(100); @@ -507,7 +515,7 @@ mod tests { a1.try_grow(30).unwrap(); assert_eq!(pool.reserved(), 30); - let mut a2 = MemoryConsumer::new("a2").register(&pool); + let a2 = MemoryConsumer::new("a2").register(&pool); a2.try_grow(25).unwrap_err(); assert_eq!(pool.reserved(), 30); @@ -521,7 +529,7 @@ mod tests { #[test] fn test_split() { let pool = Arc::new(GreedyMemoryPool::new(50)) as _; - let mut r1 = MemoryConsumer::new("r1").register(&pool); + let r1 = MemoryConsumer::new("r1").register(&pool); r1.try_grow(20).unwrap(); assert_eq!(r1.size(), 20); @@ -542,10 +550,10 @@ mod tests { #[test] fn test_new_empty() { let pool = Arc::new(GreedyMemoryPool::new(50)) as _; - let mut r1 = MemoryConsumer::new("r1").register(&pool); + let r1 = MemoryConsumer::new("r1").register(&pool); r1.try_grow(20).unwrap(); - let mut r2 = r1.new_empty(); + let r2 = r1.new_empty(); r2.try_grow(5).unwrap(); assert_eq!(r1.size(), 20); @@ -559,7 +567,7 @@ mod tests { let mut r1 = MemoryConsumer::new("r1").register(&pool); r1.try_grow(20).unwrap(); - let mut r2 = r1.take(); + let r2 = r1.take(); r2.try_grow(5).unwrap(); assert_eq!(r1.size(), 0); diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index bf74b5f6f4c6b..b10270851cc06 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -212,7 +212,7 @@ impl MemoryPool for FairSpillPool { .checked_div(state.num_spill) .unwrap_or(spill_available); - if reservation.size + additional > available { + if reservation.size() + additional > available { return Err(insufficient_capacity_err( reservation, additional, @@ -264,7 +264,7 @@ fn insufficient_capacity_err( "Failed to allocate additional {} for {} with {} already allocated for this reservation - {} remain available for the total pool", human_readable_size(additional), reservation.registration.consumer.name, - human_readable_size(reservation.size), + human_readable_size(reservation.size()), human_readable_size(available) ) } @@ -526,12 +526,12 @@ mod tests { fn test_fair() { let pool = Arc::new(FairSpillPool::new(100)) as _; - let mut r1 = MemoryConsumer::new("unspillable").register(&pool); + let r1 = MemoryConsumer::new("unspillable").register(&pool); // Can grow beyond capacity of pool r1.grow(2000); assert_eq!(pool.reserved(), 2000); - let mut r2 = MemoryConsumer::new("r2") + let r2 = MemoryConsumer::new("r2") .with_can_spill(true) .register(&pool); // Can grow beyond capacity of pool @@ -563,7 +563,7 @@ mod tests { assert_eq!(r2.size(), 10); assert_eq!(pool.reserved(), 30); - let mut r3 = MemoryConsumer::new("r3") + let r3 = MemoryConsumer::new("r3") .with_can_spill(true) .register(&pool); @@ -584,7 +584,7 @@ mod tests { r1.free(); assert_eq!(pool.reserved(), 80); - let mut r4 = MemoryConsumer::new("s4").register(&pool); + let r4 = MemoryConsumer::new("s4").register(&pool); let err = r4.try_grow(30).unwrap_err().strip_backtrace(); assert_snapshot!(err, @"Resources exhausted: Failed to allocate additional 30.0 B for s4 with 0.0 B already allocated for this reservation - 20.0 B remain available for the total pool"); } @@ -601,18 +601,18 @@ mod tests { // Test: use all the different interfaces to change reservation size // set r1=50, using grow and shrink - let mut r1 = MemoryConsumer::new("r1").register(&pool); + let r1 = MemoryConsumer::new("r1").register(&pool); r1.grow(50); r1.grow(20); r1.shrink(20); // set r2=15 using try_grow - let mut r2 = MemoryConsumer::new("r2").register(&pool); + let r2 = MemoryConsumer::new("r2").register(&pool); r2.try_grow(15) .expect("should succeed in memory allotment for r2"); // set r3=20 using try_resize - let mut r3 = MemoryConsumer::new("r3").register(&pool); + let r3 = MemoryConsumer::new("r3").register(&pool); r3.try_resize(25) .expect("should succeed in memory allotment for r3"); r3.try_resize(20) @@ -620,12 +620,12 @@ mod tests { // set r4=10 // this should not be reported in top 3 - let mut r4 = MemoryConsumer::new("r4").register(&pool); + let r4 = MemoryConsumer::new("r4").register(&pool); r4.grow(10); // Test: reports if new reservation causes error // using the previously set sizes for other consumers - let mut r5 = MemoryConsumer::new("r5").register(&pool); + let r5 = MemoryConsumer::new("r5").register(&pool); let res = r5.try_grow(150); assert!(res.is_err()); let error = res.unwrap_err().strip_backtrace(); @@ -650,7 +650,7 @@ mod tests { let same_name = "foo"; // Test: see error message when no consumers recorded yet - let mut r0 = MemoryConsumer::new(same_name).register(&pool); + let r0 = MemoryConsumer::new(same_name).register(&pool); let res = r0.try_grow(150); assert!(res.is_err()); let error = res.unwrap_err().strip_backtrace(); @@ -665,7 +665,7 @@ mod tests { r0.grow(10); // make r0=10, pool available=90 let new_consumer_same_name = MemoryConsumer::new(same_name); - let mut r1 = new_consumer_same_name.register(&pool); + let r1 = new_consumer_same_name.register(&pool); // TODO: the insufficient_capacity_err() message is per reservation, not per consumer. // a followup PR will clarify this message "0 bytes already allocated for this reservation" let res = r1.try_grow(150); @@ -695,7 +695,7 @@ mod tests { // will be recognized as different in the TrackConsumersPool let consumer_with_same_name_but_different_hash = MemoryConsumer::new(same_name).with_can_spill(true); - let mut r2 = consumer_with_same_name_but_different_hash.register(&pool); + let r2 = consumer_with_same_name_but_different_hash.register(&pool); let res = r2.try_grow(150); assert!(res.is_err()); let error = res.unwrap_err().strip_backtrace(); @@ -714,10 +714,10 @@ mod tests { // Baseline: see the 2 memory consumers let setting = make_settings(); let _bound = setting.bind_to_scope(); - let mut r0 = MemoryConsumer::new("r0").register(&pool); + let r0 = MemoryConsumer::new("r0").register(&pool); r0.grow(10); let r1_consumer = MemoryConsumer::new("r1"); - let mut r1 = r1_consumer.register(&pool); + let r1 = r1_consumer.register(&pool); r1.grow(20); let res = r0.try_grow(150); @@ -791,13 +791,13 @@ mod tests { .downcast::>() .unwrap(); // set r1=20 - let mut r1 = MemoryConsumer::new("r1").register(&pool); + let r1 = MemoryConsumer::new("r1").register(&pool); r1.grow(20); // set r2=15 - let mut r2 = MemoryConsumer::new("r2").register(&pool); + let r2 = MemoryConsumer::new("r2").register(&pool); r2.grow(15); // set r3=45 - let mut r3 = MemoryConsumer::new("r3").register(&pool); + let r3 = MemoryConsumer::new("r3").register(&pool); r3.grow(45); let downcasted = upcasted diff --git a/datafusion/physical-optimizer/src/hash_join_buffering.rs b/datafusion/physical-optimizer/src/hash_join_buffering.rs new file mode 100644 index 0000000000000..3c29b46c0fa64 --- /dev/null +++ b/datafusion/physical-optimizer/src/hash_join_buffering.rs @@ -0,0 +1,103 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::PhysicalOptimizerRule; +use datafusion_common::JoinSide; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_physical_plan::ExecutionPlan; +use datafusion_physical_plan::buffer::BufferExec; +use datafusion_physical_plan::joins::HashJoinExec; +use std::sync::Arc; + +/// Looks for all the [HashJoinExec]s in the plan and places a [BufferExec] node with the +/// configured capacity in the probe side: +/// +/// ```text +/// ┌───────────────────┐ +/// │ HashJoinExec │ +/// └─────▲────────▲────┘ +/// ┌───────┘ └─────────┐ +/// │ │ +/// ┌────────────────┐ ┌─────────────────┐ +/// │ Build side │ + │ BufferExec │ +/// └────────────────┘ └────────▲────────┘ +/// │ +/// ┌────────┴────────┐ +/// │ Probe side │ +/// └─────────────────┘ +/// ``` +/// +/// Which allows eagerly pulling it even before the build side has completely finished. +#[derive(Debug, Default)] +pub struct HashJoinBuffering {} + +impl HashJoinBuffering { + pub fn new() -> Self { + Self::default() + } +} + +impl PhysicalOptimizerRule for HashJoinBuffering { + fn optimize( + &self, + plan: Arc, + config: &ConfigOptions, + ) -> datafusion_common::Result> { + let capacity = config.execution.hash_join_buffering_capacity; + if capacity == 0 { + return Ok(plan); + } + + plan.transform_down(|plan| { + let Some(node) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + let plan = Arc::clone(&plan); + Ok(Transformed::yes( + if HashJoinExec::probe_side() == JoinSide::Left { + // Do not stack BufferExec nodes together. + if node.left.as_any().downcast_ref::().is_some() { + return Ok(Transformed::no(plan)); + } + plan.with_new_children(vec![ + Arc::new(BufferExec::new(Arc::clone(&node.left), capacity)), + Arc::clone(&node.right), + ])? + } else { + // Do not stack BufferExec nodes together. + if node.right.as_any().downcast_ref::().is_some() { + return Ok(Transformed::no(plan)); + } + plan.with_new_children(vec![ + Arc::clone(&node.left), + Arc::new(BufferExec::new(Arc::clone(&node.right), capacity)), + ])? + }, + )) + }) + .data() + } + + fn name(&self) -> &str { + "HashJoinBuffering" + } + + fn schema_check(&self) -> bool { + true + } +} diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index e98772291cbeb..8b902023d2be8 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -41,6 +41,7 @@ pub mod optimizer; pub mod output_requirements; pub mod projection_pushdown; pub use datafusion_pruning as pruning; +pub mod hash_join_buffering; pub mod pushdown_sort; pub mod sanity_checker; pub mod topk_aggregation; diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index ff71c9ec64385..2cd9c6dd3568b 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -35,6 +35,7 @@ use crate::sanity_checker::SanityCheckPlan; use crate::topk_aggregation::TopKAggregation; use crate::update_aggr_exprs::OptimizeAggregateOrder; +use crate::hash_join_buffering::HashJoinBuffering; use crate::limit_pushdown_past_window::LimitPushPastWindows; use crate::pushdown_sort::PushdownSort; use datafusion_common::Result; @@ -131,6 +132,10 @@ impl PhysicalOptimizer { // This can possibly be combined with [LimitPushdown] // It needs to come after [EnforceSorting] Arc::new(LimitPushPastWindows::new()), + // The HashJoinBuffering rule adds a BufferExec node with the configured capacity + // in the prob side of hash joins. That way, the probe side gets eagerly polled before + // the build side is completely finished. + Arc::new(HashJoinBuffering::new()), // The LimitPushdown rule tries to push limits down as far as possible, // replacing operators with fetching variants, or adding limits // past operators that support limit pushdown. diff --git a/datafusion/physical-plan/src/buffer.rs b/datafusion/physical-plan/src/buffer.rs new file mode 100644 index 0000000000000..798fdf01a4bbd --- /dev/null +++ b/datafusion/physical-plan/src/buffer.rs @@ -0,0 +1,672 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! [`BufferExec`] decouples production and consumption on messages by buffering the input in the +//! background up to a certain capacity. + +use crate::execution_plan::{CardinalityEffect, SchedulingType}; +use crate::filter_pushdown::{ + ChildPushdownResult, FilterDescription, FilterPushdownPhase, + FilterPushdownPropagation, +}; +use crate::projection::ProjectionExec; +use crate::stream::RecordBatchStreamAdapter; +use crate::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SortOrderPushdownResult, +}; +use arrow::array::RecordBatch; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; +use datafusion_common::{Result, Statistics, internal_err, plan_err}; +use datafusion_common_runtime::SpawnedTask; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr_common::metrics::{ + ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, +}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use futures::{Stream, StreamExt, TryStreamExt}; +use pin_project_lite::pin_project; +use std::any::Any; +use std::fmt; +use std::pin::Pin; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::{Arc, OnceLock}; +use std::task::{Context, Poll}; +use tokio::sync::mpsc::UnboundedReceiver; +use tokio::sync::{Notify, OwnedSemaphorePermit, Semaphore}; + +/// Decouples production and consumption of record batches with an internal queue per partition, +/// eagerly filling up the capacity of the queues even before any message is requested. +/// +/// ```text +/// ┌───────────────────────────┐ +/// │ BufferExec │ +/// │ │ +/// │┌────── Partition 0 ──────┐│ +/// ││ ┌────┐ ┌────┐││ ┌────┐ +/// ──background poll────────▶│ │ │ ├┼┼───────▶ │ +/// ││ └────┘ └────┘││ └────┘ +/// │└─────────────────────────┘│ +/// │┌────── Partition 1 ──────┐│ +/// ││ ┌────┐ ┌────┐ ┌────┐││ ┌────┐ +/// ──background poll─▶│ │ │ │ │ ├┼┼───────▶ │ +/// ││ └────┘ └────┘ └────┘││ └────┘ +/// │└─────────────────────────┘│ +/// │ │ +/// │ ... │ +/// │ │ +/// │┌────── Partition N ──────┐│ +/// ││ ┌────┐││ ┌────┐ +/// ──background poll───────────────▶│ ├┼┼───────▶ │ +/// ││ └────┘││ └────┘ +/// │└─────────────────────────┘│ +/// └───────────────────────────┘ +/// ``` +/// +/// The capacity is provided in bytes, and for each buffered record batch it will take into account +/// the size reported by [RecordBatch::get_array_memory_size]. +/// +/// This is useful for operators that conditionally start polling one of their children only after +/// other child has finished, allowing to perform some early work and accumulating batches in +/// memory so that they can be served immediately when requested. +#[derive(Debug, Clone)] +pub struct BufferExec { + input: Arc, + properties: PlanProperties, + capacity: usize, + metrics: ExecutionPlanMetricsSet, + lazy_dyn_filters_below: Arc>, +} + +impl BufferExec { + /// Builds a new [BufferExec] with the provided capacity in bytes. + pub fn new(input: Arc, capacity: usize) -> Self { + let properties = input + .properties() + .clone() + .with_scheduling_type(SchedulingType::Cooperative); + + Self { + input, + properties, + capacity, + metrics: ExecutionPlanMetricsSet::new(), + lazy_dyn_filters_below: Arc::new(OnceLock::new()), + } + } + + /// Returns the input [ExecutionPlan] of this [BufferExec]. + pub fn input(&self) -> &Arc { + &self.input + } + + /// Returns the per-partition capacity in bytes for this [BufferExec]. + pub fn capacity(&self) -> usize { + self.capacity + } +} + +impl DisplayAs for BufferExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "BufferExec: capacity={}", self.capacity) + } + DisplayFormatType::TreeRender => { + writeln!(f, "target_batch_size={}", self.capacity) + } + } + } +} + +impl ExecutionPlan for BufferExec { + fn name(&self) -> &str { + "BufferExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.properties + } + + fn maintains_input_order(&self) -> Vec { + vec![true] + } + + fn benefits_from_input_partitioning(&self) -> Vec { + vec![false] + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + fn with_new_children( + self: Arc, + mut children: Vec>, + ) -> Result> { + if children.len() != 1 { + return plan_err!("BufferExec can only have one child"); + } + Ok(Arc::new(Self::new(children.swap_remove(0), self.capacity))) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + let mem_reservation = MemoryConsumer::new(format!("BufferExec[{partition}]")) + .register(context.memory_pool()); + let in_stream = self.input.execute(partition, context)?; + + let input = Arc::clone(&self.input); + let has_dynamic_filter_below = self.lazy_dyn_filters_below.get_or_init(|| { + let mut result = false; + let _ = input.transform_up(|plan| { + if !plan.dynamic_filters().is_empty() { + result = true; + return Ok(Transformed::new(plan, false, TreeNodeRecursion::Stop)); + } + Ok(Transformed::no(plan)) + }); + result + }); + + // Set up the metrics for the stream. + let curr_mem_in = Arc::new(AtomicUsize::new(0)); + let curr_mem_out = Arc::clone(&curr_mem_in); + let mut max_mem_in = 0; + let max_mem = MetricBuilder::new(&self.metrics).gauge("max_mem_used", partition); + + let curr_queued_in = Arc::new(AtomicUsize::new(0)); + let curr_queued_out = Arc::clone(&curr_queued_in); + let mut max_queued_in = 0; + let max_queued = MetricBuilder::new(&self.metrics).gauge("max_queued", partition); + + // Capture metrics when an element is queued on the stream. + let in_stream = in_stream.inspect_ok(move |v| { + let size = v.get_array_memory_size(); + let curr_size = curr_mem_in.fetch_add(size, Ordering::Relaxed) + size; + if curr_size > max_mem_in { + max_mem_in = curr_size; + max_mem.set(max_mem_in); + } + + let curr_queued = curr_queued_in.fetch_add(1, Ordering::Relaxed) + 1; + if curr_queued > max_queued_in { + max_queued_in = curr_queued; + max_queued.set(max_queued_in); + } + }); + // Buffer the input. + let out_stream = MemoryBufferedStream::new( + in_stream, + self.capacity, + mem_reservation, + *has_dynamic_filter_below, + ); + // Update in the metrics that when an element gets out, some memory gets freed. + let out_stream = out_stream.inspect_ok(move |v| { + curr_mem_out.fetch_sub(v.get_array_memory_size(), Ordering::Relaxed); + curr_queued_out.fetch_sub(1, Ordering::Relaxed); + }); + + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + out_stream, + ))) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + + fn partition_statistics(&self, partition: Option) -> Result { + self.input.partition_statistics(partition) + } + + fn supports_limit_pushdown(&self) -> bool { + self.input.supports_limit_pushdown() + } + + fn cardinality_effect(&self) -> CardinalityEffect { + CardinalityEffect::Equal + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + match self.input.try_swapping_with_projection(projection)? { + Some(new_input) => Ok(Some( + Arc::new(self.clone()).with_new_children(vec![new_input])?, + )), + None => Ok(None), + } + } + + fn gather_filters_for_pushdown( + &self, + _phase: FilterPushdownPhase, + parent_filters: Vec>, + _config: &ConfigOptions, + ) -> Result { + FilterDescription::from_children(parent_filters, &self.children()) + } + + fn handle_child_pushdown_result( + &self, + _phase: FilterPushdownPhase, + child_pushdown_result: ChildPushdownResult, + _config: &ConfigOptions, + ) -> Result>> { + Ok(FilterPushdownPropagation::if_all(child_pushdown_result)) + } + + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + // CoalesceBatchesExec is transparent for sort ordering - it preserves order + // Delegate to the child and wrap with a new CoalesceBatchesExec + self.input.try_pushdown_sort(order)?.try_map(|new_input| { + Ok(Arc::new(Self::new(new_input, self.capacity)) as Arc) + }) + } +} + +/// Represents anything that occupies a capacity in a [MemoryBufferedStream]. +pub trait SizedMessage { + fn size(&self) -> usize; +} + +impl SizedMessage for RecordBatch { + fn size(&self) -> usize { + self.get_array_memory_size() + } +} + +pin_project! { +/// Decouples production and consumption of messages in a stream with an internal queue, eagerly +/// filling it up to the specified maximum capacity even before any message is requested. +/// +/// Allows each message to have a different size, which is taken into account for determining if +/// the queue is full or not. +pub struct MemoryBufferedStream { + task: SpawnedTask<()>, + batch_rx: UnboundedReceiver>, + memory_reservation: Arc, + first_poll_notify: Option>, +}} + +impl MemoryBufferedStream { + /// Builds a new [MemoryBufferedStream] with the provided capacity and event handler. + /// + /// This immediately spawns a Tokio task that will start consumption of the input stream. + pub fn new( + mut input: impl Stream> + Unpin + Send + 'static, + capacity: usize, + memory_reservation: MemoryReservation, + wait_first_pool: bool, + ) -> Self { + let semaphore = Arc::new(Semaphore::new(capacity)); + let (batch_tx, batch_rx) = tokio::sync::mpsc::unbounded_channel(); + + let mut first_poll_notify = None; + if wait_first_pool { + first_poll_notify = Some(Arc::new(Notify::new())); + } + let mut first_poll_notify_clone = first_poll_notify.clone(); + + let memory_reservation = Arc::new(memory_reservation); + let memory_reservation_clone = Arc::clone(&memory_reservation); + let task = SpawnedTask::spawn(async move { + if let Some(first_poll_notify) = first_poll_notify_clone.take() { + first_poll_notify.notified_owned().await; + } + while let Some(item_or_err) = input.next().await { + let item = match item_or_err { + Ok(batch) => batch, + Err(err) => { + let _ = batch_tx.send(Err(err)); // If there's an error it means the channel was closed, which is fine. + break; + } + }; + + let size = item.size(); + if let Err(err) = memory_reservation.try_grow(size) { + let _ = batch_tx.send(Err(err)); // If there's an error it means the channel was closed, which is fine. + break; + } + + // We need to cap the minimum between amount of permits and the actual size of the + // message. If at any point we try to acquire more permits than the capacity of the + // semaphore, the stream will deadlock. + let capped_size = size.min(capacity) as u32; + + let semaphore = Arc::clone(&semaphore); + let Ok(permit) = semaphore.acquire_many_owned(capped_size).await else { + let _ = batch_tx.send(internal_err!("Closed semaphore in MemoryBufferedStream. This is a bug in DataFusion, please report it!")); + break; + }; + + if batch_tx.send(Ok((item, permit))).is_err() { + break; // stream was closed + }; + } + }); + + Self { + task, + batch_rx, + memory_reservation: memory_reservation_clone, + first_poll_notify, + } + } + + /// Returns the number of queued messages. + pub fn messages_queued(&self) -> usize { + self.batch_rx.len() + } +} + +impl Stream for MemoryBufferedStream { + type Item = Result; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let self_project = self.project(); + if let Some(first_poll_notify) = self_project.first_poll_notify.take() { + first_poll_notify.notify_one(); + } + match self_project.batch_rx.poll_recv(cx) { + Poll::Ready(Some(Ok((item, _semaphore_permit)))) => { + self_project.memory_reservation.shrink(item.size()); + Poll::Ready(Some(Ok(item))) + } + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(err))), + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending, + } + } + + fn size_hint(&self) -> (usize, Option) { + if self.batch_rx.is_closed() { + let len = self.batch_rx.len(); + (len, Some(len)) + } else { + (self.batch_rx.len(), None) + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use datafusion_common::{DataFusionError, assert_contains}; + use datafusion_execution::memory_pool::{ + GreedyMemoryPool, MemoryPool, UnboundedMemoryPool, + }; + use std::error::Error; + use std::fmt::Debug; + use std::sync::Arc; + use std::time::Duration; + use tokio::time::timeout; + + #[tokio::test] + async fn buffers_only_some_messages() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let buffered = MemoryBufferedStream::new(input, 4, res, false); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 2); + Ok(()) + } + + #[tokio::test] + async fn respects_wait_for_first_poll() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let buffered = MemoryBufferedStream::new(input, 4, res, true); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 0); + Ok(()) + } + + #[tokio::test] + async fn yields_all_messages() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 10, res, false); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 4); + + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + finished(&mut buffered).await?; + Ok(()) + } + + #[tokio::test] + async fn yields_all_messages_after_first_poll() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 10, res, true); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 0); + + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + finished(&mut buffered).await?; + Ok(()) + } + + #[tokio::test] + async fn yields_first_msg_even_if_big() -> Result<(), Box> { + let input = futures::stream::iter([25, 1, 2, 3]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 10, res, false); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + Ok(()) + } + + #[tokio::test] + async fn memory_pool_kills_stream() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = bounded_memory_pool_and_reservation(7); + + let mut buffered = MemoryBufferedStream::new(input, 10, res, false); + wait_for_buffering().await; + + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + let msg = pull_err_msg(&mut buffered).await?; + + assert_contains!(msg.to_string(), "Failed to allocate additional 4.0 B"); + Ok(()) + } + + #[tokio::test] + async fn memory_pool_does_not_kill_stream() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = bounded_memory_pool_and_reservation(7); + + let mut buffered = MemoryBufferedStream::new(input, 3, res, false); + wait_for_buffering().await; + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + finished(&mut buffered).await?; + Ok(()) + } + + #[tokio::test] + async fn messages_pass_even_if_all_exceed_limit() -> Result<(), Box> { + let input = futures::stream::iter([3, 3, 3, 3]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 2, res, false); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + finished(&mut buffered).await?; + Ok(()) + } + + #[tokio::test] + async fn errors_get_propagated() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(|v| { + if v == 3 { + return internal_err!("Error on 3"); + } + Ok(v) + }); + let (_, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 10, res, false); + wait_for_buffering().await; + + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_err_msg(&mut buffered).await?; + + Ok(()) + } + + #[tokio::test] + async fn memory_gets_released_if_stream_drops() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (pool, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 10, res, false); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 4); + assert_eq!(pool.reserved(), 10); + + pull_ok_msg(&mut buffered).await?; + assert_eq!(buffered.messages_queued(), 3); + assert_eq!(pool.reserved(), 9); + + pull_ok_msg(&mut buffered).await?; + assert_eq!(buffered.messages_queued(), 2); + assert_eq!(pool.reserved(), 7); + + drop(buffered); + assert_eq!(pool.reserved(), 0); + Ok(()) + } + + fn memory_pool_and_reservation() -> (Arc, MemoryReservation) { + let pool = Arc::new(UnboundedMemoryPool::default()) as _; + let reservation = MemoryConsumer::new("test").register(&pool); + (pool, reservation) + } + + fn bounded_memory_pool_and_reservation( + size: usize, + ) -> (Arc, MemoryReservation) { + let pool = Arc::new(GreedyMemoryPool::new(size)) as _; + let reservation = MemoryConsumer::new("test").register(&pool); + (pool, reservation) + } + + async fn wait_for_buffering() { + // We do not have control over the spawned task, so the best we can do is to yield some + // cycles to the tokio runtime and let the task make progress on its own. + tokio::time::sleep(Duration::from_millis(1)).await; + } + + async fn pull_ok_msg( + buffered: &mut MemoryBufferedStream, + ) -> Result> { + Ok(timeout(Duration::from_millis(1), buffered.next()) + .await? + .unwrap_or_else(|| internal_err!("Stream should not have finished"))?) + } + + async fn pull_err_msg( + buffered: &mut MemoryBufferedStream, + ) -> Result> { + Ok(timeout(Duration::from_millis(1), buffered.next()) + .await? + .map(|v| match v { + Ok(v) => internal_err!( + "Stream should not have failed, but succeeded with {v:?}" + ), + Err(err) => Ok(err), + }) + .unwrap_or_else(|| internal_err!("Stream should not have finished"))?) + } + + async fn finished( + buffered: &mut MemoryBufferedStream, + ) -> Result<(), Box> { + match timeout(Duration::from_millis(1), buffered.next()) + .await? + .is_none() + { + true => Ok(()), + false => internal_err!("Stream should have finished")?, + } + } + + impl SizedMessage for usize { + fn size(&self) -> usize { + *self + } + } +} diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 52f4829127651..9166a69deaf93 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -60,6 +60,7 @@ use datafusion_physical_expr_common::sort_expr::{ LexOrdering, OrderingRequirements, PhysicalSortExpr, }; +use datafusion_physical_expr::expressions::DynamicFilterPhysicalExpr; use futures::stream::{StreamExt, TryStreamExt}; /// Represent nodes in the DataFusion Physical Plan. @@ -722,6 +723,10 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { ) -> Option> { None } + + fn dynamic_filters(&self) -> Vec> { + vec![] + } } /// [`ExecutionPlan`] Invariant Level diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4f32b6176ec39..7ada14be66543 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -206,7 +206,7 @@ async fn load_left_input( let (batches, _metrics, reservation) = stream .try_fold( (Vec::new(), metrics, reservation), - |(mut batches, metrics, mut reservation), batch| async { + |(mut batches, metrics, reservation), batch| async { let batch_size = batch.get_array_memory_size(); // Reserve memory for incoming batch reservation.try_grow(batch_size)?; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 44637321a7e35..b57f9132253bf 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -682,10 +682,10 @@ async fn collect_left_input( let schema = stream.schema(); // Load all batches and count the rows - let (batches, metrics, mut reservation) = stream + let (batches, metrics, reservation) = stream .try_fold( (Vec::new(), join_metrics, reservation), - |(mut batches, metrics, mut reservation), batch| async { + |(mut batches, metrics, reservation), batch| async { let batch_size = batch.get_array_memory_size(); // Reserve memory for incoming batch reservation.try_grow(batch_size)?; diff --git a/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs b/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs index 508be2e3984f4..d7ece845e943c 100644 --- a/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs +++ b/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs @@ -620,7 +620,7 @@ async fn build_buffered_data( // Combine batches and record number of rows let initial = (Vec::new(), 0, metrics, reservation); - let (batches, num_rows, metrics, mut reservation) = buffered + let (batches, num_rows, metrics, reservation) = buffered .try_fold(initial, |mut acc, batch| async { let batch_size = get_record_batch_memory_size(&batch); acc.3.try_grow(batch_size)?; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index ec8e154caec91..ba870ba65cb92 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -65,6 +65,7 @@ mod visitor; pub mod aggregates; pub mod analyze; pub mod async_func; +pub mod buffer; pub mod coalesce; pub mod coalesce_batches; pub mod coalesce_partitions; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 3e8fdf1f3ed7e..a8361f7b2941e 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -709,7 +709,7 @@ impl ExternalSorter { &self, batch: RecordBatch, metrics: &BaselineMetrics, - mut reservation: MemoryReservation, + reservation: MemoryReservation, ) -> Result { assert_eq!( get_reserved_bytes_for_record_batch(&batch)?, @@ -736,7 +736,7 @@ impl ExternalSorter { .then({ move |batches| async move { match batches { - Ok((schema, sorted_batches, mut reservation)) => { + Ok((schema, sorted_batches, reservation)) => { // Calculate the total size of sorted batches let total_sorted_size: usize = sorted_batches .iter() diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index a510f44e4f4df..779511a865b6a 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -180,7 +180,7 @@ impl RowCursorStream { self.rows.save(stream_idx, &rows); // track the memory in the newly created Rows. - let mut rows_reservation = self.reservation.new_empty(); + let rows_reservation = self.reservation.new_empty(); rows_reservation.try_grow(rows.size())?; Ok(RowValues::new(rows, rows_reservation)) } @@ -246,7 +246,7 @@ impl FieldCursorStream { let array = value.into_array(batch.num_rows())?; let size_in_mem = array.get_buffer_memory_size(); let array = array.as_any().downcast_ref::().expect("field values"); - let mut array_reservation = self.reservation.new_empty(); + let array_reservation = self.reservation.new_empty(); array_reservation.try_grow(size_in_mem)?; Ok(ArrayValues::new( self.sort.options, diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 80c2233d05db6..4b7e707fccedd 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -1005,7 +1005,7 @@ mod test { .build_arc() .unwrap(); - let mut reservation = MemoryConsumer::new("test").register(&runtime.memory_pool); + let reservation = MemoryConsumer::new("test").register(&runtime.memory_pool); let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); @@ -1071,7 +1071,7 @@ mod test { .build_arc() .unwrap(); - let mut reservation = MemoryConsumer::new("test").register(&runtime.memory_pool); + let reservation = MemoryConsumer::new("test").register(&runtime.memory_pool); let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index f1b9e3e88d123..1313909adbba2 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -283,7 +283,7 @@ mod tests { assert!(work_table.take().is_err()); let pool = Arc::new(UnboundedMemoryPool::default()) as _; - let mut reservation = MemoryConsumer::new("test_work_table").register(&pool); + let reservation = MemoryConsumer::new("test_work_table").register(&pool); // Update batch to work_table let array: ArrayRef = Arc::new((0..5).collect::()); diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 5f590560c4675..50332e23d7ee4 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -749,6 +749,7 @@ message PhysicalPlanNode { SortMergeJoinExecNode sort_merge_join = 34; MemoryScanExecNode memory_scan = 35; AsyncFuncExecNode async_func = 36; + BufferExecNode buffer = 37; } } @@ -1413,3 +1414,8 @@ message AsyncFuncExecNode { repeated PhysicalExprNode async_exprs = 2; repeated string async_expr_names = 3; } + +message BufferExecNode { + PhysicalPlanNode input = 1; + uint64 capacity = 2; +} \ No newline at end of file diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index f6d364f269b48..a6e411744ce0f 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -1838,6 +1838,118 @@ impl<'de> serde::Deserialize<'de> for BinaryExprNode { deserializer.deserialize_struct("datafusion.BinaryExprNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for BufferExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.input.is_some() { + len += 1; + } + if self.capacity != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.BufferExecNode", len)?; + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + if self.capacity != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("capacity", ToString::to_string(&self.capacity).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for BufferExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "input", + "capacity", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Input, + Capacity, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "input" => Ok(GeneratedField::Input), + "capacity" => Ok(GeneratedField::Capacity), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = BufferExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.BufferExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut input__ = None; + let mut capacity__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + GeneratedField::Capacity => { + if capacity__.is_some() { + return Err(serde::de::Error::duplicate_field("capacity")); + } + capacity__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(BufferExecNode { + input: input__, + capacity: capacity__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.BufferExecNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for CaseNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -17509,6 +17621,9 @@ impl serde::Serialize for PhysicalPlanNode { physical_plan_node::PhysicalPlanType::AsyncFunc(v) => { struct_ser.serialize_field("asyncFunc", v)?; } + physical_plan_node::PhysicalPlanType::Buffer(v) => { + struct_ser.serialize_field("buffer", v)?; + } } } struct_ser.end() @@ -17576,6 +17691,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "memoryScan", "async_func", "asyncFunc", + "buffer", ]; #[allow(clippy::enum_variant_names)] @@ -17615,6 +17731,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { SortMergeJoin, MemoryScan, AsyncFunc, + Buffer, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -17671,6 +17788,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "sortMergeJoin" | "sort_merge_join" => Ok(GeneratedField::SortMergeJoin), "memoryScan" | "memory_scan" => Ok(GeneratedField::MemoryScan), "asyncFunc" | "async_func" => Ok(GeneratedField::AsyncFunc), + "buffer" => Ok(GeneratedField::Buffer), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -17936,6 +18054,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { return Err(serde::de::Error::duplicate_field("asyncFunc")); } physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::AsyncFunc) +; + } + GeneratedField::Buffer => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("buffer")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Buffer) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index c1afd73ec3c52..bdd1e5eeb989f 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1076,7 +1076,7 @@ pub mod table_reference { pub struct PhysicalPlanNode { #[prost( oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37" )] pub physical_plan_type: ::core::option::Option, } @@ -1156,6 +1156,8 @@ pub mod physical_plan_node { MemoryScan(super::MemoryScanExecNode), #[prost(message, tag = "36")] AsyncFunc(::prost::alloc::boxed::Box), + #[prost(message, tag = "37")] + Buffer(::prost::alloc::boxed::Box), } } #[derive(Clone, PartialEq, ::prost::Message)] @@ -2136,6 +2138,13 @@ pub struct AsyncFuncExecNode { #[prost(string, repeated, tag = "3")] pub async_expr_names: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, } +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct BufferExecNode { + #[prost(message, optional, boxed, tag = "1")] + pub input: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(uint64, tag = "2")] + pub capacity: u64, +} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum WindowFrameUnits { diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 45868df4ced6c..5c1a70b7e4dec 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -103,6 +103,7 @@ use datafusion_physical_plan::{ExecutionPlan, InputOrderMode, PhysicalExpr, Wind use datafusion_physical_expr::async_scalar_function::AsyncFuncExpr; use datafusion_physical_plan::async_func::AsyncFuncExec; +use datafusion_physical_plan::buffer::BufferExec; use prost::Message; use prost::bytes::BufMut; @@ -255,6 +256,9 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { PhysicalPlanType::AsyncFunc(async_func) => { self.try_into_async_func_physical_plan(async_func, ctx, extension_codec) } + PhysicalPlanType::Buffer(buffer) => { + self.try_into_buffer_physical_plan(buffer, ctx, extension_codec) + } } } @@ -473,6 +477,13 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ); } + if let Some(exec) = plan.downcast_ref::() { + return protobuf::PhysicalPlanNode::try_from_buffer_exec( + exec, + extension_codec, + ); + } + let mut buf: Vec = vec![]; match extension_codec.try_encode(Arc::clone(&plan_clone), &mut buf) { Ok(_) => { @@ -2028,6 +2039,18 @@ impl protobuf::PhysicalPlanNode { Ok(Arc::new(AsyncFuncExec::try_new(async_exprs, input)?)) } + fn try_into_buffer_physical_plan( + &self, + buffer: &protobuf::BufferExecNode, + ctx: &TaskContext, + extension_codec: &dyn PhysicalExtensionCodec, + ) -> Result> { + let input: Arc = + into_physical_plan(&buffer.input, ctx, extension_codec)?; + + Ok(Arc::new(BufferExec::new(input, buffer.capacity as usize))) + } + fn try_from_explain_exec( exec: &ExplainExec, _extension_codec: &dyn PhysicalExtensionCodec, @@ -3309,6 +3332,25 @@ impl protobuf::PhysicalPlanNode { ))), }) } + + fn try_from_buffer_exec( + exec: &BufferExec, + extension_codec: &dyn PhysicalExtensionCodec, + ) -> Result { + let input = protobuf::PhysicalPlanNode::try_from_physical_plan( + Arc::clone(exec.input()), + extension_codec, + )?; + + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Buffer(Box::new( + protobuf::BufferExecNode { + input: Some(Box::new(input)), + capacity: exec.capacity() as u64, + }, + ))), + }) + } } pub trait AsExecutionPlan: Debug + Send + Sync + Clone { diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 3dac92938772c..3fc0409098379 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -960,7 +960,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(id@0, id@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; @@ -1150,8 +1151,9 @@ physical_plan 04)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(end@1, start@0)], projection=[start@0, end@3] 05)------RepartitionExec: partitioning=Hash([end@1], 4), input_partitions=1 06)--------WorkTableExec: name=trans -07)------RepartitionExec: partitioning=Hash([start@0], 4), input_partitions=1 -08)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/closure.csv]]}, projection=[start, end], file_type=csv, has_header=true +07)------BufferExec: capacity=1048576 +08)--------RepartitionExec: partitioning=Hash([start@0], 4), input_partitions=1 +09)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/closure.csv]]}, projection=[start, end], file_type=csv, has_header=true statement count 0 set datafusion.execution.enable_recursive_ctes = false; diff --git a/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt b/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt index bc796a51ff5a4..8e85c8f90580e 100644 --- a/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt +++ b/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt @@ -113,4 +113,3 @@ SELECT '2001-09-28'::date / '03:00'::time query error Invalid timestamp arithmetic operation SELECT '2001-09-28'::date % '03:00'::time - diff --git a/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt index 10381346f8359..aeeebe73db701 100644 --- a/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt +++ b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt @@ -144,4 +144,4 @@ query error Invalid timestamp arithmetic operation SELECT '2001-09-28T01:00:00'::timestamp % arrow_cast(12345, 'Duration(Second)'); query error Invalid timestamp arithmetic operation -SELECT '2001-09-28T01:00:00'::timestamp / arrow_cast(12345, 'Duration(Second)'); \ No newline at end of file +SELECT '2001-09-28T01:00:00'::timestamp / arrow_cast(12345, 'Duration(Second)'); diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index 3e403171e0718..5269a05791b26 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -127,7 +127,8 @@ physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Disable Join dynamic filter pushdown statement ok @@ -150,7 +151,8 @@ physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # Re-enable for next tests statement ok @@ -182,7 +184,8 @@ physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Enable TopK, disable Join statement ok @@ -208,7 +211,8 @@ physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # Test 4: Aggregate dynamic filter pushdown @@ -338,7 +342,8 @@ physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # Re-enable statement ok @@ -386,7 +391,8 @@ physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Cleanup diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 4d6df4530f801..cffaea838bac5 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -239,6 +239,7 @@ physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after OutputRequirements DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE +physical_plan after HashJoinBuffering SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after PushdownSort SAME TEXT AS ABOVE @@ -319,6 +320,7 @@ physical_plan after OutputRequirements 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE +physical_plan after HashJoinBuffering SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after PushdownSort SAME TEXT AS ABOVE @@ -363,6 +365,7 @@ physical_plan after OutputRequirements 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE +physical_plan after HashJoinBuffering SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after PushdownSort SAME TEXT AS ABOVE @@ -600,6 +603,7 @@ physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after OutputRequirements DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE +physical_plan after HashJoinBuffering SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after PushdownSort SAME TEXT AS ABOVE diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 9215ce87e3bef..2aac4b7eeace5 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -302,20 +302,26 @@ physical_plan 17)│ (int_col = int_col) │ │ 18)└─────────────┬─────────────┘ │ 19)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -20)│ DataSourceExec ││ RepartitionExec │ +20)│ DataSourceExec ││ BufferExec │ 21)│ -------------------- ││ -------------------- │ -22)│ files: 1 ││ partition_count(in->out): │ -23)│ format: parquet ││ 1 -> 4 │ -24)│ ││ │ -25)│ ││ partitioning_scheme: │ -26)│ ││ RoundRobinBatch(4) │ -27)└───────────────────────────┘└─────────────┬─────────────┘ -28)-----------------------------┌─────────────┴─────────────┐ -29)-----------------------------│ DataSourceExec │ -30)-----------------------------│ -------------------- │ -31)-----------------------------│ files: 1 │ -32)-----------------------------│ format: csv │ -33)-----------------------------└───────────────────────────┘ +22)│ files: 1 ││ target_batch_size: │ +23)│ format: parquet ││ 1048576 │ +24)└───────────────────────────┘└─────────────┬─────────────┘ +25)-----------------------------┌─────────────┴─────────────┐ +26)-----------------------------│ RepartitionExec │ +27)-----------------------------│ -------------------- │ +28)-----------------------------│ partition_count(in->out): │ +29)-----------------------------│ 1 -> 4 │ +30)-----------------------------│ │ +31)-----------------------------│ partitioning_scheme: │ +32)-----------------------------│ RoundRobinBatch(4) │ +33)-----------------------------└─────────────┬─────────────┘ +34)-----------------------------┌─────────────┴─────────────┐ +35)-----------------------------│ DataSourceExec │ +36)-----------------------------│ -------------------- │ +37)-----------------------------│ files: 1 │ +38)-----------------------------│ format: csv │ +39)-----------------------------└───────────────────────────┘ # 3 Joins query TT @@ -343,35 +349,51 @@ physical_plan 13)│ (int_col = int_col) │ │ 14)└─────────────┬─────────────┘ │ 15)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -16)│ DataSourceExec ││ ProjectionExec │ +16)│ DataSourceExec ││ BufferExec │ 17)│ -------------------- ││ -------------------- │ -18)│ bytes: 520 ││ date_col: date_col │ -19)│ format: memory ││ int_col: int_col │ +18)│ bytes: 520 ││ target_batch_size: │ +19)│ format: memory ││ 1048576 │ 20)│ rows: 1 ││ │ -21)│ ││ string_col: │ -22)│ ││ string_col │ -23)└───────────────────────────┘└─────────────┬─────────────┘ -24)-----------------------------┌─────────────┴─────────────┐ -25)-----------------------------│ HashJoinExec │ -26)-----------------------------│ -------------------- │ -27)-----------------------------│ on: ├──────────────┐ -28)-----------------------------│ (int_col = int_col) │ │ -29)-----------------------------└─────────────┬─────────────┘ │ -30)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -31)-----------------------------│ DataSourceExec ││ RepartitionExec │ -32)-----------------------------│ -------------------- ││ -------------------- │ -33)-----------------------------│ files: 1 ││ partition_count(in->out): │ -34)-----------------------------│ format: parquet ││ 1 -> 4 │ -35)-----------------------------│ ││ │ -36)-----------------------------│ predicate: ││ partitioning_scheme: │ -37)-----------------------------│ DynamicFilter [ empty ] ││ RoundRobinBatch(4) │ -38)-----------------------------└───────────────────────────┘└─────────────┬─────────────┘ -39)----------------------------------------------------------┌─────────────┴─────────────┐ -40)----------------------------------------------------------│ DataSourceExec │ -41)----------------------------------------------------------│ -------------------- │ -42)----------------------------------------------------------│ files: 1 │ -43)----------------------------------------------------------│ format: csv │ -44)----------------------------------------------------------└───────────────────────────┘ +21)└───────────────────────────┘└─────────────┬─────────────┘ +22)-----------------------------┌─────────────┴─────────────┐ +23)-----------------------------│ ProjectionExec │ +24)-----------------------------│ -------------------- │ +25)-----------------------------│ date_col: date_col │ +26)-----------------------------│ int_col: int_col │ +27)-----------------------------│ │ +28)-----------------------------│ string_col: │ +29)-----------------------------│ string_col │ +30)-----------------------------└─────────────┬─────────────┘ +31)-----------------------------┌─────────────┴─────────────┐ +32)-----------------------------│ HashJoinExec │ +33)-----------------------------│ -------------------- │ +34)-----------------------------│ on: ├──────────────┐ +35)-----------------------------│ (int_col = int_col) │ │ +36)-----------------------------└─────────────┬─────────────┘ │ +37)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +38)-----------------------------│ DataSourceExec ││ BufferExec │ +39)-----------------------------│ -------------------- ││ -------------------- │ +40)-----------------------------│ files: 1 ││ target_batch_size: │ +41)-----------------------------│ format: parquet ││ 1048576 │ +42)-----------------------------│ ││ │ +43)-----------------------------│ predicate: ││ │ +44)-----------------------------│ DynamicFilter [ empty ] ││ │ +45)-----------------------------└───────────────────────────┘└─────────────┬─────────────┘ +46)----------------------------------------------------------┌─────────────┴─────────────┐ +47)----------------------------------------------------------│ RepartitionExec │ +48)----------------------------------------------------------│ -------------------- │ +49)----------------------------------------------------------│ partition_count(in->out): │ +50)----------------------------------------------------------│ 1 -> 4 │ +51)----------------------------------------------------------│ │ +52)----------------------------------------------------------│ partitioning_scheme: │ +53)----------------------------------------------------------│ RoundRobinBatch(4) │ +54)----------------------------------------------------------└─────────────┬─────────────┘ +55)----------------------------------------------------------┌─────────────┴─────────────┐ +56)----------------------------------------------------------│ DataSourceExec │ +57)----------------------------------------------------------│ -------------------- │ +58)----------------------------------------------------------│ files: 1 │ +59)----------------------------------------------------------│ format: csv │ +60)----------------------------------------------------------└───────────────────────────┘ # Long Filter (demonstrate what happens with wrapping) query TT @@ -1095,20 +1117,26 @@ physical_plan 19)│ string_col) │ │ 20)└─────────────┬─────────────┘ │ 21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -22)│ DataSourceExec ││ RepartitionExec │ +22)│ DataSourceExec ││ BufferExec │ 23)│ -------------------- ││ -------------------- │ -24)│ files: 1 ││ partition_count(in->out): │ -25)│ format: parquet ││ 1 -> 4 │ -26)│ ││ │ -27)│ ││ partitioning_scheme: │ -28)│ ││ RoundRobinBatch(4) │ -29)└───────────────────────────┘└─────────────┬─────────────┘ -30)-----------------------------┌─────────────┴─────────────┐ -31)-----------------------------│ DataSourceExec │ -32)-----------------------------│ -------------------- │ -33)-----------------------------│ files: 1 │ -34)-----------------------------│ format: csv │ -35)-----------------------------└───────────────────────────┘ +24)│ files: 1 ││ target_batch_size: │ +25)│ format: parquet ││ 1048576 │ +26)└───────────────────────────┘└─────────────┬─────────────┘ +27)-----------------------------┌─────────────┴─────────────┐ +28)-----------------------------│ RepartitionExec │ +29)-----------------------------│ -------------------- │ +30)-----------------------------│ partition_count(in->out): │ +31)-----------------------------│ 1 -> 4 │ +32)-----------------------------│ │ +33)-----------------------------│ partitioning_scheme: │ +34)-----------------------------│ RoundRobinBatch(4) │ +35)-----------------------------└─────────────┬─────────────┘ +36)-----------------------------┌─────────────┴─────────────┐ +37)-----------------------------│ DataSourceExec │ +38)-----------------------------│ -------------------- │ +39)-----------------------------│ files: 1 │ +40)-----------------------------│ format: csv │ +41)-----------------------------└───────────────────────────┘ # Query with outer hash join. query TT @@ -1138,20 +1166,26 @@ physical_plan 21)│ string_col) │ │ 22)└─────────────┬─────────────┘ │ 23)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -24)│ DataSourceExec ││ RepartitionExec │ +24)│ DataSourceExec ││ BufferExec │ 25)│ -------------------- ││ -------------------- │ -26)│ files: 1 ││ partition_count(in->out): │ -27)│ format: parquet ││ 1 -> 4 │ -28)│ ││ │ -29)│ ││ partitioning_scheme: │ -30)│ ││ RoundRobinBatch(4) │ -31)└───────────────────────────┘└─────────────┬─────────────┘ -32)-----------------------------┌─────────────┴─────────────┐ -33)-----------------------------│ DataSourceExec │ -34)-----------------------------│ -------------------- │ -35)-----------------------------│ files: 1 │ -36)-----------------------------│ format: csv │ -37)-----------------------------└───────────────────────────┘ +26)│ files: 1 ││ target_batch_size: │ +27)│ format: parquet ││ 1048576 │ +28)└───────────────────────────┘└─────────────┬─────────────┘ +29)-----------------------------┌─────────────┴─────────────┐ +30)-----------------------------│ RepartitionExec │ +31)-----------------------------│ -------------------- │ +32)-----------------------------│ partition_count(in->out): │ +33)-----------------------------│ 1 -> 4 │ +34)-----------------------------│ │ +35)-----------------------------│ partitioning_scheme: │ +36)-----------------------------│ RoundRobinBatch(4) │ +37)-----------------------------└─────────────┬─────────────┘ +38)-----------------------------┌─────────────┴─────────────┐ +39)-----------------------------│ DataSourceExec │ +40)-----------------------------│ -------------------- │ +41)-----------------------------│ files: 1 │ +42)-----------------------------│ format: csv │ +43)-----------------------------└───────────────────────────┘ # Query with nested loop join. query TT diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index cd1ed2bc0caca..864f0527f3c64 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2023,7 +2023,8 @@ physical_plan 08)--------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] 09)----------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)] 10)------------------DataSourceExec: partitions=1, partition_sizes=[3] -11)------------------DataSourceExec: partitions=1, partition_sizes=[3] +11)------------------BufferExec: capacity=1048576 +12)--------------------DataSourceExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is DataSourceExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2944,7 +2945,8 @@ physical_plan 04)------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----------BufferExec: capacity=1048576 +08)------------DataSourceExec: partitions=1, partition_sizes=[1] query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -3928,9 +3930,10 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true -05)------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -06)--------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +05)------BufferExec: capacity=1048576 +06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +07)----------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +08)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # reset partition number to 8. statement ok @@ -4083,9 +4086,10 @@ physical_plan 03)----ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] 04)------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true -06)----ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] -07)------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -08)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +06)----BufferExec: capacity=1048576 +07)------ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] +08)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +09)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -5146,7 +5150,8 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }, ts@0, 946684800000000000) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))], aggr=[count(keywords_stream.keyword)] 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)] 04)------DataSourceExec: partitions=1, partition_sizes=[3] -05)------DataSourceExec: partitions=1, partition_sizes=[3] +05)------BufferExec: capacity=1048576 +06)--------DataSourceExec: partitions=1, partition_sizes=[3] query PI SELECT diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 2039ee93df837..6aada7d4e80e1 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -220,6 +220,7 @@ datafusion.execution.collect_statistics true datafusion.execution.enable_ansi_mode false datafusion.execution.enable_recursive_ctes true datafusion.execution.enforce_batch_size_in_joins false +datafusion.execution.hash_join_buffering_capacity 1048576 datafusion.execution.keep_partition_by_columns false datafusion.execution.listing_table_factory_infer_partitions true datafusion.execution.listing_table_ignore_subdirectory true @@ -357,6 +358,7 @@ datafusion.execution.collect_statistics true Should DataFusion collect statistic datafusion.execution.enable_ansi_mode false Whether to enable ANSI SQL mode. The flag is experimental and relevant only for DataFusion Spark built-in functions When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL semantics for expressions, casting, and error handling. This means: - **Strict type coercion rules:** implicit casts between incompatible types are disallowed. - **Standard SQL arithmetic behavior:** operations such as division by zero, numeric overflow, or invalid casts raise runtime errors rather than returning `NULL` or adjusted values. - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling. When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive, non-ANSI mode designed for user convenience and backward compatibility. In this mode: - Implicit casts between types are allowed (e.g., string to integer when possible). - Arithmetic operations are more lenient — for example, `abs()` on the minimum representable integer value returns the input value instead of raising overflow. - Division by zero or invalid casts may return `NULL` instead of failing. # Default `false` — ANSI SQL mode is disabled by default. datafusion.execution.enable_recursive_ctes true Should DataFusion support recursive CTEs datafusion.execution.enforce_batch_size_in_joins false Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. +datafusion.execution.hash_join_buffering_capacity 1048576 How many bytes to buffer in the probe side of hash joins while the build side is concurrently being built. Without this, hash joins will wait until the full materialization of the build side before polling the probe side. This is useful in scenarios where the query is not completely CPU bounded, allowing to do some early work concurrently and reducing the latency of the query. 1Mb by default. Set to 0 for disabling it. datafusion.execution.keep_partition_by_columns false Should DataFusion keep the columns used for partition_by in the output RecordBatches datafusion.execution.listing_table_factory_infer_partitions true Should a `ListingTable` created through the `ListingTableFactory` infer table partitions from Hive compliant directories. Defaults to true (partition columns are inferred and will be represented in the table schema). datafusion.execution.listing_table_ignore_subdirectory true Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index c0a838c97d552..cceacb8a93337 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -778,7 +778,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[1] # Reset the configs to old values statement ok @@ -987,7 +988,8 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] 04)------FilterExec: name@1 = Alice OR name@1 = Carol 05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------DataSourceExec: partitions=1, partition_sizes=[1] +06)------BufferExec: capacity=1048576 +07)--------DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -1181,8 +1183,10 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0), (v1@1, v1@1)], projection=[v0@0, v1@1, v0@2, v2@4, v3@5, v4@6] 06)----------DataSourceExec: partitions=1, partition_sizes=[0] -07)----------DataSourceExec: partitions=1, partition_sizes=[0] -08)--DataSourceExec: partitions=1, partition_sizes=[0] +07)----------BufferExec: capacity=1048576 +08)------------DataSourceExec: partitions=1, partition_sizes=[0] +09)--BufferExec: capacity=1048576 +10)----DataSourceExec: partitions=1, partition_sizes=[0] @@ -1371,14 +1375,16 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(s.b AS Int64)@1, col1@1)], projection=[b@0, col0@2, col1@3, a@4] 03)----ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] 04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] -06)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] -07)--------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] -08)----------DataSourceExec: partitions=1, partition_sizes=[1] -09)--------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] -10)----------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 -11)------------FilterExec: y@1 = x@0 -12)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)----BufferExec: capacity=1048576 +06)------ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] +07)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] +08)----------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] +09)------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----------BufferExec: capacity=1048576 +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)----------------FilterExec: y@1 = x@0 +14)------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table pairs; @@ -1429,7 +1435,8 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)----DataSourceExec: partitions=1, partition_sizes=[1] +09)----BufferExec: capacity=1048576 +10)------DataSourceExec: partitions=1, partition_sizes=[1] query III SELECT * @@ -1451,7 +1458,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[1] query IIII SELECT * FROM t0, LATERAL (SELECT * FROM t1 WHERE t0.v0 = t1.v0); diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index c16b3528aa7a5..9cb02c1caf2c6 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -57,8 +57,9 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], file_type=csv, has_header=true -04)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1, maintains_sort_order=true -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +04)----BufferExec: capacity=1048576 +05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1, maintains_sort_order=true +06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # preserve_inner_join query IIII nosort @@ -97,9 +98,10 @@ physical_plan 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true -05)------FilterExec: d@3 = 3 -06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1, maintains_sort_order=true -07)----------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 +05)------BufferExec: capacity=1048576 +06)--------FilterExec: d@3 = 3 +07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1, maintains_sort_order=true +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 8246f489c446d..7ee71cb235559 100644 --- a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt +++ b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt @@ -82,7 +82,8 @@ physical_plan 01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@1, val@1)], NullsEqual: true 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.format = "tree"; @@ -109,12 +110,19 @@ physical_plan 13)│ on: (val = val) │ │ 14)└─────────────┬─────────────┘ │ 15)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -16)│ DataSourceExec ││ DataSourceExec │ +16)│ DataSourceExec ││ BufferExec │ 17)│ -------------------- ││ -------------------- │ -18)│ bytes: 288 ││ bytes: 288 │ -19)│ format: memory ││ format: memory │ -20)│ rows: 1 ││ rows: 1 │ -21)└───────────────────────────┘└───────────────────────────┘ +18)│ bytes: 288 ││ target_batch_size: │ +19)│ format: memory ││ 1048576 │ +20)│ rows: 1 ││ │ +21)└───────────────────────────┘└─────────────┬─────────────┘ +22)-----------------------------┌─────────────┴─────────────┐ +23)-----------------------------│ DataSourceExec │ +24)-----------------------------│ -------------------- │ +25)-----------------------------│ bytes: 288 │ +26)-----------------------------│ format: memory │ +27)-----------------------------│ rows: 1 │ +28)-----------------------------└───────────────────────────┘ statement ok set datafusion.explain.format = "indent"; @@ -147,8 +155,9 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------FilterExec: CAST(val@1 AS Int64) + 1 IS NOT DISTINCT FROM 11 07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)----ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] -09)------DataSourceExec: partitions=1, partition_sizes=[1] +08)----BufferExec: capacity=1048576 +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 @@ -173,8 +182,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], filter=CAST(val@0 AS Int64) % 3 IS DISTINCT FROM CAST(val@1 AS Int64) % 3, projection=[id@0, val@1, id@3, val@4], NullsEqual: true 03)----ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t1.val + Int64(1)] 04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] -06)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----BufferExec: capacity=1048576 +06)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] +07)--------DataSourceExec: partitions=1, partition_sizes=[1] # Test mixed equal and IS NOT DISTINCT FROM conditions # The `IS NOT DISTINCT FROM` expr should NOT in HashJoin's `on` predicate @@ -192,7 +202,8 @@ physical_plan 01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], filter=val@0 IS NOT DISTINCT FROM val@1 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[1] # Test the mixed condition join result query IIII rowsort @@ -232,10 +243,12 @@ physical_plan 01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@0, val@1)], projection=[id@1, val@2, id@3, val@4], NullsEqual: true 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@1, val@1)], NullsEqual: true -06)--------DataSourceExec: partitions=1, partition_sizes=[1] -07)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@1, val@1)], NullsEqual: true +07)----------DataSourceExec: partitions=1, partition_sizes=[1] +08)----------BufferExec: capacity=1048576 +09)------------DataSourceExec: partitions=1, partition_sizes=[1] # Test IS NOT DISTINCT FROM with multiple columns statement ok @@ -276,8 +289,9 @@ JOIN t4 ON (t3.val1 IS NOT DISTINCT FROM t4.val1) AND (t3.val2 IS NOT DISTINCT F 02)--Inner Join: t3.val1 = t4.val1, t3.val2 = t4.val2 03)----DataSourceExec: partitions=1, partition_sizes=[1] 03)----TableScan: t3 projection=[id, val1, val2] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 04)----TableScan: t4 projection=[id, val1, val2] +05)------DataSourceExec: partitions=1, partition_sizes=[1] logical_plan physical_plan diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index ae87fd11d397c..a570ab9517032 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1343,8 +1343,9 @@ physical_plan 03)----AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------BufferExec: capacity=1048576 +07)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)------------DataSourceExec: partitions=1, partition_sizes=[1] # Join on struct query TT @@ -1359,8 +1360,9 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s3@0, s4@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)----DataSourceExec: partitions=1, partition_sizes=[1] +03)--BufferExec: capacity=1048576 +04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1] query ?? select join_t3.s3, join_t4.s4 @@ -1398,8 +1400,9 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)----------BufferExec: capacity=1048576 +08)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1425,8 +1428,9 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] 08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----------------BufferExec: capacity=1048576 +11)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1490,9 +1494,10 @@ physical_plan 04)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)----BufferExec: capacity=1048576 +08)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1515,9 +1520,10 @@ physical_plan 04)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)----BufferExec: capacity=1048576 +08)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------DataSourceExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1542,9 +1548,10 @@ physical_plan 04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)----BufferExec: capacity=1048576 +08)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1567,9 +1574,10 @@ physical_plan 04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)----BufferExec: capacity=1048576 +08)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------DataSourceExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1592,9 +1600,10 @@ physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1615,9 +1624,10 @@ physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1643,8 +1653,9 @@ physical_plan 04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)------DataSourceExec: partitions=1, partition_sizes=[1] +07)----BufferExec: capacity=1048576 +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1668,8 +1679,9 @@ physical_plan 04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)------DataSourceExec: partitions=1, partition_sizes=[1] +07)----BufferExec: capacity=1048576 +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1690,9 +1702,10 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)------DataSourceExec: partitions=1, partition_sizes=[1] +03)--BufferExec: capacity=1048576 +04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1711,9 +1724,10 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)------DataSourceExec: partitions=1, partition_sizes=[1] +03)--BufferExec: capacity=1048576 +04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -2552,8 +2566,9 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(millis@2, millis@2)] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)----DataSourceExec: partitions=1, partition_sizes=[1] +03)--BufferExec: capacity=1048576 +04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1] # left_join_using_2 query II @@ -2810,9 +2825,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2845,9 +2861,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -2901,9 +2918,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2936,9 +2954,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -2993,9 +3012,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3009,9 +3029,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3063,9 +3084,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3079,9 +3101,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)----BufferExec: capacity=1048576 +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3264,9 +3287,10 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] 02)--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, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true -03)--ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -04)----BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -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, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +03)--BufferExec: capacity=1048576 +04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +05)------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +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, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # hash join should propagate ordering equivalence of the right side for RIGHT ANTI join. # Hence final requirement rn1 ASC is already satisfied at the end of HashJoinExec. @@ -3290,9 +3314,10 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], file_type=csv, has_header=true -03)--ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -04)----BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -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, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +03)--BufferExec: capacity=1048576 +04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +05)------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +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, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # Test ordering preservation for RIGHT join query TT @@ -3312,7 +3337,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Right, on=[(b@2, b@2)] 02)--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, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true -03)--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, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +03)--BufferExec: capacity=1048576 +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, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 @@ -3336,7 +3362,8 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true +05)------BufferExec: capacity=1048576 +06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true # create a table where there more than one valid ordering # that describes table. @@ -3383,9 +3410,10 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true -05)------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -06)--------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +05)------BufferExec: capacity=1048576 +06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +07)----------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +08)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # run query above in multiple partitions statement ok @@ -3421,8 +3449,9 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] 07)------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1, maintains_sort_order=true 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true -09)------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1, maintains_sort_order=true -10)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true +09)------------BufferExec: capacity=1048576 +10)--------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1, maintains_sort_order=true +11)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT * @@ -3576,11 +3605,12 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 03)----ProjectionExec: expr=[1 as e, 3 as f] 04)------PlaceholderRowExec -05)----UnionExec -06)------ProjectionExec: expr=[1 as c, 2 as d] -07)--------PlaceholderRowExec -08)------ProjectionExec: expr=[1 as c, 3 as d] -09)--------PlaceholderRowExec +05)----BufferExec: capacity=1048576 +06)------UnionExec +07)--------ProjectionExec: expr=[1 as c, 2 as d] +08)----------PlaceholderRowExec +09)--------ProjectionExec: expr=[1 as c, 3 as d] +10)----------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3618,11 +3648,12 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] 03)----ProjectionExec: expr=[1 as e, 3 as f] 04)------PlaceholderRowExec -05)----UnionExec -06)------ProjectionExec: expr=[1 as c, 2 as d] -07)--------PlaceholderRowExec -08)------ProjectionExec: expr=[1 as c, 3 as d] -09)--------PlaceholderRowExec +05)----BufferExec: capacity=1048576 +06)------UnionExec +07)--------ProjectionExec: expr=[1 as c, 2 as d] +08)----------PlaceholderRowExec +09)--------ProjectionExec: expr=[1 as c, 3 as d] +10)----------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3830,7 +3861,8 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] 03)----SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] 04)------DataSourceExec: partitions=1, partition_sizes=[2] -05)----DataSourceExec: partitions=1, partition_sizes=[2] +05)----BufferExec: capacity=1048576 +06)------DataSourceExec: partitions=1, partition_sizes=[2] @@ -3886,7 +3918,8 @@ physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] 03)----DataSourceExec: partitions=1, partition_sizes=[2] -04)----DataSourceExec: partitions=1, partition_sizes=[2] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[2] # Null build indices: @@ -3946,7 +3979,8 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] 03)----SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] 04)------DataSourceExec: partitions=1, partition_sizes=[2] -05)----DataSourceExec: partitions=1, partition_sizes=[2] +05)----BufferExec: capacity=1048576 +06)------DataSourceExec: partitions=1, partition_sizes=[2] # Test CROSS JOIN LATERAL syntax (planning) @@ -4051,8 +4085,9 @@ physical_plan 03)----AggregateExec: mode=Single, gby=[sn@1 as sn, ts@0 as ts, amount@2 as amount, currency@3 as currency], aggr=[last_value(e.rate)] 04)------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] 05)--------DataSourceExec: partitions=1, partition_sizes=[0] -06)--------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] -07)----------DataSourceExec: partitions=1, partition_sizes=[0] +06)--------BufferExec: capacity=1048576 +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; @@ -4093,8 +4128,9 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, y@1)], filter=a@0 < x@1 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +03)--BufferExec: capacity=1048576 +04)----SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] +05)------DataSourceExec: partitions=1, partition_sizes=[0] # Test full join with limit statement ok @@ -4164,7 +4200,8 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[2] -04)----DataSourceExec: partitions=1, partition_sizes=[2] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[2] ## Test join.on.is_empty() && join.filter.is_some() -> single filter now a PWMJ query TT @@ -4194,7 +4231,8 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)], filter=c2@0 >= c2@1 03)----DataSourceExec: partitions=1, partition_sizes=[2] -04)----DataSourceExec: partitions=1, partition_sizes=[2] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[2] ## Add more test cases for join limit pushdown statement ok @@ -4258,7 +4296,8 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, b@0)] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], file_type=csv, has_header=true +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], file_type=csv, has_header=true ###### ## RIGHT JOIN w/ LIMIT @@ -4292,7 +4331,8 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@0, b@0)] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true ###### ## FULL JOIN w/ LIMIT @@ -4329,7 +4369,8 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(a@0, b@0)] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], file_type=csv, has_header=true +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], file_type=csv, has_header=true statement ok drop table t1; @@ -4370,7 +4411,8 @@ physical_plan 03)----ProjectionExec: expr=[] 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(binary_col@0, binary_col@0)] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------BufferExec: capacity=1048576 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] # Test hash join sort push down # Issue: https://github.com/apache/datafusion/issues/13559 @@ -4400,8 +4442,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------FilterExec: b@1 > 3, projection=[a@0] 05)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] -06)----SortExec: expr=[c@2 DESC], preserve_partitioning=[true] -07)------DataSourceExec: partitions=2, partition_sizes=[1, 1] +06)----BufferExec: capacity=1048576 +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; @@ -4420,8 +4463,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------FilterExec: b@1 > 3, projection=[a@0] 05)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] -06)----SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] -07)------DataSourceExec: partitions=2, partition_sizes=[1, 1] +06)----BufferExec: capacity=1048576 +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; @@ -4461,7 +4505,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2, state@5] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT age FROM (SELECT * FROM person a join person b using (id, age, state)); @@ -4476,7 +4521,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[age@1] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT a.* FROM person a join person b using (id, age); @@ -4491,7 +4537,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT a.*, b.* FROM person a join person b using (id, age); @@ -4505,7 +4552,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM person a join person b using (id, age, state) join person c using (id, age, state); @@ -4525,8 +4573,10 @@ physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] -05)--DataSourceExec: partitions=1, partition_sizes=[0] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[0] +06)--BufferExec: capacity=1048576 +07)----DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM person a NATURAL JOIN lineitem b; @@ -4554,7 +4604,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c1@0, c1@0)], projection=[c1@0] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; @@ -4690,8 +4741,9 @@ physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(person.id + Int64(10)@1, orders.customer_id * Int64(2)@1)], projection=[id@0, customer_id@2] 02)--ProjectionExec: expr=[id@0 as id, CAST(id@0 AS Int64) + 10 as person.id + Int64(10)] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)--ProjectionExec: expr=[customer_id@0 as customer_id, CAST(customer_id@0 AS Int64) * 2 as orders.customer_id * Int64(2)] -05)----DataSourceExec: partitions=1, partition_sizes=[0] +04)--BufferExec: capacity=1048576 +05)----ProjectionExec: expr=[customer_id@0 as customer_id, CAST(customer_id@0 AS Int64) * 2 as orders.customer_id * Int64(2)] +06)------DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; @@ -4780,7 +4832,8 @@ physical_plan 01)SortExec: TopK(fetch=2), expr=[k@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[3334] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[3334] query II @@ -4810,8 +4863,9 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--SortExec: expr=[k@0 ASC NULLS LAST], preserve_partitioning=[false] -04)----DataSourceExec: partitions=1, partition_sizes=[3334] +03)--BufferExec: capacity=1048576 +04)----SortExec: expr=[k@0 ASC NULLS LAST], preserve_partitioning=[false] +05)------DataSourceExec: partitions=1, partition_sizes=[3334] statement ok DROP TABLE t1; @@ -4845,7 +4899,8 @@ physical_plan 01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k2@0, k1@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -4865,7 +4920,8 @@ physical_plan 01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(k2@0, k1@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -4885,7 +4941,8 @@ physical_plan 01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(k2@0, k1@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -4904,7 +4961,8 @@ LEFT SEMI JOIN t2 ON k1 = k2 physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(k2@0, k1@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[10000] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT sum(k1), sum(v1) @@ -4923,7 +4981,8 @@ RIGHT SEMI JOIN t2 ON k1 = k2 physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(k2@0, k1@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[10000] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT sum(k2), sum(v2) @@ -4942,7 +5001,8 @@ LEFT ANTI JOIN t2 ON k1 = k2 physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[10000] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT sum(k1), sum(v1) @@ -4961,7 +5021,8 @@ RIGHT ANTI JOIN t2 ON k1 = k2 physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(k2@0, k1@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--DataSourceExec: partitions=1, partition_sizes=[10000] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT sum(k2), sum(v2) @@ -4981,7 +5042,8 @@ physical_plan 01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(k2@0, k1@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -5006,7 +5068,8 @@ physical_plan 01)FilterExec: k2@0 > 0 OR mark@2, projection=[k2@0, v2@1] 02)--HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(k2@0, k1@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * @@ -5030,7 +5093,8 @@ physical_plan 01)AggregateExec: mode=Single, gby=[v1@0 as v1], aggr=[] 02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)], projection=[v1@1] 03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +04)----BufferExec: capacity=1048576 +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query I SELECT distinct(v1) @@ -5050,8 +5114,9 @@ WHERE k1 < 0 physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[0] -03)--FilterExec: k1@0 < 0 -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +03)--BufferExec: capacity=1048576 +04)----FilterExec: k1@0 < 0 +05)------DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * @@ -5257,7 +5322,8 @@ physical_plan 02)--ProjectionExec: expr=[id@2 as id, join_key@3 as left_key, join_key@0 as right_key, value@1 as value] 03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(join_key@0, join_key@1)] 04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------BufferExec: capacity=1048576 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] statement count 0 set datafusion.explain.physical_plan_only = false; diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 96471411e0f95..524304546d569 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -871,4 +871,4 @@ DROP TABLE test_limit_with_partitions; # Tear down src_table table: statement ok -DROP TABLE src_table; \ No newline at end of file +DROP TABLE src_table; diff --git a/datafusion/sqllogictest/test_files/limit_single_row_batches.slt b/datafusion/sqllogictest/test_files/limit_single_row_batches.slt index fbdb0140e047a..9f626816e2146 100644 --- a/datafusion/sqllogictest/test_files/limit_single_row_batches.slt +++ b/datafusion/sqllogictest/test_files/limit_single_row_batches.slt @@ -19,4 +19,4 @@ SELECT COUNT(*) FROM (SELECT i FROM filter_limit WHERE i <> 0 LIMIT 1); 1 statement ok -DROP TABLE filter_limit; \ No newline at end of file +DROP TABLE filter_limit; diff --git a/datafusion/sqllogictest/test_files/null_aware_anti_join.slt b/datafusion/sqllogictest/test_files/null_aware_anti_join.slt index 5907a85a9b923..6ba98774fc6bf 100644 --- a/datafusion/sqllogictest/test_files/null_aware_anti_join.slt +++ b/datafusion/sqllogictest/test_files/null_aware_anti_join.slt @@ -60,7 +60,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[1] ############# ## Test 2: NOT IN with NULL in subquery - should return 0 rows (null-aware semantics) @@ -200,7 +201,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[1] ############# ## Test 12: Correlated NOT IN subquery with NULL @@ -291,7 +293,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(id@0, id@0)] 02)--DataSourceExec: partitions=1, partition_sizes=[1] -03)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--BufferExec: capacity=1048576 +04)----DataSourceExec: partitions=1, partition_sizes=[1] ############# ## Test 15: NOT EXISTS - No NULLs diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 7d33814b8bdbf..7e562bacf8696 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -673,10 +673,11 @@ physical_plan 03)----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 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------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 -06)--RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -07)----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 -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------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 +06)--BufferExec: capacity=1048576 +07)----RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +08)------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 +09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)----------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) @@ -758,13 +759,15 @@ physical_plan 02)--ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey, ps_suppkey@0 as ps_suppkey] 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, p_partkey@2)], projection=[ps_suppkey@1, l_extendedprice@2, l_discount@3, p_partkey@4] 04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] -06)--------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=1 -07)----------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 -08)--------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -09)----------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -10)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------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 +05)------BufferExec: capacity=1048576 +06)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] +07)----------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=1 +08)------------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 +09)----------BufferExec: capacity=1048576 +10)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +11)--------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +12)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------------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 diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 34c5fd97b51f3..edd493372fa67 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -340,7 +340,8 @@ physical_plan 10)------------------FilterExec: service@2 = log 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet]]}, projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -13)----------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +13)----------------BufferExec: capacity=1048576 +14)------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify results without optimization query TTTIR rowsort @@ -392,7 +393,8 @@ physical_plan 07)------------FilterExec: service@2 = log 08)--------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet]]}, projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -10)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +10)----------BufferExec: capacity=1048576 +11)------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] query TTTIR rowsort SELECT f.f_dkey, MAX(d.env), MAX(d.service), count(*), sum(f.value) diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index 4353f805c848b..823d4a5dd14be 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -301,8 +301,9 @@ explain select * from small_table join large_table on small_table.k = large_tabl physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k@0, k@0)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/small_table.parquet]]}, projection=[k], file_type=parquet -03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/large_table.parquet]]}, projection=[k, v], file_type=parquet, predicate=v@1 >= 50 AND DynamicFilter [ empty ], pruning_predicate=v_null_count@1 != row_count@2 AND v_max@0 >= 50, required_guarantees=[] +03)--BufferExec: capacity=1048576 +04)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/large_table.parquet]]}, projection=[k, v], file_type=parquet, predicate=v@1 >= 50 AND DynamicFilter [ empty ], pruning_predicate=v_null_count@1 != row_count@2 AND v_max@0 >= 50, required_guarantees=[] statement ok drop table small_table; diff --git a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt index e2c9fa4237939..d0fbd4120f152 100644 --- a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt +++ b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt @@ -383,7 +383,8 @@ physical_plan 14)--------------------------CoalescePartitionsExec 15)----------------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] 16)------------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -17)--------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +17)--------------------------BufferExec: capacity=1048576 +18)----------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify results without subset satisfaction query TPR rowsort @@ -479,7 +480,8 @@ physical_plan 11)--------------------CoalescePartitionsExec 12)----------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] 13)------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -14)--------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +14)--------------------BufferExec: capacity=1048576 +15)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify results match with subset satisfaction query TPR rowsort diff --git a/datafusion/sqllogictest/test_files/spark/collection/size.slt b/datafusion/sqllogictest/test_files/spark/collection/size.slt index dabcfd069bce8..106760eebfe42 100644 --- a/datafusion/sqllogictest/test_files/spark/collection/size.slt +++ b/datafusion/sqllogictest/test_files/spark/collection/size.slt @@ -129,4 +129,3 @@ SELECT size(column1) FROM VALUES (map(['a'], [1])), (map(['a','b'], [1,2])), (NU 1 2 -1 - diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index e73f4ec3e32da..7244bd9012aad 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -209,8 +209,9 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] -10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)------DataSourceExec: partitions=1, partition_sizes=[2] +10)----BufferExec: capacity=1048576 +11)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)--------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) as t2_sum from t1 @@ -242,8 +243,9 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] -10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)------DataSourceExec: partitions=1, partition_sizes=[2] +10)----BufferExec: capacity=1048576 +11)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)--------DataSourceExec: partitions=1, partition_sizes=[2] query IR rowsort SELECT t1_id, (SELECT sum(t2_int * 1.0) + 1 FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -275,8 +277,9 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] -10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)------DataSourceExec: partitions=1, partition_sizes=[2] +10)----BufferExec: capacity=1048576 +11)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)--------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 group by t2_id, 'a') as t2_sum from t1 @@ -310,8 +313,9 @@ physical_plan 08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: partitions=1, partition_sizes=[2] -11)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)------DataSourceExec: partitions=1, partition_sizes=[2] +11)----BufferExec: capacity=1048576 +12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)--------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 @@ -1183,8 +1187,9 @@ physical_plan 01)FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] 02)--HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(t2_id@0, t1_id@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[2] -04)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)------DataSourceExec: partitions=1, partition_sizes=[2] +04)----BufferExec: capacity=1048576 +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/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 62649148bf058..1262c07135b5f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -82,11 +82,14 @@ physical_plan 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] 12)----------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 13)------------------------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 -14)----------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -15)------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -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_custkey, o_orderdate], file_type=csv, has_header=false -17)------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -18)--------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -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_extendedprice, l_discount, l_returnflag], file_type=csv, has_header=false -20)--------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -21)----------------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 +14)----------------------BufferExec: capacity=1048576 +15)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +16)--------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +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_custkey, o_orderdate], file_type=csv, has_header=false +18)------------------BufferExec: capacity=1048576 +19)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +20)----------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +21)------------------------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 +22)--------------BufferExec: capacity=1048576 +23)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +24)------------------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 a31579eb1e09d..e23f0936d6b6e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -86,24 +86,28 @@ physical_plan 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] 12)----------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 13)------------------------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, ps_supplycost], file_type=csv, has_header=false -14)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -15)------------------------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 -16)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -17)--------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -18)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------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 -20)------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)] -21)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -22)----------CoalescePartitionsExec -23)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -24)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -25)----------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -26)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -27)--------------------RepartitionExec: partitioning=Hash([ps_suppkey@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_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false -29)--------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -30)----------------------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 -31)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -32)------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -33)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -34)----------------------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 +14)----------------------BufferExec: capacity=1048576 +15)------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +16)--------------------------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 +17)------------------BufferExec: capacity=1048576 +18)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +19)----------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +20)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +21)--------------------------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 +22)------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)] +23)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +24)----------CoalescePartitionsExec +25)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +26)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +27)----------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +28)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +29)--------------------RepartitionExec: partitioning=Hash([ps_suppkey@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_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false +31)--------------------BufferExec: capacity=1048576 +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_nationkey], file_type=csv, has_header=false +34)----------------BufferExec: capacity=1048576 +35)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +36)--------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +37)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +38)------------------------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 b152fde02f060..7afd151580a3a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -69,5 +69,6 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 09)----------------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] 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_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false -11)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------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 +11)--------------BufferExec: capacity=1048576 +12)----------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +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_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 94e0848bfcce1..11281155a43b7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -64,6 +64,7 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] 10)------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false -12)------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -13)--------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -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_custkey, o_comment], file_type=csv, has_header=false +12)------------------BufferExec: capacity=1048576 +13)--------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +14)----------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +15)------------------------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 a9ac517f287d0..2c768bd42c92d 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -50,5 +50,6 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 08)--------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] 09)----------------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 -10)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 -11)--------------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)------------BufferExec: capacity=1048576 +11)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 +12)----------------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 ae0c0a93a3552..c5fcd0f7efd63 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -83,12 +83,14 @@ physical_plan 10)------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 11)--------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false -13)------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] -14)--------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -15)----------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 -16)--------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -17)----------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -18)------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -19)--------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -20)----------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -21)------------------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)------BufferExec: capacity=1048576 +14)--------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] +15)----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +16)------------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 +17)----------BufferExec: capacity=1048576 +18)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +19)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +20)----------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +21)------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +22)--------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +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_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 b01110b567ca8..ea0ce242fd4a2 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -79,10 +79,12 @@ physical_plan 12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] 13)------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 14)--------------------------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 -15)------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -16)--------------------------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]) -17)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)------------------------------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 -19)--------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] -20)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)------------------------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 +15)------------------------BufferExec: capacity=1048576 +16)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +17)----------------------------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]) +18)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)--------------------------------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 +20)--------------------BufferExec: capacity=1048576 +21)----------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] +22)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +23)--------------------------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 83294d61a1698..9d7b1b0d7313b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -59,12 +59,14 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] 07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 08)--------------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 -09)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -10)--------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] -11)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)------------------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 -13)----------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] -14)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -15)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -16)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -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_partkey, l_quantity], file_type=csv, has_header=false +09)------------BufferExec: capacity=1048576 +10)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +11)----------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] +12)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)--------------------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 +14)----------BufferExec: capacity=1048576 +15)------------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] +16)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +17)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +18)------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +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_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 617051d602bd6..63a70b5ce500b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -76,12 +76,15 @@ physical_plan 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] 08)--------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], file_type=csv, has_header=false -10)--------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -11)----------------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_totalprice, o_orderdate], file_type=csv, has_header=false -12)----------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=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_quantity], file_type=csv, has_header=false -14)--------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -15)----------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -16)------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -17)--------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -18)----------------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 +10)--------------BufferExec: capacity=1048576 +11)----------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +12)------------------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_totalprice, o_orderdate], file_type=csv, has_header=false +13)----------BufferExec: capacity=1048576 +14)------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +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_orderkey, l_quantity], file_type=csv, has_header=false +16)--------BufferExec: capacity=1048576 +17)----------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +18)------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +19)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +20)----------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +21)------------------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 72c21e060fa66..38131fe4ce9f4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -72,7 +72,8 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 07)------------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] 08)--------------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 -09)----------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -10)------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) AND p_size@2 >= 1 -11)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------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)----------BufferExec: capacity=1048576 +10)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +11)--------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) AND p_size@2 >= 1 +12)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------------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 d5ff6724402ad..6572db2ab59a0 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -116,33 +116,41 @@ physical_plan 15)----------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@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_mfgr, p_type, p_size], file_type=csv, has_header=false -18)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -19)----------------------------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 -20)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -21)------------------------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 -22)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -23)------------------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 -24)------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -25)--------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -26)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)------------------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 -28)--------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -29)----------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -30)------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -31)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -32)----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -33)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -34)--------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -35)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -36)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -37)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -38)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 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_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false -40)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -41)------------------------------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 -42)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -43)--------------------------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 -44)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -45)----------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -46)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)--------------------------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 +18)--------------------------BufferExec: capacity=1048576 +19)----------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +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, ps_supplycost], file_type=csv, has_header=false +21)----------------------BufferExec: capacity=1048576 +22)------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +23)--------------------------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 +24)----------------BufferExec: capacity=1048576 +25)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 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, n_regionkey], file_type=csv, has_header=false +27)------------BufferExec: capacity=1048576 +28)--------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +29)----------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +30)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)--------------------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 +32)--------BufferExec: capacity=1048576 +33)----------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +34)------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +35)--------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +36)----------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +37)------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +38)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +39)----------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +40)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +41)--------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +42)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +43)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +44)--------------------------------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 +45)------------------------------BufferExec: capacity=1048576 +46)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +47)----------------------------------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 +48)--------------------------BufferExec: capacity=1048576 +49)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +50)------------------------------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 +51)----------------------BufferExec: capacity=1048576 +52)------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +53)--------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +54)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +55)------------------------------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 426a1cbaa4e22..8aa4298bf83d0 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -88,23 +88,27 @@ physical_plan 05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] 06)----------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=1 07)------------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 -08)----------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -09)------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] -10)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)----------------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 -12)------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -13)--------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] -14)----------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -15)------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -16)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -17)----------------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 -18)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -19)----------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] -20)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)--------------------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 -22)----------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] -23)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -24)--------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -25)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -26)------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -27)--------------------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 +08)----------BufferExec: capacity=1048576 +09)------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +10)--------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +11)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)------------------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 +13)------BufferExec: capacity=1048576 +14)--------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +15)----------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] +16)------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +17)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +18)----------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +19)------------------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 +20)----------------BufferExec: capacity=1048576 +21)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +22)--------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] +23)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +24)------------------------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 +25)------------BufferExec: capacity=1048576 +26)--------------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] +27)----------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +28)------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +29)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +30)----------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] +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_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 5e9192d677532..3e9018f94a102 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -106,18 +106,23 @@ physical_plan 14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] 15)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 16)------------------------------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 -17)----------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -18)------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -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_suppkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false -20)------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -21)--------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -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_orderstatus], file_type=csv, has_header=false -23)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -24)----------------------FilterExec: n_name@1 = SAUDI ARABIA, 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)----------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -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_suppkey], file_type=csv, has_header=false -29)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -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 +17)----------------------------BufferExec: capacity=1048576 +18)------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +19)--------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +20)----------------------------------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 +21)------------------------BufferExec: capacity=1048576 +22)--------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +23)----------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +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_orderstatus], file_type=csv, has_header=false +25)--------------------BufferExec: capacity=1048576 +26)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +27)------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] +28)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)----------------------------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 +30)----------------BufferExec: capacity=1048576 +31)------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +32)--------------------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 +33)--------------BufferExec: capacity=1048576 +34)----------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +35)------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +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_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 add578c3b079d..1089de20d6cdd 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -90,11 +90,12 @@ physical_plan 14)--------------------------FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) 15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 16)------------------------------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 -17)------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -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_custkey], file_type=csv, has_header=false -19)------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] -20)--------------------CoalescePartitionsExec -21)----------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -22)------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] -23)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -24)----------------------------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)------------------------BufferExec: capacity=1048576 +18)--------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +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_custkey], file_type=csv, has_header=false +20)------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] +21)--------------------CoalescePartitionsExec +22)----------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] +23)------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] +24)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)----------------------------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 7fec4e5f5d624..290fcb59e8640 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -69,9 +69,11 @@ physical_plan 09)----------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)--------------------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 -12)--------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -13)----------------FilterExec: o_orderdate@2 < 1995-03-15 -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_custkey, o_orderdate, o_shippriority], file_type=csv, has_header=false -15)----------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -16)------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -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_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +12)--------------BufferExec: capacity=1048576 +13)----------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +14)------------------FilterExec: o_orderdate@2 < 1995-03-15 +15)--------------------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 +16)----------BufferExec: capacity=1048576 +17)------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +18)--------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +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_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 0007666f15365..471920ce8abb9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -63,6 +63,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 09)----------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] 10)------------------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 -11)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] -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_commitdate, l_receiptdate], file_type=csv, has_header=false +11)--------------BufferExec: capacity=1048576 +12)----------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +13)------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] +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_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 d854001f3cc4c..4d499b13ecd14 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -84,16 +84,21 @@ physical_plan 15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] 16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 17)--------------------------------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 -18)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -19)--------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -20)----------------------------------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 -21)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -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_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false -23)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 -24)------------------------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 -25)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 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, n_regionkey], file_type=csv, has_header=false -27)--------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -28)----------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -29)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)--------------------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 +18)------------------------------BufferExec: capacity=1048576 +19)--------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +20)----------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +21)------------------------------------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 +22)--------------------------BufferExec: capacity=1048576 +23)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +24)------------------------------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 +25)----------------------BufferExec: capacity=1048576 +26)------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 +27)--------------------------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 +28)------------------BufferExec: capacity=1048576 +29)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +30)----------------------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 +31)--------------BufferExec: capacity=1048576 +32)----------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +33)------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +34)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)----------------------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/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index b4e70993396e6..8e1442f3d0f79 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -102,18 +102,23 @@ physical_plan 16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] 17)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 18)----------------------------------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 -19)--------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -20)----------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 -21)------------------------------------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 -22)----------------------------RepartitionExec: partitioning=Hash([o_orderkey@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_orderkey, o_custkey], file_type=csv, has_header=false -24)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -25)--------------------------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 -26)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -27)----------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY -28)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)--------------------------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 -30)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -31)------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -32)--------------------RepartitionExec: partitioning=RoundRobinBatch(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 +19)--------------------------------BufferExec: capacity=1048576 +20)----------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +21)------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 +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_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +23)----------------------------BufferExec: capacity=1048576 +24)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +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], file_type=csv, has_header=false +26)------------------------BufferExec: capacity=1048576 +27)--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +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)--------------------BufferExec: capacity=1048576 +30)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +31)------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +32)--------------------------RepartitionExec: partitioning=RoundRobinBatch(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 +34)----------------BufferExec: capacity=1048576 +35)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +36)--------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +37)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +38)------------------------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 12f19d43d40e7..af20b99681d91 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -114,20 +114,27 @@ physical_plan 22)------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] 23)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 24)----------------------------------------------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 -25)----------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -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_partkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false -27)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -28)--------------------------------------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 -29)--------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -30)----------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -31)------------------------------------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 -32)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -33)------------------------------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 -34)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -35)--------------------------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 -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], file_type=csv, has_header=false -38)----------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -39)------------------FilterExec: r_name@1 = AMERICA, 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 +25)----------------------------------------BufferExec: capacity=1048576 +26)------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +27)--------------------------------------------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 +28)------------------------------------BufferExec: capacity=1048576 +29)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +30)----------------------------------------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 +31)--------------------------------BufferExec: capacity=1048576 +32)----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +33)------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +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_custkey, o_orderdate], file_type=csv, has_header=false +35)----------------------------BufferExec: capacity=1048576 +36)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +37)--------------------------------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 +38)------------------------BufferExec: capacity=1048576 +39)--------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +40)----------------------------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 +41)--------------------BufferExec: capacity=1048576 +42)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +43)------------------------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 +44)----------------BufferExec: capacity=1048576 +45)------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +46)--------------------FilterExec: r_name@1 = AMERICA, 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 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 4ec434c90368f..6d640f5dfa6e7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -95,13 +95,18 @@ physical_plan 18)----------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] 19)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 20)--------------------------------------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 -21)--------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -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_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=csv, has_header=false -23)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -24)------------------------------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 -25)------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -26)--------------------------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 -27)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -28)----------------------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 -29)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -30)------------------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 +21)--------------------------------BufferExec: capacity=1048576 +22)----------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=csv, has_header=false +24)----------------------------BufferExec: capacity=1048576 +25)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +26)--------------------------------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 +27)------------------------BufferExec: capacity=1048576 +28)--------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +29)----------------------------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 +30)--------------------BufferExec: capacity=1048576 +31)----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +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_orderdate], file_type=csv, has_header=false +33)----------------BufferExec: capacity=1048576 +34)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +35)--------------------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 b79b6d2fe5e9e..82bd0960136ec 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -310,22 +310,24 @@ physical_plan 04)------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -08)------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -09)--------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)------------DataSourceExec: partitions=1, partition_sizes=[1] -12)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] -13)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1], NullsEqual: true -14)------CoalescePartitionsExec -15)--------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -16)----------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -17)------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -18)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -19)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -20)------------------DataSourceExec: partitions=1, partition_sizes=[1] -21)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -22)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)----BufferExec: capacity=1048576 +08)------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +09)--------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +10)----------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +11)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] +13)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +14)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1], NullsEqual: true +15)------CoalescePartitionsExec +16)--------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +17)----------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +18)------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +19)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +20)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +21)------------------DataSourceExec: partitions=1, partition_sizes=[1] +22)------BufferExec: capacity=1048576 +23)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +24)----------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -378,17 +380,19 @@ physical_plan 06)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------DataSourceExec: partitions=1, partition_sizes=[1] -11)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true -12)----CoalescePartitionsExec -13)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] -14)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -15)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -16)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------DataSourceExec: partitions=1, partition_sizes=[1] -18)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------DataSourceExec: partitions=1, partition_sizes=[1] +09)----BufferExec: capacity=1048576 +10)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------DataSourceExec: partitions=1, partition_sizes=[1] +12)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true +13)----CoalescePartitionsExec +14)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +15)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +16)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +17)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)--------------DataSourceExec: partitions=1, partition_sizes=[1] +19)----BufferExec: capacity=1048576 +20)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +21)--------DataSourceExec: partitions=1, partition_sizes=[1] # union_upcast_types query TT diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index f0ee0cbbc4e55..65580110186e4 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -133,6 +133,7 @@ The following configuration settings are available: | datafusion.execution.enforce_batch_size_in_joins | false | Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. | | datafusion.execution.objectstore_writer_buffer_size | 10485760 | Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. | | datafusion.execution.enable_ansi_mode | false | Whether to enable ANSI SQL mode. The flag is experimental and relevant only for DataFusion Spark built-in functions When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL semantics for expressions, casting, and error handling. This means: - **Strict type coercion rules:** implicit casts between incompatible types are disallowed. - **Standard SQL arithmetic behavior:** operations such as division by zero, numeric overflow, or invalid casts raise runtime errors rather than returning `NULL` or adjusted values. - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling. When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive, non-ANSI mode designed for user convenience and backward compatibility. In this mode: - Implicit casts between types are allowed (e.g., string to integer when possible). - Arithmetic operations are more lenient — for example, `abs()` on the minimum representable integer value returns the input value instead of raising overflow. - Division by zero or invalid casts may return `NULL` instead of failing. # Default `false` — ANSI SQL mode is disabled by default. | +| datafusion.execution.hash_join_buffering_capacity | 1048576 | How many bytes to buffer in the probe side of hash joins while the build side is concurrently being built. Without this, hash joins will wait until the full materialization of the build side before polling the probe side. This is useful in scenarios where the query is not completely CPU bounded, allowing to do some early work concurrently and reducing the latency of the query. 1Mb by default. Set to 0 for disabling it. | | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible |