From 34963edd09be61c4625cb688b307659dddbb93a0 Mon Sep 17 00:00:00 2001 From: "google-labs-jules[bot]" <161369871+google-labs-jules[bot]@users.noreply.github.com> Date: Sun, 22 Feb 2026 13:12:13 +0000 Subject: [PATCH 01/78] Implement morsel-driven execution for ParquetExec This PR implements morsel-driven execution for Parquet files in DataFusion, enabling row-group level work sharing across partitions to mitigate data skew. Key changes: - Introduced `WorkQueue` in `datafusion/datasource/src/file_stream.rs` for shared pool of work. - Added `morselize` method to `FileOpener` trait to allow dynamic splitting of files into morsels. - Implemented `morselize` for `ParquetOpener` to split files into individual row groups. - Cached `ParquetMetaData` in `ParquetMorsel` extensions to avoid redundant I/O. - Modified `FileStream` to support work stealing from the shared queue. - Implemented `Weak` pointer pattern for `WorkQueue` in `FileScanConfig` to support plan re-executability. - Added `MorselizingGuard` to ensure shared state consistency on cancellation. - Added `allow_morsel_driven` configuration option (enabled by default for Parquet). - Implemented row-group pruning during the morselization phase for better efficiency. Tests: - Added `parquet_morsel_driven_execution` test to verify work distribution and re-executability. - Added `parquet_morsel_driven_enabled_by_default` to verify the default configuration. Co-authored-by: Dandandan <163737+Dandandan@users.noreply.github.com> --- datafusion/common/src/config.rs | 4 + .../common/src/file_options/parquet_writer.rs | 2 + .../src/datasource/physical_plan/parquet.rs | 147 ++++++++++- .../datasource-parquet/src/file_format.rs | 1 + datafusion/datasource-parquet/src/opener.rs | 150 ++++++++++- datafusion/datasource/src/file_scan_config.rs | 20 ++ datafusion/datasource/src/file_stream.rs | 245 +++++++++++++++++- 7 files changed, 553 insertions(+), 16 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index dad12c1c6bc91..23adb989f7981 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -743,6 +743,10 @@ config_namespace! { /// (reading) Use any available bloom filters when reading parquet files pub bloom_filter_on_read: bool, default = true + /// (reading) If true, the parquet reader will share work between partitions + /// using morsel-driven execution. This can help mitigate data skew. + pub allow_morsel_driven: bool, default = true + /// (reading) The maximum predicate cache size, in bytes. When /// `pushdown_filters` is enabled, sets the maximum memory used to cache /// the results of predicate evaluation between filter evaluation and diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index f6608d16c1022..95fedd25aaa72 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -208,6 +208,7 @@ impl ParquetOptions { binary_as_string: _, // not used for writer props coerce_int96: _, // not used for writer props skip_arrow_metadata: _, + allow_morsel_driven: _, max_predicate_cache_size: _, } = self; @@ -573,6 +574,7 @@ mod tests { schema_force_view_types: global_options_defaults.schema_force_view_types, binary_as_string: global_options_defaults.binary_as_string, skip_arrow_metadata: global_options_defaults.skip_arrow_metadata, + allow_morsel_driven: global_options_defaults.allow_morsel_driven, coerce_int96: None, }, column_specific_options, diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 4c6d915d5bcaa..697113b093c4f 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -50,7 +50,7 @@ mod tests { use datafusion_common::test_util::{batches_to_sort_string, batches_to_string}; use datafusion_common::{Result, ScalarValue, assert_contains}; use datafusion_datasource::file_format::FileFormat; - use datafusion_datasource::file_scan_config::FileScanConfigBuilder; + use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::file::FileSource; @@ -2459,4 +2459,149 @@ mod tests { assert_eq!(calls.len(), 2); assert_eq!(calls, vec![Some(123), Some(456)]); } + + #[tokio::test] + async fn parquet_morsel_driven_execution() -> Result<()> { + let store = + Arc::new(object_store::memory::InMemory::new()) as Arc; + let store_url = ObjectStoreUrl::parse("memory://test").unwrap(); + + let ctx = SessionContext::new(); + ctx.register_object_store(store_url.as_ref(), store.clone()); + + // Create a Parquet file with 100 row groups, each with 10 rows + let schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + + let mut out = Vec::new(); + let props = WriterProperties::builder() + .set_max_row_group_size(10) + .build(); + { + let mut writer = + ArrowWriter::try_new(&mut out, Arc::clone(&schema), Some(props))?; + // Write many batches to ensure they are not coalesced and we can verify work distribution + for i in 0..100 { + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(Int32Array::from(vec![i; 10]))], + )?; + writer.write(&batch)?; + } + writer.close()?; + } + + let path = Path::from("skewed.parquet"); + store.put(&path, out.into()).await?; + let meta = store.head(&path).await?; + + // Set up DataSourceExec with 2 partitions, but the file is only in partition 0 (skewed) + let source = Arc::new(ParquetSource::new(schema)); + let config = FileScanConfigBuilder::new(store_url, source) + .with_file_group(FileGroup::new(vec![PartitionedFile::new_from_meta( + meta, + )])) + .with_file_group(FileGroup::new(vec![])) // Partition 1 is empty + .with_morsel_driven(true) + .build(); + + let exec = DataSourceExec::from_data_source(config); + + // Execute both partitions concurrently + let task_ctx = ctx.task_ctx(); + let stream0 = exec.execute(0, Arc::clone(&task_ctx))?; + let stream1 = exec.execute(1, Arc::clone(&task_ctx))?; + + let handle0 = tokio::spawn(async move { + let mut count = 0; + let mut s = stream0; + while let Some(batch) = s.next().await { + count += batch.unwrap().num_rows(); + tokio::task::yield_now().await; + } + count + }); + + let handle1 = tokio::spawn(async move { + let mut count = 0; + let mut s = stream1; + while let Some(batch) = s.next().await { + count += batch.unwrap().num_rows(); + tokio::task::yield_now().await; + } + count + }); + + let count0 = handle0.await.unwrap(); + let count1 = handle1.await.unwrap(); + + // Total rows should be 1000 + assert_eq!(count0 + count1, 1000); + + // Since it's morsel-driven, both partitions should have done some work + // because the work from partition 0 (the single file) was split into + // individual row groups and shared via the shared queue. + assert!(count0 > 0, "Partition 0 should have produced rows"); + assert!(count1 > 0, "Partition 1 should have produced rows"); + + // Test re-executability: executing the same plan again should work + let stream0 = exec.execute(0, Arc::clone(&task_ctx))?; + let stream1 = exec.execute(1, Arc::clone(&task_ctx))?; + + let mut count = 0; + let mut s0 = stream0; + while let Some(batch) = s0.next().await { + count += batch.unwrap().num_rows(); + } + let mut s1 = stream1; + while let Some(batch) = s1.next().await { + count += batch.unwrap().num_rows(); + } + assert_eq!(count, 1000, "Second execution should also produce 1000 rows"); + + Ok(()) + } + + #[tokio::test] + async fn parquet_morsel_driven_enabled_by_default() -> Result<()> { + let tmp_dir = TempDir::new()?; + let path = tmp_dir.path().join("test.parquet"); + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], + )?; + + let file = File::create(&path)?; + let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), None)?; + writer.write(&batch)?; + writer.close()?; + + let ctx = SessionContext::new(); + ctx.register_parquet("t", path.to_str().unwrap(), ParquetReadOptions::default()) + .await?; + + let df = ctx.sql("SELECT * FROM t").await?; + let plan = df.create_physical_plan().await?; + + // Plan should be a ProjectionExec over a DataSourceExec + let ds_exec = if let Some(ds) = plan.as_any().downcast_ref::() { + ds + } else { + plan.children()[0] + .as_any() + .downcast_ref::() + .expect("Expected DataSourceExec") + }; + + let config = ds_exec + .data_source() + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + + assert!(config.morsel_driven, "morsel_driven should be enabled by default for Parquet"); + + Ok(()) + } } diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index d59b42ed15d15..59c77c8939124 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -536,6 +536,7 @@ impl FileFormat for ParquetFormat { let conf = FileScanConfigBuilder::from(conf) .with_source(Arc::new(source)) + .with_morsel_driven(self.options.global.allow_morsel_driven) .build(); Ok(DataSourceExec::from_data_source(conf)) } diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index f87a30265a17b..5525271ba0836 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -52,6 +52,8 @@ use datafusion_physical_plan::metrics::{ use datafusion_pruning::{FilePruner, PruningPredicate, build_pruning_predicate}; use crate::sort::reverse_row_selection; +use futures::future::{BoxFuture, ready}; +use parquet::file::metadata::ParquetMetaData; #[cfg(feature = "parquet_encryption")] use datafusion_common::config::EncryptionFactoryOptions; #[cfg(feature = "parquet_encryption")] @@ -122,6 +124,14 @@ pub(super) struct ParquetOpener { pub reverse_row_groups: bool, } +/// A morsel of work for Parquet execution, containing cached metadata and an access plan. +pub struct ParquetMorsel { + /// Cached Parquet metadata + pub metadata: Arc, + /// Access plan for this morsel (usually selecting a single row group) + pub access_plan: ParquetAccessPlan, +} + /// Represents a prepared access plan with optional row selection pub(crate) struct PreparedAccessPlan { /// Row group indexes to read @@ -146,10 +156,7 @@ impl PreparedAccessPlan { } /// Reverse the access plan for reverse scanning - pub(crate) fn reverse( - mut self, - file_metadata: &parquet::file::metadata::ParquetMetaData, - ) -> Result { + pub(crate) fn reverse(mut self, file_metadata: &ParquetMetaData) -> Result { // Get the row group indexes before reversing let row_groups_to_scan = self.row_group_indexes.clone(); @@ -181,6 +188,119 @@ impl PreparedAccessPlan { } impl FileOpener for ParquetOpener { + fn morselize( + &self, + partitioned_file: PartitionedFile, + ) -> BoxFuture<'static, Result>> { + if partitioned_file + .extensions + .as_ref() + .map(|e| e.is::()) + .unwrap_or(false) + { + return Box::pin(ready(Ok(vec![partitioned_file]))); + } + + let file_metrics = ParquetFileMetrics::new( + self.partition_index, + partitioned_file.object_meta.location.as_ref(), + &self.metrics, + ); + + let metadata_size_hint = partitioned_file + .metadata_size_hint + .or(self.metadata_size_hint); + + let mut async_file_reader: Box = + match self.parquet_file_reader_factory.create_reader( + self.partition_index, + partitioned_file.clone(), + metadata_size_hint, + &self.metrics, + ) { + Ok(reader) => reader, + Err(e) => return Box::pin(ready(Err(e))), + }; + + let options = ArrowReaderOptions::new().with_page_index(false); + #[cfg(feature = "parquet_encryption")] + let encryption_context = self.get_encryption_context(); + + let expr_adapter_factory = Arc::clone(&self.expr_adapter_factory); + let table_schema = self.table_schema.clone(); + let predicate = self.predicate.clone(); + let metrics = self.metrics.clone(); + + Box::pin(async move { + #[cfg(feature = "parquet_encryption")] + let options = if let Some(fd_val) = encryption_context + .get_file_decryption_properties(&partitioned_file.object_meta.location) + .await? + { + options.with_file_decryption_properties(Arc::clone(&fd_val)) + } else { + options + }; + + let mut _metadata_timer = file_metrics.metadata_load_time.timer(); + let reader_metadata = + ArrowReaderMetadata::load_async(&mut async_file_reader, options).await?; + let metadata = reader_metadata.metadata(); + let num_row_groups = metadata.num_row_groups(); + + // Adapt the physical schema to the file schema for pruning + let physical_file_schema = Arc::clone(reader_metadata.schema()); + let logical_file_schema = table_schema.file_schema(); + let rewriter = expr_adapter_factory.create( + Arc::clone(logical_file_schema), + Arc::clone(&physical_file_schema), + )?; + let simplifier = PhysicalExprSimplifier::new(&physical_file_schema); + let adapted_predicate = predicate + .as_ref() + .map(|p| simplifier.simplify(rewriter.rewrite(Arc::clone(p))?)) + .transpose()?; + + let predicate_creation_errors = MetricBuilder::new(&metrics) + .global_counter("num_predicate_creation_errors"); + + let (pruning_predicate, _) = build_pruning_predicates( + adapted_predicate.as_ref(), + &physical_file_schema, + &predicate_creation_errors, + ); + + let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(num_row_groups)); + if let Some(predicate) = pruning_predicate { + row_groups.prune_by_statistics( + &physical_file_schema, + reader_metadata.parquet_schema(), + metadata.row_groups(), + predicate.as_ref(), + &file_metrics, + ); + } + let access_plan = row_groups.build(); + + let mut morsels = Vec::with_capacity(access_plan.len()); + for i in 0..num_row_groups { + if !access_plan.should_scan(i) { + continue; + } + let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); + morsel_access_plan.scan(i); + let morsel = ParquetMorsel { + metadata: Arc::clone(metadata), + access_plan: morsel_access_plan, + }; + let mut f = partitioned_file.clone(); + f.extensions = Some(Arc::new(morsel)); + morsels.push(f); + } + Ok(morsels) + }) + } + fn open(&self, partitioned_file: PartitionedFile) -> Result { // ----------------------------------- // Step: prepare configurations, etc. @@ -358,10 +478,18 @@ impl FileOpener for ParquetOpener { // Begin by loading the metadata from the underlying reader (note // the returned metadata may actually include page indexes as some // readers may return page indexes even when not requested -- for - // example when they are cached) - let mut reader_metadata = + // example when they are cached). + // If this is a morsel, we might already have the metadata cached. + let mut reader_metadata = if let Some(morsel) = partitioned_file + .extensions + .as_ref() + .and_then(|e| e.downcast_ref::()) + { + ArrowReaderMetadata::try_new(Arc::clone(&morsel.metadata), options.clone())? + } else { ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) - .await?; + .await? + }; // Note about schemas: we are actually dealing with **3 different schemas** here: // - The table schema as defined by the TableProvider. @@ -927,6 +1055,14 @@ fn create_initial_plan( // check row group count matches the plan return Ok(access_plan.clone()); + } else if let Some(morsel) = extensions.downcast_ref::() { + let plan_len = morsel.access_plan.len(); + if plan_len != row_group_count { + return exec_err!( + "Invalid ParquetMorsel AccessPlan for {file_name}. Specified {plan_len} row groups, but file has {row_group_count}" + ); + } + return Ok(morsel.access_plan.clone()); } else { debug!("DataSourceExec Ignoring unknown extension specified for {file_name}"); } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index c3e5cabce7bc2..7351f54f6462e 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -53,6 +53,7 @@ use datafusion_physical_plan::{ metrics::ExecutionPlanMetricsSet, }; use log::{debug, warn}; +use std::sync::{Mutex, Weak}; use std::{any::Any, fmt::Debug, fmt::Formatter, fmt::Result as FmtResult, sync::Arc}; /// [`FileScanConfig`] represents scanning data from a group of files @@ -204,6 +205,13 @@ pub struct FileScanConfig { /// If the number of file partitions > target_partitions, the file partitions will be grouped /// in a round-robin fashion such that number of file partitions = target_partitions. pub partitioned_by_file_group: bool, + /// When true, use morsel-driven execution to avoid data skew. + /// This means all partitions share a single pool of work. + pub morsel_driven: bool, + /// Shared work queue for morsel-driven execution. + /// This uses a Weak pointer to allow the queue to be dropped when all execution + /// partitions are finished, supporting re-executability of the physical plan. + pub(crate) morsel_queue: Arc>>, } /// A builder for [`FileScanConfig`]'s. @@ -274,6 +282,7 @@ pub struct FileScanConfigBuilder { batch_size: Option, expr_adapter_factory: Option>, partitioned_by_file_group: bool, + morsel_driven: bool, } impl FileScanConfigBuilder { @@ -300,6 +309,7 @@ impl FileScanConfigBuilder { batch_size: None, expr_adapter_factory: None, partitioned_by_file_group: false, + morsel_driven: false, } } @@ -500,6 +510,12 @@ impl FileScanConfigBuilder { self } + /// Set whether to use morsel-driven execution. + pub fn with_morsel_driven(mut self, morsel_driven: bool) -> Self { + self.morsel_driven = morsel_driven; + self + } + /// Build the final [`FileScanConfig`] with all the configured settings. /// /// This method takes ownership of the builder and returns the constructed `FileScanConfig`. @@ -521,6 +537,7 @@ impl FileScanConfigBuilder { batch_size, expr_adapter_factory: expr_adapter, partitioned_by_file_group, + morsel_driven, } = self; let constraints = constraints.unwrap_or_default(); @@ -546,6 +563,8 @@ impl FileScanConfigBuilder { expr_adapter_factory: expr_adapter, statistics, partitioned_by_file_group, + morsel_driven, + morsel_queue: Arc::new(Mutex::new(Weak::new())), } } } @@ -565,6 +584,7 @@ impl From for FileScanConfigBuilder { batch_size: config.batch_size, expr_adapter_factory: config.expr_adapter_factory, partitioned_by_file_group: config.partitioned_by_file_group, + morsel_driven: config.morsel_driven, } } } diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index c8090382094ef..23d85b8263e8d 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -24,8 +24,10 @@ use std::collections::VecDeque; use std::mem; use std::pin::Pin; -use std::sync::Arc; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::{Arc, Mutex}; use std::task::{Context, Poll}; +use tokio::sync::Notify; use crate::PartitionedFile; use crate::file_scan_config::FileScanConfig; @@ -43,10 +45,25 @@ use futures::future::BoxFuture; use futures::stream::BoxStream; use futures::{FutureExt as _, Stream, StreamExt as _, ready}; +/// A guard that decrements the morselizing count when dropped. +struct MorselizingGuard { + queue: Arc, +} + +impl Drop for MorselizingGuard { + fn drop(&mut self) { + self.queue.stop_morselizing(); + } +} + /// A stream that iterates record batch by record batch, file over file. pub struct FileStream { /// An iterator over input files. file_iter: VecDeque, + /// Shared work queue for morsel-driven execution. + shared_queue: Option>, + /// Whether to use morsel-driven execution. + morsel_driven: bool, /// The stream schema (file schema including partition columns and after /// projection). projected_schema: SchemaRef, @@ -63,6 +80,8 @@ pub struct FileStream { baseline_metrics: BaselineMetrics, /// Describes the behavior of the `FileStream` if file opening or scanning fails on_error: OnError, + /// Guard for morselizing state to ensure counter is decremented on drop + morsel_guard: Option, } impl FileStream { @@ -75,10 +94,30 @@ impl FileStream { ) -> Result { let projected_schema = config.projected_schema()?; - let file_group = config.file_groups[partition].clone(); + let (file_iter, shared_queue) = if config.morsel_driven { + let mut guard = config.morsel_queue.lock().unwrap(); + let queue = if let Some(queue) = guard.upgrade() { + queue + } else { + let all_files = config + .file_groups + .iter() + .flat_map(|g| g.files().to_vec()) + .collect(); + let queue = Arc::new(WorkQueue::new(all_files)); + *guard = Arc::downgrade(&queue); + queue + }; + (VecDeque::new(), Some(queue)) + } else { + let file_group = config.file_groups[partition].clone(); + (file_group.into_inner().into_iter().collect(), None) + }; Ok(Self { - file_iter: file_group.into_inner().into_iter().collect(), + file_iter, + shared_queue, + morsel_driven: config.morsel_driven, projected_schema, remain: config.limit, file_opener, @@ -86,6 +125,7 @@ impl FileStream { file_stream_metrics: FileStreamMetrics::new(metrics, partition), baseline_metrics: BaselineMetrics::new(metrics, partition), on_error: OnError::Fail, + morsel_guard: None, }) } @@ -103,6 +143,9 @@ impl FileStream { /// Since file opening is mostly IO (and may involve a /// bunch of sequential IO), it can be parallelized with decoding. fn start_next_file(&mut self) -> Option> { + if self.morsel_driven { + return None; + } let part_file = self.file_iter.pop_front()?; Some(self.file_opener.open(part_file)) } @@ -113,15 +156,86 @@ impl FileStream { FileStreamState::Idle => { self.file_stream_metrics.time_opening.start(); - match self.start_next_file().transpose() { - Ok(Some(future)) => self.state = FileStreamState::Open { future }, - Ok(None) => return Poll::Ready(None), + if self.morsel_driven { + let queue = self.shared_queue.as_ref().expect("shared queue"); + match queue.pull() { + WorkStatus::Work(part_file) => { + self.morsel_guard = Some(MorselizingGuard { + queue: Arc::clone(queue), + }); + self.state = FileStreamState::Morselizing { + future: self.file_opener.morselize(*part_file), + }; + } + WorkStatus::Wait => { + self.file_stream_metrics.time_opening.stop(); + let queue_captured = Arc::clone(queue); + self.state = FileStreamState::Waiting { + future: Box::pin(async move { + let notified = queue_captured.notify.notified(); + if !queue_captured.has_work_or_done() { + notified.await; + } + }), + }; + } + WorkStatus::Done => { + self.file_stream_metrics.time_opening.stop(); + return Poll::Ready(None); + } + } + } else { + match self.start_next_file().transpose() { + Ok(Some(future)) => self.state = FileStreamState::Open { future }, + Ok(None) => return Poll::Ready(None), + Err(e) => { + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } + } + FileStreamState::Morselizing { future } => { + match ready!(future.poll_unpin(cx)) { + Ok(morsels) => { + let queue = self.shared_queue.as_ref().expect("shared queue"); + // Take the guard to decrement morselizing_count + let _guard = self.morsel_guard.take(); + + if morsels.len() > 1 { + self.file_stream_metrics.time_opening.stop(); + // Expanded into multiple morsels. Put all back and pull again. + queue.push_many(morsels); + self.state = FileStreamState::Idle; + } else if morsels.len() == 1 { + // No further expansion possible. Proceed to open. + let morsel = morsels.into_iter().next().unwrap(); + match self.file_opener.open(morsel) { + Ok(future) => self.state = FileStreamState::Open { future }, + Err(e) => { + self.file_stream_metrics.time_opening.stop(); + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } else { + self.file_stream_metrics.time_opening.stop(); + // No morsels returned, skip this file + self.state = FileStreamState::Idle; + } + } Err(e) => { + let _guard = self.morsel_guard.take(); + self.file_stream_metrics.time_opening.stop(); self.state = FileStreamState::Error; return Poll::Ready(Some(Err(e))); } } } + FileStreamState::Waiting { future } => { + ready!(future.poll_unpin(cx)); + self.state = FileStreamState::Idle; + } FileStreamState::Open { future } => match ready!(future.poll_unpin(cx)) { Ok(reader) => { // include time needed to start opening in `start_next_file` @@ -214,7 +328,13 @@ impl FileStream { } } } - None => return Poll::Ready(None), + None => { + if self.morsel_driven { + self.state = FileStreamState::Idle; + } else { + return Poll::Ready(None); + } + } }, OnError::Fail => { self.state = FileStreamState::Error; @@ -243,7 +363,13 @@ impl FileStream { } } } - None => return Poll::Ready(None), + None => { + if self.morsel_driven { + self.state = FileStreamState::Idle; + } else { + return Poll::Ready(None); + } + } } } } @@ -276,6 +402,89 @@ impl RecordBatchStream for FileStream { } } +/// Result of pulling work from the queue +#[derive(Debug)] +pub enum WorkStatus { + /// A morsel is available + Work(Box), + /// No morsel available now, but others are morselizing + Wait, + /// No more work available + Done, +} + +/// A shared queue of [`PartitionedFile`] morsels for morsel-driven execution. +#[derive(Debug)] +pub struct WorkQueue { + queue: Mutex>, + /// Number of workers currently morselizing a file. + morselizing_count: AtomicUsize, + /// Notify waiters when work is added or morselizing finishes. + notify: Notify, +} + +impl WorkQueue { + /// Create a new `WorkQueue` with the given initial files + pub fn new(initial_files: Vec) -> Self { + Self { + queue: Mutex::new(VecDeque::from(initial_files)), + morselizing_count: AtomicUsize::new(0), + notify: Notify::new(), + } + } + + /// Pull a file from the queue. + pub fn pull(&self) -> WorkStatus { + let mut queue = self.queue.lock().unwrap(); + if let Some(file) = queue.pop_front() { + self.morselizing_count.fetch_add(1, Ordering::SeqCst); + WorkStatus::Work(Box::new(file)) + } else if self.morselizing_count.load(Ordering::SeqCst) > 0 { + WorkStatus::Wait + } else { + WorkStatus::Done + } + } + + /// Returns true if there is work in the queue or if all morselizing is done. + pub fn has_work_or_done(&self) -> bool { + let queue = self.queue.lock().unwrap(); + !queue.is_empty() || self.morselizing_count.load(Ordering::SeqCst) == 0 + } + + /// Push many files back to the queue. + /// + /// This is used when a file is expanded into multiple morsels. + pub fn push_many(&self, files: Vec) { + if files.is_empty() { + return; + } + self.queue.lock().unwrap().extend(files); + self.notify.notify_waiters(); + } + + /// Increment the morselizing count. + pub fn start_morselizing(&self) { + self.morselizing_count.fetch_add(1, Ordering::SeqCst); + } + + /// Decrement the morselizing count and notify waiters. + pub fn stop_morselizing(&self) { + self.morselizing_count.fetch_sub(1, Ordering::SeqCst); + self.notify.notify_waiters(); + } + + /// Return true if any worker is currently morselizing. + pub fn is_morselizing(&self) -> bool { + self.morselizing_count.load(Ordering::SeqCst) > 0 + } + + /// Return a future that resolves when work is added or morselizing finishes. + pub async fn wait_for_work(&self) { + self.notify.notified().await; + } +} + /// A fallible future that resolves to a stream of [`RecordBatch`] pub type FileOpenFuture = BoxFuture<'static, Result>>>; @@ -298,6 +507,16 @@ pub trait FileOpener: Unpin + Send + Sync { /// Asynchronously open the specified file and return a stream /// of [`RecordBatch`] fn open(&self, partitioned_file: PartitionedFile) -> Result; + + /// Optional: Split a file into smaller morsels for morsel-driven execution. + /// + /// By default, returns the file as a single morsel. + fn morselize( + &self, + file: PartitionedFile, + ) -> BoxFuture<'static, Result>> { + Box::pin(futures::future::ready(Ok(vec![file]))) + } } /// Represents the state of the next `FileOpenFuture`. Since we need to poll @@ -317,6 +536,16 @@ pub enum FileStreamState { /// A [`FileOpenFuture`] returned by [`FileOpener::open`] future: FileOpenFuture, }, + /// Currently splitting a file into smaller morsels. + Morselizing { + /// A future that resolves to a list of morsels + future: BoxFuture<'static, Result>>, + }, + /// Waiting for more work to be added to the queue. + Waiting { + /// A future that resolves when more work is available + future: BoxFuture<'static, ()>, + }, /// Scanning the [`BoxStream`] returned by the completion of a [`FileOpenFuture`] /// returned by [`FileOpener::open`] Scan { From a67f9acad3f94049fcd0b3a0affcf04f0b3bed7f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 22 Feb 2026 18:29:22 +0100 Subject: [PATCH 02/78] Proto --- .../proto-common/proto/datafusion_common.proto | 1 + datafusion/proto-common/src/from_proto/mod.rs | 2 ++ .../proto-common/src/generated/pbjson.rs | 18 ++++++++++++++++++ datafusion/proto-common/src/generated/prost.rs | 3 +++ .../src/generated/datafusion_proto_common.rs | 3 +++ 5 files changed, 27 insertions(+) diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 62c6bbe85612a..da7491c6115c8 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -549,6 +549,7 @@ message ParquetOptions { bool schema_force_view_types = 28; // default = false bool binary_as_string = 29; // default = false bool skip_arrow_metadata = 30; // default = false + bool allow_morsel_driven = 35; // default = true oneof metadata_size_hint_opt { uint64 metadata_size_hint = 4; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index ca8a269958d73..2b8740d7a0420 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -1089,7 +1089,9 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { skip_arrow_metadata: value.skip_arrow_metadata, max_predicate_cache_size: value.max_predicate_cache_size_opt.map(|opt| match opt { protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v) => Some(v as usize), + }).unwrap_or(None), + allow_morsel_driven: value.allow_morsel_driven, }) } } diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index b00e7546bba20..e5c6a509a5f4c 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -5683,6 +5683,9 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { len += 1; } + if self.allow_morsel_driven { + len += 1; + } if self.dictionary_page_size_limit != 0 { len += 1; } @@ -5788,6 +5791,9 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { struct_ser.serialize_field("skipArrowMetadata", &self.skip_arrow_metadata)?; } + if self.allow_morsel_driven { + struct_ser.serialize_field("allowMorselDriven", &self.allow_morsel_driven)?; + } if self.dictionary_page_size_limit != 0 { #[allow(clippy::needless_borrow)] #[allow(clippy::needless_borrows_for_generic_args)] @@ -5936,6 +5942,8 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "binaryAsString", "skip_arrow_metadata", "skipArrowMetadata", + "allow_morsel_driven", + "allowMorselDriven", "dictionary_page_size_limit", "dictionaryPageSizeLimit", "data_page_row_count_limit", @@ -5985,6 +5993,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { SchemaForceViewTypes, BinaryAsString, SkipArrowMetadata, + AllowMorselDriven, DictionaryPageSizeLimit, DataPageRowCountLimit, MaxRowGroupSize, @@ -6038,6 +6047,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "schemaForceViewTypes" | "schema_force_view_types" => Ok(GeneratedField::SchemaForceViewTypes), "binaryAsString" | "binary_as_string" => Ok(GeneratedField::BinaryAsString), "skipArrowMetadata" | "skip_arrow_metadata" => Ok(GeneratedField::SkipArrowMetadata), + "allowMorselDriven" | "allow_morsel_driven" => Ok(GeneratedField::AllowMorselDriven), "dictionaryPageSizeLimit" | "dictionary_page_size_limit" => Ok(GeneratedField::DictionaryPageSizeLimit), "dataPageRowCountLimit" | "data_page_row_count_limit" => Ok(GeneratedField::DataPageRowCountLimit), "maxRowGroupSize" | "max_row_group_size" => Ok(GeneratedField::MaxRowGroupSize), @@ -6089,6 +6099,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut schema_force_view_types__ = None; let mut binary_as_string__ = None; let mut skip_arrow_metadata__ = None; + let mut allow_morsel_driven__ = None; let mut dictionary_page_size_limit__ = None; let mut data_page_row_count_limit__ = None; let mut max_row_group_size__ = None; @@ -6216,6 +6227,12 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } skip_arrow_metadata__ = Some(map_.next_value()?); } + GeneratedField::AllowMorselDriven => { + if allow_morsel_driven__.is_some() { + return Err(serde::de::Error::duplicate_field("allowMorselDriven")); + } + allow_morsel_driven__ = Some(map_.next_value()?); + } GeneratedField::DictionaryPageSizeLimit => { if dictionary_page_size_limit__.is_some() { return Err(serde::de::Error::duplicate_field("dictionaryPageSizeLimit")); @@ -6332,6 +6349,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { schema_force_view_types: schema_force_view_types__.unwrap_or_default(), binary_as_string: binary_as_string__.unwrap_or_default(), skip_arrow_metadata: skip_arrow_metadata__.unwrap_or_default(), + allow_morsel_driven: allow_morsel_driven__.unwrap_or_default(), dictionary_page_size_limit: dictionary_page_size_limit__.unwrap_or_default(), data_page_row_count_limit: data_page_row_count_limit__.unwrap_or_default(), max_row_group_size: max_row_group_size__.unwrap_or_default(), diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index a09826a29be52..739bd28188fa7 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -830,6 +830,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, + /// default = true + #[prost(bool, tag = "35")] + pub allow_morsel_driven: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index a09826a29be52..739bd28188fa7 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -830,6 +830,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, + /// default = true + #[prost(bool, tag = "35")] + pub allow_morsel_driven: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] From d0da5daa22da51f01e6fe0187abfca43eb3e1d7f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 22 Feb 2026 18:55:57 +0100 Subject: [PATCH 03/78] Proto --- .../common/src/file_options/parquet_writer.rs | 1 + .../src/datasource/physical_plan/parquet.rs | 21 ++++++++++++------- datafusion/datasource/src/file_stream.rs | 8 +++++-- datafusion/proto-common/src/from_proto/mod.rs | 1 - datafusion/proto-common/src/to_proto/mod.rs | 1 + 5 files changed, 21 insertions(+), 11 deletions(-) diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 95fedd25aaa72..d00c22adc6559 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -461,6 +461,7 @@ mod tests { skip_arrow_metadata: defaults.skip_arrow_metadata, coerce_int96: None, max_predicate_cache_size: defaults.max_predicate_cache_size, + allow_morsel_driven: defaults.allow_morsel_driven, } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 697113b093c4f..8d3a52175169c 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -50,7 +50,9 @@ mod tests { use datafusion_common::test_util::{batches_to_sort_string, batches_to_string}; use datafusion_common::{Result, ScalarValue, assert_contains}; use datafusion_datasource::file_format::FileFormat; - use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; + use datafusion_datasource::file_scan_config::{ + FileScanConfig, FileScanConfigBuilder, + }; use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::file::FileSource; @@ -2470,8 +2472,7 @@ mod tests { ctx.register_object_store(store_url.as_ref(), store.clone()); // Create a Parquet file with 100 row groups, each with 10 rows - let schema = - Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); let mut out = Vec::new(); let props = WriterProperties::builder() @@ -2498,9 +2499,7 @@ mod tests { // Set up DataSourceExec with 2 partitions, but the file is only in partition 0 (skewed) let source = Arc::new(ParquetSource::new(schema)); let config = FileScanConfigBuilder::new(store_url, source) - .with_file_group(FileGroup::new(vec![PartitionedFile::new_from_meta( - meta, - )])) + .with_file_group(FileGroup::new(vec![PartitionedFile::new_from_meta(meta)])) .with_file_group(FileGroup::new(vec![])) // Partition 1 is empty .with_morsel_driven(true) .build(); @@ -2557,7 +2556,10 @@ mod tests { while let Some(batch) = s1.next().await { count += batch.unwrap().num_rows(); } - assert_eq!(count, 1000, "Second execution should also produce 1000 rows"); + assert_eq!( + count, 1000, + "Second execution should also produce 1000 rows" + ); Ok(()) } @@ -2600,7 +2602,10 @@ mod tests { .downcast_ref::() .expect("Expected FileScanConfig"); - assert!(config.morsel_driven, "morsel_driven should be enabled by default for Parquet"); + assert!( + config.morsel_driven, + "morsel_driven should be enabled by default for Parquet" + ); Ok(()) } diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 23d85b8263e8d..c89a92f758dc8 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -186,7 +186,9 @@ impl FileStream { } } else { match self.start_next_file().transpose() { - Ok(Some(future)) => self.state = FileStreamState::Open { future }, + Ok(Some(future)) => { + self.state = FileStreamState::Open { future } + } Ok(None) => return Poll::Ready(None), Err(e) => { self.state = FileStreamState::Error; @@ -211,7 +213,9 @@ impl FileStream { // No further expansion possible. Proceed to open. let morsel = morsels.into_iter().next().unwrap(); match self.file_opener.open(morsel) { - Ok(future) => self.state = FileStreamState::Open { future }, + Ok(future) => { + self.state = FileStreamState::Open { future } + } Err(e) => { self.file_stream_metrics.time_opening.stop(); self.state = FileStreamState::Error; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index 2b8740d7a0420..387aeda657e1f 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -1089,7 +1089,6 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { skip_arrow_metadata: value.skip_arrow_metadata, max_predicate_cache_size: value.max_predicate_cache_size_opt.map(|opt| match opt { protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v) => Some(v as usize), - }).unwrap_or(None), allow_morsel_driven: value.allow_morsel_driven, }) diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 79e3306a4df1b..7afe1d57c8940 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -904,6 +904,7 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { skip_arrow_metadata: value.skip_arrow_metadata, coerce_int96_opt: value.coerce_int96.clone().map(protobuf::parquet_options::CoerceInt96Opt::CoerceInt96), max_predicate_cache_size_opt: value.max_predicate_cache_size.map(|v| protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v as u64)), + allow_morsel_driven: value.allow_morsel_driven, }) } } From 32eec3cbf67494ee12897a5d3fb1c50efd58f57a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 22 Feb 2026 21:22:35 +0100 Subject: [PATCH 04/78] Fmt --- datafusion/datasource-parquet/src/opener.rs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 5525271ba0836..18e62a7bad87e 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -52,12 +52,11 @@ use datafusion_physical_plan::metrics::{ use datafusion_pruning::{FilePruner, PruningPredicate, build_pruning_predicate}; use crate::sort::reverse_row_selection; -use futures::future::{BoxFuture, ready}; -use parquet::file::metadata::ParquetMetaData; #[cfg(feature = "parquet_encryption")] use datafusion_common::config::EncryptionFactoryOptions; #[cfg(feature = "parquet_encryption")] use datafusion_execution::parquet_encryption::EncryptionFactory; +use futures::future::{BoxFuture, ready}; use futures::{Stream, StreamExt, TryStreamExt, ready}; use log::debug; use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; @@ -66,6 +65,7 @@ use parquet::arrow::arrow_reader::{ }; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; +use parquet::file::metadata::ParquetMetaData; use parquet::file::metadata::{PageIndexPolicy, ParquetMetaDataReader, RowGroupMetaData}; /// Implements [`FileOpener`] for a parquet file @@ -270,7 +270,8 @@ impl FileOpener for ParquetOpener { &predicate_creation_errors, ); - let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(num_row_groups)); + let mut row_groups = + RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(num_row_groups)); if let Some(predicate) = pruning_predicate { row_groups.prune_by_statistics( &physical_file_schema, @@ -485,7 +486,10 @@ impl FileOpener for ParquetOpener { .as_ref() .and_then(|e| e.downcast_ref::()) { - ArrowReaderMetadata::try_new(Arc::clone(&morsel.metadata), options.clone())? + ArrowReaderMetadata::try_new( + Arc::clone(&morsel.metadata), + options.clone(), + )? } else { ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) .await? From cc73788989d0c7223f63535d9adfcb827df1011b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 22 Feb 2026 21:26:04 +0100 Subject: [PATCH 05/78] Proto --- datafusion/proto/src/logical_plan/file_formats.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 08f42b0af7290..573dc533cdf57 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -426,6 +426,7 @@ mod parquet { max_predicate_cache_size_opt: global_options.global.max_predicate_cache_size.map(|size| { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size as u64) }), + allow_morsel_driven: global_options.global.allow_morsel_driven, }), column_specific_options: column_specific_options.into_iter().map(|(column_name, options)| { ParquetColumnSpecificOptions { @@ -525,6 +526,7 @@ mod parquet { max_predicate_cache_size: proto.max_predicate_cache_size_opt.as_ref().map(|opt| match opt { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size) => *size as usize, }), + allow_morsel_driven: proto.allow_morsel_driven, } } } From d517b5d95db8aedfdec20a3c0b680ac46d15a88b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 22 Feb 2026 22:37:50 +0100 Subject: [PATCH 06/78] Fix --- datafusion/datasource-parquet/src/opener.rs | 15 +++++- datafusion/datasource/src/file_scan_config.rs | 25 +++++++-- datafusion/datasource/src/file_stream.rs | 51 ++++++++++++++++--- 3 files changed, 76 insertions(+), 15 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 18e62a7bad87e..66697ab9627da 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -206,6 +206,9 @@ impl FileOpener for ParquetOpener { partitioned_file.object_meta.location.as_ref(), &self.metrics, ); + let file_name = partitioned_file.object_meta.location.to_string(); + let file_range = partitioned_file.range.clone(); + let extensions = partitioned_file.extensions.clone(); let metadata_size_hint = partitioned_file .metadata_size_hint @@ -270,8 +273,16 @@ impl FileOpener for ParquetOpener { &predicate_creation_errors, ); - let mut row_groups = - RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(num_row_groups)); + let mut row_groups = RowGroupAccessPlanFilter::new(create_initial_plan( + &file_name, + extensions, + num_row_groups, + )?); + + if let Some(range) = file_range.as_ref() { + row_groups.prune_by_range(metadata.row_groups(), range); + } + if let Some(predicate) = pruning_predicate { row_groups.prune_by_statistics( &physical_file_schema, diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 7351f54f6462e..0c9a404cce588 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -19,6 +19,7 @@ //! file sources. use crate::file_groups::FileGroup; +use crate::file_stream::WorkQueue; use crate::{ PartitionedFile, display::FileGroupsDisplay, file::FileSource, file_compression_type::FileCompressionType, file_stream::FileStream, @@ -53,7 +54,8 @@ use datafusion_physical_plan::{ metrics::ExecutionPlanMetricsSet, }; use log::{debug, warn}; -use std::sync::{Mutex, Weak}; +use std::sync::atomic::AtomicUsize; +use std::sync::Mutex; use std::{any::Any, fmt::Debug, fmt::Formatter, fmt::Result as FmtResult, sync::Arc}; /// [`FileScanConfig`] represents scanning data from a group of files @@ -209,9 +211,13 @@ pub struct FileScanConfig { /// This means all partitions share a single pool of work. pub morsel_driven: bool, /// Shared work queue for morsel-driven execution. - /// This uses a Weak pointer to allow the queue to be dropped when all execution - /// partitions are finished, supporting re-executability of the physical plan. - pub(crate) morsel_queue: Arc>>, + pub(crate) morsel_queue: Arc>>, + /// Number of morsel streams opened in the current execution cycle. + pub(crate) morsel_queue_streams_opened: Arc, + /// Number of active morsel streams in the current execution cycle. + pub(crate) morsel_queue_active_streams: Arc, + /// Expected number of streams in a full execution cycle. + pub(crate) morsel_queue_expected_streams: usize, } /// A builder for [`FileScanConfig`]'s. @@ -549,6 +555,12 @@ impl FileScanConfigBuilder { // If there is an output ordering, we should preserve it. let preserve_order = preserve_order || !output_ordering.is_empty(); + let morsel_queue_expected_streams = file_groups.len(); + + let all_files = file_groups + .iter() + .flat_map(|g| g.files().to_vec()) + .collect(); FileScanConfig { object_store_url, @@ -564,7 +576,10 @@ impl FileScanConfigBuilder { statistics, partitioned_by_file_group, morsel_driven, - morsel_queue: Arc::new(Mutex::new(Weak::new())), + morsel_queue: Arc::new(Mutex::new(Arc::new(WorkQueue::new(all_files)))), + morsel_queue_streams_opened: Arc::new(AtomicUsize::new(0)), + morsel_queue_active_streams: Arc::new(AtomicUsize::new(0)), + morsel_queue_expected_streams, } } } diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index c89a92f758dc8..0e4863889d545 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -82,6 +82,12 @@ pub struct FileStream { on_error: OnError, /// Guard for morselizing state to ensure counter is decremented on drop morsel_guard: Option, + /// Number of streams opened in current morsel execution cycle. + morsel_streams_opened: Option>, + /// Number of currently active morsel streams. + morsel_active_streams: Option>, + /// Expected streams in a full morsel execution cycle. + morsel_expected_streams: usize, } impl FileStream { @@ -95,19 +101,24 @@ impl FileStream { let projected_schema = config.projected_schema()?; let (file_iter, shared_queue) = if config.morsel_driven { - let mut guard = config.morsel_queue.lock().unwrap(); - let queue = if let Some(queue) = guard.upgrade() { - queue - } else { + let opened = config + .morsel_queue_streams_opened + .fetch_add(1, Ordering::SeqCst); + config + .morsel_queue_active_streams + .fetch_add(1, Ordering::SeqCst); + + if opened == 0 { let all_files = config .file_groups .iter() .flat_map(|g| g.files().to_vec()) .collect(); - let queue = Arc::new(WorkQueue::new(all_files)); - *guard = Arc::downgrade(&queue); - queue - }; + let mut guard = config.morsel_queue.lock().unwrap(); + *guard = Arc::new(WorkQueue::new(all_files)); + } + + let queue = Arc::clone(&config.morsel_queue.lock().unwrap()); (VecDeque::new(), Some(queue)) } else { let file_group = config.file_groups[partition].clone(); @@ -126,6 +137,13 @@ impl FileStream { baseline_metrics: BaselineMetrics::new(metrics, partition), on_error: OnError::Fail, morsel_guard: None, + morsel_streams_opened: config + .morsel_driven + .then(|| Arc::clone(&config.morsel_queue_streams_opened)), + morsel_active_streams: config + .morsel_driven + .then(|| Arc::clone(&config.morsel_queue_active_streams)), + morsel_expected_streams: config.morsel_queue_expected_streams, }) } @@ -386,6 +404,23 @@ impl FileStream { } } +impl Drop for FileStream { + fn drop(&mut self) { + let (Some(opened), Some(active)) = ( + self.morsel_streams_opened.as_ref(), + self.morsel_active_streams.as_ref(), + ) else { + return; + }; + + if active.fetch_sub(1, Ordering::SeqCst) == 1 + && opened.load(Ordering::SeqCst) == self.morsel_expected_streams + { + opened.store(0, Ordering::SeqCst); + } + } +} + impl Stream for FileStream { type Item = Result; From de1606dcd3d8a9e12e2cf93904a9284c42faed5b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 22 Feb 2026 22:38:12 +0100 Subject: [PATCH 07/78] Fix --- datafusion/datasource/src/file_scan_config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 0c9a404cce588..0ba371cea7a89 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -54,8 +54,8 @@ use datafusion_physical_plan::{ metrics::ExecutionPlanMetricsSet, }; use log::{debug, warn}; -use std::sync::atomic::AtomicUsize; use std::sync::Mutex; +use std::sync::atomic::AtomicUsize; use std::{any::Any, fmt::Debug, fmt::Formatter, fmt::Result as FmtResult, sync::Arc}; /// [`FileScanConfig`] represents scanning data from a group of files From 950f6db30ac4ebe19239f03690ba8300a2b8216f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 22 Feb 2026 22:47:10 +0100 Subject: [PATCH 08/78] Clippy --- datafusion/core/src/datasource/physical_plan/parquet.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 8d3a52175169c..bee11c06ca073 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -49,6 +49,7 @@ mod tests { use datafusion_common::config::TableParquetOptions; use datafusion_common::test_util::{batches_to_sort_string, batches_to_string}; use datafusion_common::{Result, ScalarValue, assert_contains}; + use datafusion_common_runtime::SpawnedTask; use datafusion_datasource::file_format::FileFormat; use datafusion_datasource::file_scan_config::{ FileScanConfig, FileScanConfigBuilder, @@ -2511,7 +2512,7 @@ mod tests { let stream0 = exec.execute(0, Arc::clone(&task_ctx))?; let stream1 = exec.execute(1, Arc::clone(&task_ctx))?; - let handle0 = tokio::spawn(async move { + let handle0 = SpawnedTask::spawn(async move { let mut count = 0; let mut s = stream0; while let Some(batch) = s.next().await { @@ -2521,7 +2522,7 @@ mod tests { count }); - let handle1 = tokio::spawn(async move { + let handle1 = SpawnedTask::spawn(async move { let mut count = 0; let mut s = stream1; while let Some(batch) = s.next().await { From 7f5731770d3b6d4423dda5c59d8bc2d99d2a116a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 07:35:46 +0100 Subject: [PATCH 09/78] Refactor --- datafusion/datasource/src/file_scan_config.rs | 41 +++++------- datafusion/datasource/src/file_stream.rs | 53 +-------------- datafusion/datasource/src/memory.rs | 2 + datafusion/datasource/src/source.rs | 67 +++++++++++++++++-- 4 files changed, 83 insertions(+), 80 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 0ba371cea7a89..89dde06c4d957 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -19,11 +19,14 @@ //! file sources. use crate::file_groups::FileGroup; -use crate::file_stream::WorkQueue; use crate::{ - PartitionedFile, display::FileGroupsDisplay, file::FileSource, - file_compression_type::FileCompressionType, file_stream::FileStream, - source::DataSource, statistics::MinMaxStatistics, + PartitionedFile, + display::FileGroupsDisplay, + file::FileSource, + file_compression_type::FileCompressionType, + file_stream::{FileStream, WorkQueue}, + source::DataSource, + statistics::MinMaxStatistics, }; use arrow::datatypes::FieldRef; use arrow::datatypes::{DataType, Schema, SchemaRef}; @@ -54,8 +57,6 @@ use datafusion_physical_plan::{ metrics::ExecutionPlanMetricsSet, }; use log::{debug, warn}; -use std::sync::Mutex; -use std::sync::atomic::AtomicUsize; use std::{any::Any, fmt::Debug, fmt::Formatter, fmt::Result as FmtResult, sync::Arc}; /// [`FileScanConfig`] represents scanning data from a group of files @@ -210,14 +211,6 @@ pub struct FileScanConfig { /// When true, use morsel-driven execution to avoid data skew. /// This means all partitions share a single pool of work. pub morsel_driven: bool, - /// Shared work queue for morsel-driven execution. - pub(crate) morsel_queue: Arc>>, - /// Number of morsel streams opened in the current execution cycle. - pub(crate) morsel_queue_streams_opened: Arc, - /// Number of active morsel streams in the current execution cycle. - pub(crate) morsel_queue_active_streams: Arc, - /// Expected number of streams in a full execution cycle. - pub(crate) morsel_queue_expected_streams: usize, } /// A builder for [`FileScanConfig`]'s. @@ -555,13 +548,6 @@ impl FileScanConfigBuilder { // If there is an output ordering, we should preserve it. let preserve_order = preserve_order || !output_ordering.is_empty(); - let morsel_queue_expected_streams = file_groups.len(); - - let all_files = file_groups - .iter() - .flat_map(|g| g.files().to_vec()) - .collect(); - FileScanConfig { object_store_url, file_source, @@ -576,10 +562,6 @@ impl FileScanConfigBuilder { statistics, partitioned_by_file_group, morsel_driven, - morsel_queue: Arc::new(Mutex::new(Arc::new(WorkQueue::new(all_files)))), - morsel_queue_streams_opened: Arc::new(AtomicUsize::new(0)), - morsel_queue_active_streams: Arc::new(AtomicUsize::new(0)), - morsel_queue_expected_streams, } } } @@ -609,6 +591,7 @@ impl DataSource for FileScanConfig { &self, partition: usize, context: Arc, + shared_morsel_queue: Option>, ) -> Result { let object_store = context.runtime_env().object_store(&self.object_store_url)?; let batch_size = self @@ -619,7 +602,13 @@ impl DataSource for FileScanConfig { let opener = source.create_file_opener(object_store, self, partition)?; - let stream = FileStream::new(self, partition, opener, source.metrics())?; + let stream = FileStream::new( + self, + partition, + opener, + source.metrics(), + shared_morsel_queue, + )?; Ok(Box::pin(cooperative(stream))) } diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 0e4863889d545..1e0e8dfdc942c 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -82,12 +82,6 @@ pub struct FileStream { on_error: OnError, /// Guard for morselizing state to ensure counter is decremented on drop morsel_guard: Option, - /// Number of streams opened in current morsel execution cycle. - morsel_streams_opened: Option>, - /// Number of currently active morsel streams. - morsel_active_streams: Option>, - /// Expected streams in a full morsel execution cycle. - morsel_expected_streams: usize, } impl FileStream { @@ -97,29 +91,12 @@ impl FileStream { partition: usize, file_opener: Arc, metrics: &ExecutionPlanMetricsSet, + shared_queue: Option>, ) -> Result { let projected_schema = config.projected_schema()?; let (file_iter, shared_queue) = if config.morsel_driven { - let opened = config - .morsel_queue_streams_opened - .fetch_add(1, Ordering::SeqCst); - config - .morsel_queue_active_streams - .fetch_add(1, Ordering::SeqCst); - - if opened == 0 { - let all_files = config - .file_groups - .iter() - .flat_map(|g| g.files().to_vec()) - .collect(); - let mut guard = config.morsel_queue.lock().unwrap(); - *guard = Arc::new(WorkQueue::new(all_files)); - } - - let queue = Arc::clone(&config.morsel_queue.lock().unwrap()); - (VecDeque::new(), Some(queue)) + (VecDeque::new(), shared_queue) } else { let file_group = config.file_groups[partition].clone(); (file_group.into_inner().into_iter().collect(), None) @@ -137,13 +114,6 @@ impl FileStream { baseline_metrics: BaselineMetrics::new(metrics, partition), on_error: OnError::Fail, morsel_guard: None, - morsel_streams_opened: config - .morsel_driven - .then(|| Arc::clone(&config.morsel_queue_streams_opened)), - morsel_active_streams: config - .morsel_driven - .then(|| Arc::clone(&config.morsel_queue_active_streams)), - morsel_expected_streams: config.morsel_queue_expected_streams, }) } @@ -404,23 +374,6 @@ impl FileStream { } } -impl Drop for FileStream { - fn drop(&mut self) { - let (Some(opened), Some(active)) = ( - self.morsel_streams_opened.as_ref(), - self.morsel_active_streams.as_ref(), - ) else { - return; - }; - - if active.fetch_sub(1, Ordering::SeqCst) == 1 - && opened.load(Ordering::SeqCst) == self.morsel_expected_streams - { - opened.store(0, Ordering::SeqCst); - } - } -} - impl Stream for FileStream { type Item = Result; @@ -854,7 +807,7 @@ mod tests { .build(); let metrics_set = ExecutionPlanMetricsSet::new(); let file_stream = - FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set) + FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set, None) .unwrap() .with_on_error(on_error); diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 1d12bb3200309..90c217ca1047e 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -24,6 +24,7 @@ use std::ops::Deref; use std::slice::from_ref; use std::sync::Arc; +use crate::file_stream::WorkQueue; use crate::sink::DataSink; use crate::source::{DataSource, DataSourceExec}; @@ -80,6 +81,7 @@ impl DataSource for MemorySourceConfig { &self, partition: usize, _context: Arc, + _shared_morsel_queue: Option>, ) -> Result { Ok(Box::pin(cooperative( MemoryStream::try_new( diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index a4e27dac769af..7fef32605bf83 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::fmt; use std::fmt::{Debug, Formatter}; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use datafusion_physical_expr::projection::ProjectionExprs; use datafusion_physical_plan::execution_plan::{ @@ -36,6 +36,7 @@ use datafusion_physical_plan::{ use itertools::Itertools; use crate::file_scan_config::FileScanConfig; +use crate::file_stream::WorkQueue; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Result, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -124,6 +125,7 @@ pub trait DataSource: Send + Sync + Debug { &self, partition: usize, context: Arc, + shared_morsel_queue: Option>, ) -> Result; fn as_any(&self) -> &dyn Any; /// Format this source for display in explain plans @@ -231,6 +233,15 @@ pub struct DataSourceExec { data_source: Arc, /// Cached plan properties such as sort order cache: PlanProperties, + /// Shared morsel queue for current execution lifecycle. + morsel_state: Arc>, +} + +#[derive(Debug, Default)] +struct MorselState { + queue: Option>, + streams_opened: usize, + expected_streams: usize, } impl DisplayAs for DataSourceExec { @@ -300,7 +311,46 @@ impl ExecutionPlan for DataSourceExec { partition: usize, context: Arc, ) -> Result { - let stream = self.data_source.open(partition, Arc::clone(&context))?; + let shared_morsel_queue = if let Some(config) = + self.data_source.as_any().downcast_ref::() + { + if config.morsel_driven { + let mut state = self.morsel_state.lock().unwrap(); + + // Start a new cycle once all expected partition streams for the + // previous cycle have been opened. + if state.expected_streams > 0 + && state.streams_opened >= state.expected_streams + { + state.queue = None; + state.streams_opened = 0; + state.expected_streams = 0; + } + + if state.queue.is_none() { + let all_files = config + .file_groups + .iter() + .flat_map(|g| g.files().to_vec()) + .collect(); + state.queue = Some(Arc::new(WorkQueue::new(all_files))); + state.expected_streams = config.file_groups.len(); + } + + state.streams_opened += 1; + state.queue.as_ref().cloned() + } else { + None + } + } else { + None + }; + + let stream = self.data_source.open( + partition, + Arc::clone(&context), + shared_morsel_queue, + )?; let batch_size = context.session_config().batch_size(); log::debug!( "Batch splitting enabled for partition {partition}: batch_size={batch_size}" @@ -326,7 +376,11 @@ impl ExecutionPlan for DataSourceExec { let data_source = self.data_source.with_fetch(limit)?; let cache = self.cache.clone(); - Some(Arc::new(Self { data_source, cache })) + Some(Arc::new(Self { + data_source, + cache, + morsel_state: Arc::new(Mutex::new(MorselState::default())), + })) } fn fetch(&self) -> Option { @@ -416,7 +470,11 @@ impl DataSourceExec { // Default constructor for `DataSourceExec`, setting the `cooperative` flag to `true`. pub fn new(data_source: Arc) -> Self { let cache = Self::compute_properties(&data_source); - Self { data_source, cache } + Self { + data_source, + cache, + morsel_state: Arc::new(Mutex::new(MorselState::default())), + } } /// Return the source object @@ -427,6 +485,7 @@ impl DataSourceExec { pub fn with_data_source(mut self, data_source: Arc) -> Self { self.cache = Self::compute_properties(&data_source); self.data_source = data_source; + self.morsel_state = Arc::new(Mutex::new(MorselState::default())); self } From fd6d7fdb6f07c70b14ea98079ead48083dd05f7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 08:46:32 +0100 Subject: [PATCH 10/78] WIP --- .../examples/custom_data_source/csv_json_opener.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion-examples/examples/custom_data_source/csv_json_opener.rs b/datafusion-examples/examples/custom_data_source/csv_json_opener.rs index fc1130313e00c..36b81d0489984 100644 --- a/datafusion-examples/examples/custom_data_source/csv_json_opener.rs +++ b/datafusion-examples/examples/custom_data_source/csv_json_opener.rs @@ -81,7 +81,7 @@ async fn csv_opener() -> Result<()> { let mut result = vec![]; let mut stream = - FileStream::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new())?; + FileStream::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new(), None)?; while let Some(batch) = stream.next().await.transpose()? { result.push(batch); } @@ -142,6 +142,7 @@ async fn json_opener() -> Result<()> { 0, Arc::new(opener), &ExecutionPlanMetricsSet::new(), + None )?; let mut result = vec![]; while let Some(batch) = stream.next().await.transpose()? { From 37126bfa3887d3296993c3dbfce3d8703623a43c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 09:25:10 +0100 Subject: [PATCH 11/78] WIP --- .../examples/custom_data_source/csv_json_opener.rs | 11 ++++++++--- .../examples/custom_data_source/custom_datasource.rs | 6 ++++-- .../examples/data_io/json_shredding.rs | 1 + .../tests/physical_optimizer/partition_statistics.rs | 3 +++ 4 files changed, 16 insertions(+), 5 deletions(-) diff --git a/datafusion-examples/examples/custom_data_source/csv_json_opener.rs b/datafusion-examples/examples/custom_data_source/csv_json_opener.rs index 36b81d0489984..008cb7db88e2d 100644 --- a/datafusion-examples/examples/custom_data_source/csv_json_opener.rs +++ b/datafusion-examples/examples/custom_data_source/csv_json_opener.rs @@ -80,8 +80,13 @@ async fn csv_opener() -> Result<()> { .create_file_opener(object_store, &scan_config, 0)?; let mut result = vec![]; - let mut stream = - FileStream::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new(), None)?; + let mut stream = FileStream::new( + &scan_config, + 0, + opener, + &ExecutionPlanMetricsSet::new(), + None, + )?; while let Some(batch) = stream.next().await.transpose()? { result.push(batch); } @@ -142,7 +147,7 @@ async fn json_opener() -> Result<()> { 0, Arc::new(opener), &ExecutionPlanMetricsSet::new(), - None + None, )?; let mut result = vec![]; while let Some(batch) = stream.next().await.transpose()? { diff --git a/datafusion-examples/examples/custom_data_source/custom_datasource.rs b/datafusion-examples/examples/custom_data_source/custom_datasource.rs index b276ae32cf247..9d96b5e021288 100644 --- a/datafusion-examples/examples/custom_data_source/custom_datasource.rs +++ b/datafusion-examples/examples/custom_data_source/custom_datasource.rs @@ -31,6 +31,7 @@ use datafusion::datasource::{TableProvider, TableType, provider_as_source}; use datafusion::error::Result; use datafusion::execution::context::TaskContext; use datafusion::logical_expr::LogicalPlanBuilder; +use datafusion::optimizer::OptimizerConfig; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::memory::MemoryStream; @@ -62,8 +63,9 @@ async fn search_accounts( expected_result_length: usize, ) -> Result<()> { // create local execution context - let ctx = SessionContext::new(); - + let mut config = SessionConfig::new() + .set("datafusion.execution.parquet.allow_morsel_driven", "false"); + let ctx = SessionContext::new_with_config(config); // create logical plan composed of a single TableScan let logical_plan = LogicalPlanBuilder::scan_with_filters( "accounts", diff --git a/datafusion-examples/examples/data_io/json_shredding.rs b/datafusion-examples/examples/data_io/json_shredding.rs index 77dba5a98ac6f..1040b7d3df04e 100644 --- a/datafusion-examples/examples/data_io/json_shredding.rs +++ b/datafusion-examples/examples/data_io/json_shredding.rs @@ -93,6 +93,7 @@ pub async fn json_shredding() -> Result<()> { // Set up query execution let mut cfg = SessionConfig::new(); cfg.options_mut().execution.parquet.pushdown_filters = true; + cfg.options_mut().execution.parquet.allow_morsel_driven = false; let ctx = SessionContext::new_with_config(cfg); ctx.runtime_env().register_object_store( ObjectStoreUrl::parse("memory://")?.as_ref(), diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index fa021ed3dcce3..3187913394f5b 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -78,6 +78,9 @@ mod test { target_partition: Option, ) -> Arc { let mut session_config = SessionConfig::new().with_collect_statistics(true); + session_config + .set_bool("datafusion.execution.parquet.allow_morsel_driven", true) + .unwrap(); if let Some(partition) = target_partition { session_config = session_config.with_target_partitions(partition); } From 2d3c33ed66a0952a945a8d892b3e3847845fdaea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 09:31:08 +0100 Subject: [PATCH 12/78] WIP --- .../examples/custom_data_source/custom_datasource.rs | 2 +- .../core/tests/physical_optimizer/partition_statistics.rs | 7 +++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/datafusion-examples/examples/custom_data_source/custom_datasource.rs b/datafusion-examples/examples/custom_data_source/custom_datasource.rs index 9d96b5e021288..0ae089f6c35e4 100644 --- a/datafusion-examples/examples/custom_data_source/custom_datasource.rs +++ b/datafusion-examples/examples/custom_data_source/custom_datasource.rs @@ -64,7 +64,7 @@ async fn search_accounts( ) -> Result<()> { // create local execution context let mut config = SessionConfig::new() - .set("datafusion.execution.parquet.allow_morsel_driven", "false"); + .set_bool("datafusion.execution.parquet.allow_morsel_driven", false); let ctx = SessionContext::new_with_config(config); // create logical plan composed of a single TableScan let logical_plan = LogicalPlanBuilder::scan_with_filters( diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index 3187913394f5b..c05dc391cf415 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -77,10 +77,9 @@ mod test { create_table_sql: Option<&str>, target_partition: Option, ) -> Arc { - let mut session_config = SessionConfig::new().with_collect_statistics(true); - session_config - .set_bool("datafusion.execution.parquet.allow_morsel_driven", true) - .unwrap(); + let mut session_config = SessionConfig::new() + .with_collect_statistics(true) + .set_bool("datafusion.execution.parquet.allow_morsel_driven", true); if let Some(partition) = target_partition { session_config = session_config.with_target_partitions(partition); } From 98f0ea9c8802faaeee8a4e26a2c8baf6e741f88b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 09:35:47 +0100 Subject: [PATCH 13/78] WIP --- .../examples/custom_data_source/custom_datasource.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-examples/examples/custom_data_source/custom_datasource.rs b/datafusion-examples/examples/custom_data_source/custom_datasource.rs index 0ae089f6c35e4..988246313f101 100644 --- a/datafusion-examples/examples/custom_data_source/custom_datasource.rs +++ b/datafusion-examples/examples/custom_data_source/custom_datasource.rs @@ -63,7 +63,7 @@ async fn search_accounts( expected_result_length: usize, ) -> Result<()> { // create local execution context - let mut config = SessionConfig::new() + let config = SessionConfig::new() .set_bool("datafusion.execution.parquet.allow_morsel_driven", false); let ctx = SessionContext::new_with_config(config); // create logical plan composed of a single TableScan From a389b0287d878753fa07d7526443b59fdeb40729 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 09:45:45 +0100 Subject: [PATCH 14/78] WIP --- .../core/tests/physical_optimizer/partition_statistics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index c05dc391cf415..b04090f0dc813 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -79,7 +79,7 @@ mod test { ) -> Arc { let mut session_config = SessionConfig::new() .with_collect_statistics(true) - .set_bool("datafusion.execution.parquet.allow_morsel_driven", true); + .set_bool("datafusion.execution.parquet.allow_morsel_driven", false); if let Some(partition) = target_partition { session_config = session_config.with_target_partitions(partition); } From 406544866238c54ac1876baa888aad051b7f7049 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 09:51:04 +0100 Subject: [PATCH 15/78] Update --- datafusion/core/tests/sql/explain_analyze.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 5f62f7204eff1..ea3eb84ba74a2 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -293,7 +293,7 @@ async fn explain_analyze_parquet_pruning_metrics() { // select * from tpch_lineitem_small where l_orderkey = 5; // If change filter to `l_orderkey=10`, the whole file can be pruned using stat. for (l_orderkey, expected_pruning_metrics) in - [(5, "1 total → 1 matched"), (10, "1 total → 0 matched")] + [(5, "2 total → 2 matched"), (10, "1 total → 0 matched")] { let sql = format!( "explain analyze select * from {table_name} where l_orderkey = {l_orderkey};" @@ -303,7 +303,7 @@ async fn explain_analyze_parquet_pruning_metrics() { collect_plan_with_context(&sql, &ctx, ExplainAnalyzeLevel::Summary).await; let expected_metrics = - format!("files_ranges_pruned_statistics={expected_pruning_metrics}"); + format!("row_groups_pruned_statistics={expected_pruning_metrics}"); assert_metrics!(&plan, "DataSourceExec", &expected_metrics); } From 415315d9e049f5f1b4b4779b78e9527d19db6dc2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 09:57:52 +0100 Subject: [PATCH 16/78] Update --- .../examples/custom_data_source/custom_datasource.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion-examples/examples/custom_data_source/custom_datasource.rs b/datafusion-examples/examples/custom_data_source/custom_datasource.rs index 988246313f101..f86c749ce6556 100644 --- a/datafusion-examples/examples/custom_data_source/custom_datasource.rs +++ b/datafusion-examples/examples/custom_data_source/custom_datasource.rs @@ -31,7 +31,6 @@ use datafusion::datasource::{TableProvider, TableType, provider_as_source}; use datafusion::error::Result; use datafusion::execution::context::TaskContext; use datafusion::logical_expr::LogicalPlanBuilder; -use datafusion::optimizer::OptimizerConfig; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::memory::MemoryStream; From 13b4977bec2a5055980050beeba705eecfb112d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 12:35:37 +0100 Subject: [PATCH 17/78] Config --- docs/source/user-guide/configs.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index e48f0a7c92276..8a8c6983444b2 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -91,6 +91,7 @@ The following configuration settings are available: | datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | | datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | | datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | +| datafusion.execution.parquet.allow_morsel_driven | true | (reading) If true, the parquet reader will share work between partitions using morsel-driven execution. This can help mitigate data skew. | | datafusion.execution.parquet.max_predicate_cache_size | NULL | (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. | | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in rows | From a30c3f85113f08a08f998cce8100c4194e384890 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 12:53:04 +0100 Subject: [PATCH 18/78] Test --- datafusion/core/tests/sql/explain_analyze.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index ea3eb84ba74a2..799145c6f0340 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -869,7 +869,7 @@ async fn parquet_explain_analyze() { ); assert_contains!( &formatted, - "row_groups_pruned_statistics=1 total \u{2192} 1 matched" + "row_groups_pruned_statistics=2 total \u{2192} 2 matched" ); assert_contains!(&formatted, "scan_efficiency_ratio=14%"); From 8b32ca8dab1de160b106412307431b52c789fdb2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 14:21:02 +0100 Subject: [PATCH 19/78] Refactor --- datafusion/datasource-parquet/src/opener.rs | 228 +++++++++++++++++--- 1 file changed, 193 insertions(+), 35 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 66697ab9627da..1c285eb4bfcb7 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -40,7 +40,7 @@ use datafusion_common::stats::Precision; use datafusion_common::{ ColumnStatistics, DataFusionError, Result, ScalarValue, Statistics, exec_err, }; -use datafusion_datasource::{PartitionedFile, TableSchema}; +use datafusion_datasource::{FileRange, PartitionedFile, TableSchema}; use datafusion_physical_expr::simplifier::PhysicalExprSimplifier; use datafusion_physical_expr_adapter::PhysicalExprAdapterFactory; use datafusion_physical_expr_common::physical_expr::{ @@ -67,6 +67,7 @@ use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::file::metadata::ParquetMetaData; use parquet::file::metadata::{PageIndexPolicy, ParquetMetaDataReader, RowGroupMetaData}; +use parquet::schema::types::SchemaDescriptor; /// Implements [`FileOpener`] for a parquet file pub(super) struct ParquetOpener { @@ -140,6 +141,13 @@ pub(crate) struct PreparedAccessPlan { pub(crate) row_selection: Option, } +struct RowGroupStatisticsPruningContext<'a> { + physical_file_schema: &'a SchemaRef, + parquet_schema: &'a SchemaDescriptor, + predicate: &'a PruningPredicate, + file_metrics: &'a ParquetFileMetrics, +} + impl PreparedAccessPlan { /// Create a new prepared access plan from a ParquetAccessPlan pub(crate) fn from_access_plan( @@ -187,6 +195,39 @@ impl PreparedAccessPlan { } } +impl ParquetOpener { + fn build_row_group_access_filter( + file_name: &str, + extensions: Option>, + row_group_count: usize, + row_group_metadata: &[RowGroupMetaData], + file_range: Option<&FileRange>, + stats_pruning: Option>, + ) -> Result { + let mut row_groups = RowGroupAccessPlanFilter::new(create_initial_plan( + file_name, + extensions, + row_group_count, + )?); + + if let Some(range) = file_range { + row_groups.prune_by_range(row_group_metadata, range); + } + + if let Some(stats_pruning) = stats_pruning { + row_groups.prune_by_statistics( + stats_pruning.physical_file_schema.as_ref(), + stats_pruning.parquet_schema, + row_group_metadata, + stats_pruning.predicate, + stats_pruning.file_metrics, + ); + } + + Ok(row_groups) + } +} + impl FileOpener for ParquetOpener { fn morselize( &self, @@ -233,6 +274,7 @@ impl FileOpener for ParquetOpener { let table_schema = self.table_schema.clone(); let predicate = self.predicate.clone(); let metrics = self.metrics.clone(); + let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; Box::pin(async move { #[cfg(feature = "parquet_encryption")] @@ -273,25 +315,23 @@ impl FileOpener for ParquetOpener { &predicate_creation_errors, ); - let mut row_groups = RowGroupAccessPlanFilter::new(create_initial_plan( + let row_groups = Self::build_row_group_access_filter( &file_name, extensions, num_row_groups, - )?); - - if let Some(range) = file_range.as_ref() { - row_groups.prune_by_range(metadata.row_groups(), range); - } + metadata.row_groups(), + file_range.as_ref(), + pruning_predicate + .as_deref() + .filter(|_| enable_row_group_stats_pruning) + .map(|predicate| RowGroupStatisticsPruningContext { + physical_file_schema: &physical_file_schema, + parquet_schema: reader_metadata.parquet_schema(), + predicate, + file_metrics: &file_metrics, + }), + )?; - if let Some(predicate) = pruning_predicate { - row_groups.prune_by_statistics( - &physical_file_schema, - reader_metadata.parquet_schema(), - metadata.row_groups(), - predicate.as_ref(), - &file_metrics, - ); - } let access_plan = row_groups.build(); let mut morsels = Vec::with_capacity(access_plan.len()); @@ -412,6 +452,11 @@ impl FileOpener for ParquetOpener { let reverse_row_groups = self.reverse_row_groups; let preserve_order = self.preserve_order; + let is_morsel = partitioned_file + .extensions + .as_ref() + .map(|e| e.is::()) + .unwrap_or(false); Ok(Box::pin(async move { #[cfg(feature = "parquet_encryption")] @@ -638,26 +683,25 @@ impl FileOpener for ParquetOpener { let file_metadata = Arc::clone(builder.metadata()); let predicate = pruning_predicate.as_ref().map(|p| p.as_ref()); let rg_metadata = file_metadata.row_groups(); - // track which row groups to actually read - let access_plan = - create_initial_plan(&file_name, extensions, rg_metadata.len())?; - let mut row_groups = RowGroupAccessPlanFilter::new(access_plan); - // if there is a range restricting what parts of the file to read - if let Some(range) = file_range.as_ref() { - row_groups.prune_by_range(rg_metadata, range); - } + let mut row_groups = Self::build_row_group_access_filter( + &file_name, + extensions, + rg_metadata.len(), + rg_metadata, + file_range.as_ref(), + predicate + .filter(|_| enable_row_group_stats_pruning && !is_morsel) + .map(|predicate| RowGroupStatisticsPruningContext { + physical_file_schema: &physical_file_schema, + parquet_schema: builder.parquet_schema(), + predicate, + file_metrics: &file_metrics, + }), + )?; // If there is a predicate that can be evaluated against the metadata if let Some(predicate) = predicate.as_ref() { - if enable_row_group_stats_pruning { - row_groups.prune_by_statistics( - &physical_file_schema, - builder.parquet_schema(), - rg_metadata, - predicate, - &file_metrics, - ); - } else { + if !enable_row_group_stats_pruning { // Update metrics: statistics unavailable, so all row groups are // matched (not pruned) file_metrics @@ -1168,7 +1212,10 @@ mod test { use std::sync::Arc; use super::{ConstantColumns, constant_columns_from_stats}; - use crate::{DefaultParquetFileReaderFactory, RowGroupAccess, opener::ParquetOpener}; + use crate::{ + DefaultParquetFileReaderFactory, RowGroupAccess, + opener::{ParquetMorsel, ParquetOpener}, + }; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use bytes::{BufMut, BytesMut}; use datafusion_common::{ @@ -1186,7 +1233,7 @@ mod test { use datafusion_physical_expr_adapter::{ DefaultPhysicalExprAdapterFactory, replace_columns_with_literals, }; - use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; + use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricValue}; use futures::{Stream, StreamExt}; use object_store::{ObjectStore, memory::InMemory, path::Path}; use parquet::arrow::ArrowWriter; @@ -1516,6 +1563,19 @@ mod test { )) } + fn get_pruning_metric( + metrics: &ExecutionPlanMetricsSet, + metric_name: &str, + ) -> (usize, usize) { + match metrics.clone_inner().sum_by_name(metric_name) { + Some(MetricValue::PruningMetrics { + pruning_metrics, .. + }) => (pruning_metrics.pruned(), pruning_metrics.matched()), + Some(_) => panic!("Metric '{metric_name}' is not a pruning metric"), + None => panic!("Metric '{metric_name}' not found"), + } + } + #[tokio::test] async fn test_prune_on_statistics() { let store = Arc::new(InMemory::new()) as Arc; @@ -2155,4 +2215,102 @@ mod test { "Reverse scan with non-contiguous row groups should correctly map RowSelection" ); } + + #[tokio::test] + async fn test_open_and_morselize_are_equivalent_except_for_morsels() { + use parquet::file::properties::WriterProperties; + + let store = Arc::new(InMemory::new()) as Arc; + + let batch1 = record_batch!(("a", Int32, vec![Some(1), Some(2)])).unwrap(); + let batch2 = record_batch!(("a", Int32, vec![Some(10), Some(11)])).unwrap(); + let batch3 = record_batch!(("a", Int32, vec![Some(20), Some(21)])).unwrap(); + + let props = WriterProperties::builder() + .set_max_row_group_size(2) + .build(); + + let data_len = write_parquet_batches( + Arc::clone(&store), + "test.parquet", + vec![batch1.clone(), batch2.clone(), batch3.clone()], + Some(props), + ) + .await; + + let schema = batch1.schema(); + let file = PartitionedFile::new( + "test.parquet".to_string(), + u64::try_from(data_len).unwrap(), + ); + + for enable_row_group_stats_pruning in [false, true] { + let expr = col("a").gt(lit(5)).and(col("a").lt(lit(20))); + let predicate = logical2physical(&expr, &schema); + + let baseline_opener = ParquetOpenerBuilder::new() + .with_store(Arc::clone(&store)) + .with_schema(Arc::clone(&schema)) + .with_projection_indices(&[0]) + .with_predicate(Arc::clone(&predicate)) + .with_row_group_stats_pruning(enable_row_group_stats_pruning) + .build(); + + // Baseline: regular open path + let stream = baseline_opener.open(file.clone()).unwrap().await.unwrap(); + let baseline_values = collect_int32_values(stream).await; + let baseline_stats_metrics = get_pruning_metric( + &baseline_opener.metrics, + "row_groups_pruned_statistics", + ); + + let morsel_opener = ParquetOpenerBuilder::new() + .with_store(Arc::clone(&store)) + .with_schema(Arc::clone(&schema)) + .with_projection_indices(&[0]) + .with_predicate(predicate) + .with_row_group_stats_pruning(enable_row_group_stats_pruning) + .build(); + + // Morsel path: split into morsels and open each morsel + let morsels = morsel_opener.morselize(file.clone()).await.unwrap(); + assert!( + !morsels.is_empty(), + "Expected at least one morsel for the selected row groups" + ); + + let mut morsel_values = vec![]; + for morsel_file in morsels { + let morsel = morsel_file + .extensions + .as_ref() + .and_then(|ext| ext.downcast_ref::()) + .expect("morselized file should carry ParquetMorsel extension"); + + assert_eq!( + morsel.access_plan.row_group_indexes().len(), + 1, + "each morsel should scan exactly one row group" + ); + + let stream = morsel_opener.open(morsel_file).unwrap().await.unwrap(); + morsel_values.extend(collect_int32_values(stream).await); + } + + let morsel_stats_metrics = get_pruning_metric( + &morsel_opener.metrics, + "row_groups_pruned_statistics", + ); + + assert_eq!( + baseline_values, morsel_values, + "open and morselize paths should scan equivalent data; morselize only changes work granularity" + ); + + assert_eq!( + baseline_stats_metrics, morsel_stats_metrics, + "row_groups_pruned_statistics should be equivalent for open vs morselize path (enable_row_group_stats_pruning={enable_row_group_stats_pruning})" + ); + } + } } From 876c29657b3e055b62d57e0e7f9fc7ff4e401221 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 17:22:48 +0100 Subject: [PATCH 20/78] Update test --- datafusion/core/tests/sql/explain_analyze.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 799145c6f0340..2442eb68379de 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -293,7 +293,7 @@ async fn explain_analyze_parquet_pruning_metrics() { // select * from tpch_lineitem_small where l_orderkey = 5; // If change filter to `l_orderkey=10`, the whole file can be pruned using stat. for (l_orderkey, expected_pruning_metrics) in - [(5, "2 total → 2 matched"), (10, "1 total → 0 matched")] + [(5, "1 total → 1 matched"), (10, "1 total → 0 matched")] { let sql = format!( "explain analyze select * from {table_name} where l_orderkey = {l_orderkey};" From d2df36b4d766a3efdff81115548c20e7abeefc94 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 17:24:38 +0100 Subject: [PATCH 21/78] Update test --- datafusion/core/tests/sql/explain_analyze.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 2442eb68379de..544a5a3a92122 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -869,7 +869,7 @@ async fn parquet_explain_analyze() { ); assert_contains!( &formatted, - "row_groups_pruned_statistics=2 total \u{2192} 2 matched" + "row_groups_pruned_statistics=1 total \u{2192} 1 matched" ); assert_contains!(&formatted, "scan_efficiency_ratio=14%"); From 869b7d3c6cc559c101668db4a1cce9c82625e235 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 19:18:44 +0100 Subject: [PATCH 22/78] Autofix --- .../core/tests/parquet/row_group_pruning.rs | 18 ++++++++++++------ datafusion/datasource-parquet/src/opener.rs | 9 ++++++++- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index 445ae7e97f228..810a6cfebb440 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -531,14 +531,18 @@ macro_rules! int_tests { #[tokio::test] async fn []() { // result of sql "SELECT * FROM t where in (1000)", prune all - // test whether statistics works + // test whether statistics works. + // With morsel-driven execution (default), the file is opened in + // morselize() to create per-row-group morsels. Row group statistics + // pruning prunes all 4 row groups there, so pruning is at row-group + // level, not file level. RowGroupPruningTest::new() .with_scenario(Scenario::Int) .with_query(&format!("SELECT * FROM t where i{} in (100)", $bits)) .with_expected_errors(Some(0)) .with_matched_by_stats(Some(0)) - .with_pruned_by_stats(Some(0)) - .with_pruned_files(Some(1)) + .with_pruned_by_stats(Some(4)) + .with_pruned_files(Some(0)) .with_matched_by_bloom_filter(Some(0)) .with_pruned_by_bloom_filter(Some(0)) .with_expected_rows(0) @@ -1483,14 +1487,16 @@ async fn test_row_group_with_null_values() { .test_row_group_prune() .await; - // All row groups will be pruned + // All row groups will be pruned. + // With morsel-driven execution (default), pruning happens at row-group level + // in morselize(), not at file level. All 3 row groups are pruned by statistics. RowGroupPruningTest::new() .with_scenario(Scenario::WithNullValues) .with_query("SELECT * FROM t WHERE \"i32\" > 7") .with_expected_errors(Some(0)) .with_matched_by_stats(Some(0)) - .with_pruned_by_stats(Some(0)) - .with_pruned_files(Some(1)) + .with_pruned_by_stats(Some(3)) + .with_pruned_files(Some(0)) .with_expected_rows(0) .with_matched_by_bloom_filter(Some(0)) .with_pruned_by_bloom_filter(Some(0)) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 1c285eb4bfcb7..4dec022d124db 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -275,6 +275,8 @@ impl FileOpener for ParquetOpener { let predicate = self.predicate.clone(); let metrics = self.metrics.clone(); let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; + let limit = self.limit; + let preserve_order = self.preserve_order; Box::pin(async move { #[cfg(feature = "parquet_encryption")] @@ -315,7 +317,7 @@ impl FileOpener for ParquetOpener { &predicate_creation_errors, ); - let row_groups = Self::build_row_group_access_filter( + let mut row_groups = Self::build_row_group_access_filter( &file_name, extensions, num_row_groups, @@ -332,6 +334,11 @@ impl FileOpener for ParquetOpener { }), )?; + // Prune by limit if limit is set and order is not sensitive + if let (Some(limit), false) = (limit, preserve_order) { + row_groups.prune_by_limit(limit, metadata.row_groups(), &file_metrics); + } + let access_plan = row_groups.build(); let mut morsels = Vec::with_capacity(access_plan.len()); From 67ea9ab2cfb281592e0182d355122a48343f3d4c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 19:24:16 +0100 Subject: [PATCH 23/78] Prune files --- .../core/tests/parquet/row_group_pruning.rs | 18 +++++---------- datafusion/datasource-parquet/src/opener.rs | 23 ++++++++++++++++--- 2 files changed, 26 insertions(+), 15 deletions(-) diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index 810a6cfebb440..445ae7e97f228 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -531,18 +531,14 @@ macro_rules! int_tests { #[tokio::test] async fn []() { // result of sql "SELECT * FROM t where in (1000)", prune all - // test whether statistics works. - // With morsel-driven execution (default), the file is opened in - // morselize() to create per-row-group morsels. Row group statistics - // pruning prunes all 4 row groups there, so pruning is at row-group - // level, not file level. + // test whether statistics works RowGroupPruningTest::new() .with_scenario(Scenario::Int) .with_query(&format!("SELECT * FROM t where i{} in (100)", $bits)) .with_expected_errors(Some(0)) .with_matched_by_stats(Some(0)) - .with_pruned_by_stats(Some(4)) - .with_pruned_files(Some(0)) + .with_pruned_by_stats(Some(0)) + .with_pruned_files(Some(1)) .with_matched_by_bloom_filter(Some(0)) .with_pruned_by_bloom_filter(Some(0)) .with_expected_rows(0) @@ -1487,16 +1483,14 @@ async fn test_row_group_with_null_values() { .test_row_group_prune() .await; - // All row groups will be pruned. - // With morsel-driven execution (default), pruning happens at row-group level - // in morselize(), not at file level. All 3 row groups are pruned by statistics. + // All row groups will be pruned RowGroupPruningTest::new() .with_scenario(Scenario::WithNullValues) .with_query("SELECT * FROM t WHERE \"i32\" > 7") .with_expected_errors(Some(0)) .with_matched_by_stats(Some(0)) - .with_pruned_by_stats(Some(3)) - .with_pruned_files(Some(0)) + .with_pruned_by_stats(Some(0)) + .with_pruned_files(Some(1)) .with_expected_rows(0) .with_matched_by_bloom_filter(Some(0)) .with_pruned_by_bloom_filter(Some(0)) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 4dec022d124db..95cb2f099d02b 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -289,6 +289,26 @@ impl FileOpener for ParquetOpener { options }; + let predicate_creation_errors = MetricBuilder::new(&metrics) + .global_counter("num_predicate_creation_errors"); + + // Step: try to prune the file using file-level statistics before loading + // parquet metadata. This avoids the I/O cost of reading metadata when + // file-level stats (available from the catalog) indicate no rows can match. + if let Some(pred) = predicate.as_ref() { + let logical_file_schema = Arc::clone(table_schema.file_schema()); + if let Some(mut file_pruner) = FilePruner::try_new( + Arc::clone(pred), + &logical_file_schema, + &partitioned_file, + predicate_creation_errors.clone(), + ) && file_pruner.should_prune()? + { + file_metrics.files_ranges_pruned_statistics.add_pruned(1); + return Ok(vec![]); + } + } + let mut _metadata_timer = file_metrics.metadata_load_time.timer(); let reader_metadata = ArrowReaderMetadata::load_async(&mut async_file_reader, options).await?; @@ -308,9 +328,6 @@ impl FileOpener for ParquetOpener { .map(|p| simplifier.simplify(rewriter.rewrite(Arc::clone(p))?)) .transpose()?; - let predicate_creation_errors = MetricBuilder::new(&metrics) - .global_counter("num_predicate_creation_errors"); - let (pruning_predicate, _) = build_pruning_predicates( adapted_predicate.as_ref(), &physical_file_schema, From e845675fbd32d306deeb94ca73ee65dc4c2f43fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 19:26:56 +0100 Subject: [PATCH 24/78] Update test --- datafusion/core/tests/sql/explain_analyze.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 544a5a3a92122..5f62f7204eff1 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -303,7 +303,7 @@ async fn explain_analyze_parquet_pruning_metrics() { collect_plan_with_context(&sql, &ctx, ExplainAnalyzeLevel::Summary).await; let expected_metrics = - format!("row_groups_pruned_statistics={expected_pruning_metrics}"); + format!("files_ranges_pruned_statistics={expected_pruning_metrics}"); assert_metrics!(&plan, "DataSourceExec", &expected_metrics); } From 6885981f7c8d56b50f8b7525c52733ffe5549533 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 20:36:02 +0100 Subject: [PATCH 25/78] Update test --- datafusion/datasource-parquet/src/opener.rs | 21 ++++++++++++++++++- .../physical-expr/src/simplifier/mod.rs | 19 +++++++++++------ .../test_files/information_schema.slt | 2 ++ .../sqllogictest/test_files/limit_pruning.slt | 4 ++-- 4 files changed, 37 insertions(+), 9 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 95cb2f099d02b..77aa0cd779614 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -323,9 +323,28 @@ impl FileOpener for ParquetOpener { Arc::clone(&physical_file_schema), )?; let simplifier = PhysicalExprSimplifier::new(&physical_file_schema); + + // Replace partition column references with their literal values before rewriting. + // This mirrors what `open()` does. Without this, expressions like `val != part` + // (where `part` is a partition column) would cause `rewriter.rewrite` to fail + // since the partition column is not in the logical file schema. + let literal_columns: HashMap = table_schema + .table_partition_cols() + .iter() + .zip(partitioned_file.partition_values.iter()) + .map(|(field, value)| (field.name().clone(), value.clone())) + .collect(); + let adapted_predicate = predicate .as_ref() - .map(|p| simplifier.simplify(rewriter.rewrite(Arc::clone(p))?)) + .map(|p| { + let p = if !literal_columns.is_empty() { + replace_columns_with_literals(Arc::clone(p), &literal_columns)? + } else { + Arc::clone(p) + }; + simplifier.simplify(rewriter.rewrite(p)?) + }) .transpose()?; let (pruning_predicate, _) = build_pruning_predicates( diff --git a/datafusion/physical-expr/src/simplifier/mod.rs b/datafusion/physical-expr/src/simplifier/mod.rs index 45ead82a0a93d..ce7339e1acc8c 100644 --- a/datafusion/physical-expr/src/simplifier/mod.rs +++ b/datafusion/physical-expr/src/simplifier/mod.rs @@ -63,7 +63,10 @@ impl<'a> PhysicalExprSimplifier<'a> { count += 1; let result = current_expr.transform(|node| { #[cfg(debug_assertions)] - let original_type = node.data_type(schema).unwrap(); + // Use `ok()` to skip the assertion when data_type fails (e.g., for + // DynamicFilterPhysicalExpr whose inner expression may reference columns + // outside the provided schema when the filter has been updated concurrently). + let original_type = node.data_type(schema).ok(); // Apply NOT expression simplification first, then unwrap cast optimization, // then constant expression evaluation @@ -74,11 +77,15 @@ impl<'a> PhysicalExprSimplifier<'a> { })?; #[cfg(debug_assertions)] - assert_eq!( - rewritten.data.data_type(schema).unwrap(), - original_type, - "Simplified expression should have the same data type as the original" - ); + if let Some(original_type) = original_type { + if let Ok(rewritten_type) = rewritten.data.data_type(schema) { + assert_eq!( + rewritten_type, + original_type, + "Simplified expression should have the same data type as the original" + ); + } + } Ok(rewritten) })?; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index b61ceecb24fc0..35aee28b949dd 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -228,6 +228,7 @@ datafusion.execution.max_spill_file_size_bytes 134217728 datafusion.execution.meta_fetch_concurrency 32 datafusion.execution.minimum_parallel_output_files 4 datafusion.execution.objectstore_writer_buffer_size 10485760 +datafusion.execution.parquet.allow_morsel_driven true datafusion.execution.parquet.allow_single_file_parallelism true datafusion.execution.parquet.binary_as_string false datafusion.execution.parquet.bloom_filter_fpp NULL @@ -366,6 +367,7 @@ datafusion.execution.max_spill_file_size_bytes 134217728 Maximum size in bytes f datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. 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.parquet.allow_morsel_driven true (reading) If true, the parquet reader will share work between partitions using morsel-driven execution. This can help mitigate data skew. datafusion.execution.parquet.allow_single_file_parallelism true (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. datafusion.execution.parquet.binary_as_string false (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. datafusion.execution.parquet.bloom_filter_fpp NULL (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index 72672b707d4f5..62790e6683049 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -63,7 +63,7 @@ set datafusion.explain.analyze_level = summary; query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 limit 3; ---- -Plan with Metrics DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (171/2.35 K)] +Plan with Metrics DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (171/2.35 K)] # limit_pruned_row_groups=0 total → 0 matched # because of order by, scan needs to preserve sort, so limit pruning is disabled @@ -72,7 +72,7 @@ explain analyze select * from tracking_data where species > 'M' AND s >= 50 orde ---- Plan with Metrics 01)SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=, output_bytes=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (521/2.35 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=7 total → 7 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (521/2.35 K)] statement ok drop table tracking_data; From 3384b8f00afcdda77aa7152c431f2dc431c6de21 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 21:04:43 +0100 Subject: [PATCH 26/78] Update morsel_driven --- datafusion/datasource/src/file_scan_config.rs | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 89dde06c4d957..3c153a203722f 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -548,6 +548,15 @@ impl FileScanConfigBuilder { // If there is an output ordering, we should preserve it. let preserve_order = preserve_order || !output_ordering.is_empty(); + + // Morsel-driven execution pools all files from all file groups into a shared + // work queue that any partition may consume. When `partitioned_by_file_group` + // is true the optimizer has declared Hash partitioning based on the assumption + // that partition N reads only from file_group[N]. Enabling morsel-driven in + // that case would break that guarantee (e.g. for `HashJoinExec: mode=Partitioned` + // downstream), so we force it off. + let morsel_driven = morsel_driven && !partitioned_by_file_group; + FileScanConfig { object_store_url, file_source, From 211d4fcd51df2b1164575e70398b00bb9a6ab2fd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 21:09:03 +0100 Subject: [PATCH 27/78] Update morsel_driven --- datafusion/datasource/src/file_scan_config.rs | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 3c153a203722f..84595775d8d0d 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -550,12 +550,21 @@ impl FileScanConfigBuilder { let preserve_order = preserve_order || !output_ordering.is_empty(); // Morsel-driven execution pools all files from all file groups into a shared - // work queue that any partition may consume. When `partitioned_by_file_group` - // is true the optimizer has declared Hash partitioning based on the assumption - // that partition N reads only from file_group[N]. Enabling morsel-driven in - // that case would break that guarantee (e.g. for `HashJoinExec: mode=Partitioned` - // downstream), so we force it off. - let morsel_driven = morsel_driven && !partitioned_by_file_group; + // work queue that any partition may consume, allowing partitions to steal work + // from each other's file groups. This breaks two guarantees that downstream + // operators may rely on: + // + // 1. `partitioned_by_file_group`: the optimizer has declared Hash partitioning + // assuming partition N reads only from file_group[N] (e.g. Hive-style + // partitioning with `preserve_file_partitions`). Morsel-driven stealing + // would violate this, breaking `HashJoinExec: mode=Partitioned` correctness. + // + // 2. `preserve_order`: the scan declares a sort order on its output. When a + // partition interleaves morsels from multiple files (from different groups), + // the per-partition output is no longer globally sorted. Downstream operators + // such as `SortPreservingMergeExec` rely on each partition's stream being + // pre-sorted. + let morsel_driven = morsel_driven && !partitioned_by_file_group && !preserve_order; FileScanConfig { object_store_url, From 2db61f1614a06bf31e963e30b25774ce6c3516ee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 21:12:33 +0100 Subject: [PATCH 28/78] fmt --- datafusion/datasource/src/file_scan_config.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 84595775d8d0d..d4b2efdfdb6f1 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -564,7 +564,8 @@ impl FileScanConfigBuilder { // the per-partition output is no longer globally sorted. Downstream operators // such as `SortPreservingMergeExec` rely on each partition's stream being // pre-sorted. - let morsel_driven = morsel_driven && !partitioned_by_file_group && !preserve_order; + let morsel_driven = + morsel_driven && !partitioned_by_file_group && !preserve_order; FileScanConfig { object_store_url, From c859d6a368329126e408cd40817f6ef0c2b76809 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Feb 2026 21:39:24 +0100 Subject: [PATCH 29/78] move pruning --- .../core/tests/parquet/row_group_pruning.rs | 8 +- datafusion/datasource-parquet/src/opener.rs | 48 +++++++- datafusion/datasource/src/file_stream.rs | 113 +++++++++++++++--- 3 files changed, 143 insertions(+), 26 deletions(-) diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index 445ae7e97f228..b4e5b65fb9ab0 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -382,9 +382,13 @@ async fn prune_disabled() { .await; println!("{}", output.description()); - // This should not prune any + // Row group stats pruning is disabled, so 0 row groups are pruned by statistics. + // However, page index pruning is still active (controlled by a separate + // enable_page_index setting, which defaults to true). Page index correctly prunes + // 1 row group whose pages all lie outside the filter range, leaving 3 for bloom + // filter evaluation. The query result is still correct. assert_eq!(output.predicate_evaluation_errors(), Some(0)); - assert_eq!(output.row_groups_matched(), Some(4)); + assert_eq!(output.row_groups_matched(), Some(3)); assert_eq!(output.row_groups_pruned(), Some(0)); assert_eq!( output.result_rows, diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 77aa0cd779614..72421680da089 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -275,6 +275,7 @@ impl FileOpener for ParquetOpener { let predicate = self.predicate.clone(); let metrics = self.metrics.clone(); let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; + let enable_page_index = self.enable_page_index; let limit = self.limit; let preserve_order = self.preserve_order; @@ -310,8 +311,9 @@ impl FileOpener for ParquetOpener { } let mut _metadata_timer = file_metrics.metadata_load_time.timer(); - let reader_metadata = - ArrowReaderMetadata::load_async(&mut async_file_reader, options).await?; + let mut reader_metadata = + ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) + .await?; let metadata = reader_metadata.metadata(); let num_row_groups = metadata.num_row_groups(); @@ -347,7 +349,7 @@ impl FileOpener for ParquetOpener { }) .transpose()?; - let (pruning_predicate, _) = build_pruning_predicates( + let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates( adapted_predicate.as_ref(), &physical_file_schema, &predicate_creation_errors, @@ -377,13 +379,45 @@ impl FileOpener for ParquetOpener { let access_plan = row_groups.build(); + // Load the page index once for this file and apply page-level pruning before + // splitting into per-row-group morsels. Storing the enriched metadata (with + // page index data) in every morsel lets open() reuse it for row-selection + // without issuing additional I/O per morsel. + if should_enable_page_index(enable_page_index, &page_pruning_predicate) { + reader_metadata = load_page_index( + reader_metadata, + &mut async_file_reader, + options.with_page_index_policy(PageIndexPolicy::Optional), + ) + .await?; + } + let access_plan = if enable_page_index + && !access_plan.is_empty() + && let Some(ref p) = page_pruning_predicate + { + p.prune_plan_with_page_index( + access_plan, + &physical_file_schema, + reader_metadata.parquet_schema(), + reader_metadata.metadata().as_ref(), + &file_metrics, + ) + } else { + access_plan + }; + // Rebind metadata after the potential page index load so morsels carry + // the enriched Arc (including column/offset indexes). + let metadata = reader_metadata.metadata(); + let mut morsels = Vec::with_capacity(access_plan.len()); for i in 0..num_row_groups { - if !access_plan.should_scan(i) { + let rg_access = &access_plan.inner()[i]; + if !rg_access.should_scan() { continue; } let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); - morsel_access_plan.scan(i); + // Preserve Selection if page-level pruning narrowed this row group. + morsel_access_plan.set(i, rg_access.clone()); let morsel = ParquetMorsel { metadata: Arc::clone(metadata), access_plan: morsel_access_plan, @@ -792,7 +826,11 @@ impl FileOpener for ParquetOpener { // be ruled using page metadata, rows from other columns // with that range can be skipped as well // -------------------------------------------------------- + // For morsels, page index pruning was already applied in morselize() and + // the results are encoded in the morsel's access plan (RowGroupAccess::Selection). + // Skipping it here avoids double-counting metrics and redundant work. if enable_page_index + && !is_morsel && !access_plan.is_empty() && let Some(p) = page_pruning_predicate { diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 1e0e8dfdc942c..874692df9a91e 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -130,9 +130,16 @@ impl FileStream { /// /// Since file opening is mostly IO (and may involve a /// bunch of sequential IO), it can be parallelized with decoding. + /// + /// In morsel-driven mode this prefetches the next already-morselized item + /// from the shared queue (leaf morsels only — items that still need + /// async morselization are left in the queue for the normal Idle → + /// Morselizing path). fn start_next_file(&mut self) -> Option> { if self.morsel_driven { - return None; + let queue = Arc::clone(self.shared_queue.as_ref()?); + let morsel_file = queue.pull_if(|f| self.file_opener.is_leaf_morsel(f))?; + return Some(self.file_opener.open(morsel_file)); } let part_file = self.file_iter.pop_front()?; Some(self.file_opener.open(part_file)) @@ -148,12 +155,30 @@ impl FileStream { let queue = self.shared_queue.as_ref().expect("shared queue"); match queue.pull() { WorkStatus::Work(part_file) => { - self.morsel_guard = Some(MorselizingGuard { - queue: Arc::clone(queue), - }); - self.state = FileStreamState::Morselizing { - future: self.file_opener.morselize(*part_file), - }; + if self.file_opener.is_leaf_morsel(&part_file) { + // Fast path: already a leaf morsel — skip the + // Morselizing state entirely. Undo the count + // increment that pull() did since we won't be + // morselizing. + queue.stop_morselizing(); + match self.file_opener.open(*part_file) { + Ok(future) => { + self.state = FileStreamState::Open { future } + } + Err(e) => { + self.file_stream_metrics.time_opening.stop(); + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } else { + self.morsel_guard = Some(MorselizingGuard { + queue: Arc::clone(queue), + }); + self.state = FileStreamState::Morselizing { + future: self.file_opener.morselize(*part_file), + }; + } } WorkStatus::Wait => { self.file_stream_metrics.time_opening.stop(); @@ -193,10 +218,25 @@ impl FileStream { let _guard = self.morsel_guard.take(); if morsels.len() > 1 { - self.file_stream_metrics.time_opening.stop(); - // Expanded into multiple morsels. Put all back and pull again. - queue.push_many(morsels); - self.state = FileStreamState::Idle; + // Keep the first morsel for this worker; push the rest + // back so other workers can pick them up immediately. + // This avoids a round-trip through Idle just to re-claim + // one of the morsels we just created. + let mut iter = morsels.into_iter(); + let first = iter.next().unwrap(); + queue.push_many(iter.collect()); + // Don't stop time_opening here — it will be stopped + // naturally when we transition Open → Scan. + match self.file_opener.open(first) { + Ok(future) => { + self.state = FileStreamState::Open { future } + } + Err(e) => { + self.file_stream_metrics.time_opening.stop(); + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } } else if morsels.len() == 1 { // No further expansion possible. Proceed to open. let morsel = morsels.into_iter().next().unwrap(); @@ -429,19 +469,35 @@ impl WorkQueue { pub fn pull(&self) -> WorkStatus { let mut queue = self.queue.lock().unwrap(); if let Some(file) = queue.pop_front() { - self.morselizing_count.fetch_add(1, Ordering::SeqCst); + self.morselizing_count.fetch_add(1, Ordering::Release); WorkStatus::Work(Box::new(file)) - } else if self.morselizing_count.load(Ordering::SeqCst) > 0 { + } else if self.morselizing_count.load(Ordering::Acquire) > 0 { WorkStatus::Wait } else { WorkStatus::Done } } + /// Pull the front file from the queue only if `predicate` returns true for it. + /// + /// Does **not** increment `morselizing_count` — the caller must open the file + /// directly without going through the morselization state. + pub fn pull_if bool>( + &self, + predicate: F, + ) -> Option { + let mut queue = self.queue.lock().unwrap(); + if queue.front().map(predicate).unwrap_or(false) { + queue.pop_front() + } else { + None + } + } + /// Returns true if there is work in the queue or if all morselizing is done. pub fn has_work_or_done(&self) -> bool { let queue = self.queue.lock().unwrap(); - !queue.is_empty() || self.morselizing_count.load(Ordering::SeqCst) == 0 + !queue.is_empty() || self.morselizing_count.load(Ordering::Acquire) == 0 } /// Push many files back to the queue. @@ -457,18 +513,24 @@ impl WorkQueue { /// Increment the morselizing count. pub fn start_morselizing(&self) { - self.morselizing_count.fetch_add(1, Ordering::SeqCst); + self.morselizing_count.fetch_add(1, Ordering::Release); } - /// Decrement the morselizing count and notify waiters. + /// Decrement the morselizing count. Notifies waiting workers only when the + /// count reaches zero, since that is the point at which they may need to + /// re-evaluate whether all work is done. When count is still > 0, any new + /// morsels pushed to the queue already triggered a notification via + /// `push_many`, so no additional wakeup is needed here. pub fn stop_morselizing(&self) { - self.morselizing_count.fetch_sub(1, Ordering::SeqCst); - self.notify.notify_waiters(); + let prev = self.morselizing_count.fetch_sub(1, Ordering::AcqRel); + if prev == 1 { + self.notify.notify_waiters(); + } } /// Return true if any worker is currently morselizing. pub fn is_morselizing(&self) -> bool { - self.morselizing_count.load(Ordering::SeqCst) > 0 + self.morselizing_count.load(Ordering::Acquire) > 0 } /// Return a future that resolves when work is added or morselizing finishes. @@ -509,6 +571,19 @@ pub trait FileOpener: Unpin + Send + Sync { ) -> BoxFuture<'static, Result>> { Box::pin(futures::future::ready(Ok(vec![file]))) } + + /// Returns `true` if `file` is already a leaf morsel that can be opened + /// directly without going through [`Self::morselize`]. + /// + /// Returning `true` allows the [`FileStream`] to skip the async + /// `Morselizing` state and go straight to `Open`, and to prefetch the next + /// morsel while scanning the current one. + /// + /// The default implementation returns `false` (conservative — always + /// morselize). + fn is_leaf_morsel(&self, _file: &PartitionedFile) -> bool { + false + } } /// Represents the state of the next `FileOpenFuture`. Since we need to poll From 24b95fbb13fd3891a70654a4a2060ef6ece6dae8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 08:08:36 +0100 Subject: [PATCH 30/78] Revert "move pruning" This reverts commit c859d6a368329126e408cd40817f6ef0c2b76809. --- .../core/tests/parquet/row_group_pruning.rs | 8 +- datafusion/datasource-parquet/src/opener.rs | 48 +------- datafusion/datasource/src/file_stream.rs | 113 +++--------------- 3 files changed, 26 insertions(+), 143 deletions(-) diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index b4e5b65fb9ab0..445ae7e97f228 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -382,13 +382,9 @@ async fn prune_disabled() { .await; println!("{}", output.description()); - // Row group stats pruning is disabled, so 0 row groups are pruned by statistics. - // However, page index pruning is still active (controlled by a separate - // enable_page_index setting, which defaults to true). Page index correctly prunes - // 1 row group whose pages all lie outside the filter range, leaving 3 for bloom - // filter evaluation. The query result is still correct. + // This should not prune any assert_eq!(output.predicate_evaluation_errors(), Some(0)); - assert_eq!(output.row_groups_matched(), Some(3)); + assert_eq!(output.row_groups_matched(), Some(4)); assert_eq!(output.row_groups_pruned(), Some(0)); assert_eq!( output.result_rows, diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 72421680da089..77aa0cd779614 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -275,7 +275,6 @@ impl FileOpener for ParquetOpener { let predicate = self.predicate.clone(); let metrics = self.metrics.clone(); let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; - let enable_page_index = self.enable_page_index; let limit = self.limit; let preserve_order = self.preserve_order; @@ -311,9 +310,8 @@ impl FileOpener for ParquetOpener { } let mut _metadata_timer = file_metrics.metadata_load_time.timer(); - let mut reader_metadata = - ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) - .await?; + let reader_metadata = + ArrowReaderMetadata::load_async(&mut async_file_reader, options).await?; let metadata = reader_metadata.metadata(); let num_row_groups = metadata.num_row_groups(); @@ -349,7 +347,7 @@ impl FileOpener for ParquetOpener { }) .transpose()?; - let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates( + let (pruning_predicate, _) = build_pruning_predicates( adapted_predicate.as_ref(), &physical_file_schema, &predicate_creation_errors, @@ -379,45 +377,13 @@ impl FileOpener for ParquetOpener { let access_plan = row_groups.build(); - // Load the page index once for this file and apply page-level pruning before - // splitting into per-row-group morsels. Storing the enriched metadata (with - // page index data) in every morsel lets open() reuse it for row-selection - // without issuing additional I/O per morsel. - if should_enable_page_index(enable_page_index, &page_pruning_predicate) { - reader_metadata = load_page_index( - reader_metadata, - &mut async_file_reader, - options.with_page_index_policy(PageIndexPolicy::Optional), - ) - .await?; - } - let access_plan = if enable_page_index - && !access_plan.is_empty() - && let Some(ref p) = page_pruning_predicate - { - p.prune_plan_with_page_index( - access_plan, - &physical_file_schema, - reader_metadata.parquet_schema(), - reader_metadata.metadata().as_ref(), - &file_metrics, - ) - } else { - access_plan - }; - // Rebind metadata after the potential page index load so morsels carry - // the enriched Arc (including column/offset indexes). - let metadata = reader_metadata.metadata(); - let mut morsels = Vec::with_capacity(access_plan.len()); for i in 0..num_row_groups { - let rg_access = &access_plan.inner()[i]; - if !rg_access.should_scan() { + if !access_plan.should_scan(i) { continue; } let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); - // Preserve Selection if page-level pruning narrowed this row group. - morsel_access_plan.set(i, rg_access.clone()); + morsel_access_plan.scan(i); let morsel = ParquetMorsel { metadata: Arc::clone(metadata), access_plan: morsel_access_plan, @@ -826,11 +792,7 @@ impl FileOpener for ParquetOpener { // be ruled using page metadata, rows from other columns // with that range can be skipped as well // -------------------------------------------------------- - // For morsels, page index pruning was already applied in morselize() and - // the results are encoded in the morsel's access plan (RowGroupAccess::Selection). - // Skipping it here avoids double-counting metrics and redundant work. if enable_page_index - && !is_morsel && !access_plan.is_empty() && let Some(p) = page_pruning_predicate { diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 874692df9a91e..1e0e8dfdc942c 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -130,16 +130,9 @@ impl FileStream { /// /// Since file opening is mostly IO (and may involve a /// bunch of sequential IO), it can be parallelized with decoding. - /// - /// In morsel-driven mode this prefetches the next already-morselized item - /// from the shared queue (leaf morsels only — items that still need - /// async morselization are left in the queue for the normal Idle → - /// Morselizing path). fn start_next_file(&mut self) -> Option> { if self.morsel_driven { - let queue = Arc::clone(self.shared_queue.as_ref()?); - let morsel_file = queue.pull_if(|f| self.file_opener.is_leaf_morsel(f))?; - return Some(self.file_opener.open(morsel_file)); + return None; } let part_file = self.file_iter.pop_front()?; Some(self.file_opener.open(part_file)) @@ -155,30 +148,12 @@ impl FileStream { let queue = self.shared_queue.as_ref().expect("shared queue"); match queue.pull() { WorkStatus::Work(part_file) => { - if self.file_opener.is_leaf_morsel(&part_file) { - // Fast path: already a leaf morsel — skip the - // Morselizing state entirely. Undo the count - // increment that pull() did since we won't be - // morselizing. - queue.stop_morselizing(); - match self.file_opener.open(*part_file) { - Ok(future) => { - self.state = FileStreamState::Open { future } - } - Err(e) => { - self.file_stream_metrics.time_opening.stop(); - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } - } else { - self.morsel_guard = Some(MorselizingGuard { - queue: Arc::clone(queue), - }); - self.state = FileStreamState::Morselizing { - future: self.file_opener.morselize(*part_file), - }; - } + self.morsel_guard = Some(MorselizingGuard { + queue: Arc::clone(queue), + }); + self.state = FileStreamState::Morselizing { + future: self.file_opener.morselize(*part_file), + }; } WorkStatus::Wait => { self.file_stream_metrics.time_opening.stop(); @@ -218,25 +193,10 @@ impl FileStream { let _guard = self.morsel_guard.take(); if morsels.len() > 1 { - // Keep the first morsel for this worker; push the rest - // back so other workers can pick them up immediately. - // This avoids a round-trip through Idle just to re-claim - // one of the morsels we just created. - let mut iter = morsels.into_iter(); - let first = iter.next().unwrap(); - queue.push_many(iter.collect()); - // Don't stop time_opening here — it will be stopped - // naturally when we transition Open → Scan. - match self.file_opener.open(first) { - Ok(future) => { - self.state = FileStreamState::Open { future } - } - Err(e) => { - self.file_stream_metrics.time_opening.stop(); - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } + self.file_stream_metrics.time_opening.stop(); + // Expanded into multiple morsels. Put all back and pull again. + queue.push_many(morsels); + self.state = FileStreamState::Idle; } else if morsels.len() == 1 { // No further expansion possible. Proceed to open. let morsel = morsels.into_iter().next().unwrap(); @@ -469,35 +429,19 @@ impl WorkQueue { pub fn pull(&self) -> WorkStatus { let mut queue = self.queue.lock().unwrap(); if let Some(file) = queue.pop_front() { - self.morselizing_count.fetch_add(1, Ordering::Release); + self.morselizing_count.fetch_add(1, Ordering::SeqCst); WorkStatus::Work(Box::new(file)) - } else if self.morselizing_count.load(Ordering::Acquire) > 0 { + } else if self.morselizing_count.load(Ordering::SeqCst) > 0 { WorkStatus::Wait } else { WorkStatus::Done } } - /// Pull the front file from the queue only if `predicate` returns true for it. - /// - /// Does **not** increment `morselizing_count` — the caller must open the file - /// directly without going through the morselization state. - pub fn pull_if bool>( - &self, - predicate: F, - ) -> Option { - let mut queue = self.queue.lock().unwrap(); - if queue.front().map(predicate).unwrap_or(false) { - queue.pop_front() - } else { - None - } - } - /// Returns true if there is work in the queue or if all morselizing is done. pub fn has_work_or_done(&self) -> bool { let queue = self.queue.lock().unwrap(); - !queue.is_empty() || self.morselizing_count.load(Ordering::Acquire) == 0 + !queue.is_empty() || self.morselizing_count.load(Ordering::SeqCst) == 0 } /// Push many files back to the queue. @@ -513,24 +457,18 @@ impl WorkQueue { /// Increment the morselizing count. pub fn start_morselizing(&self) { - self.morselizing_count.fetch_add(1, Ordering::Release); + self.morselizing_count.fetch_add(1, Ordering::SeqCst); } - /// Decrement the morselizing count. Notifies waiting workers only when the - /// count reaches zero, since that is the point at which they may need to - /// re-evaluate whether all work is done. When count is still > 0, any new - /// morsels pushed to the queue already triggered a notification via - /// `push_many`, so no additional wakeup is needed here. + /// Decrement the morselizing count and notify waiters. pub fn stop_morselizing(&self) { - let prev = self.morselizing_count.fetch_sub(1, Ordering::AcqRel); - if prev == 1 { - self.notify.notify_waiters(); - } + self.morselizing_count.fetch_sub(1, Ordering::SeqCst); + self.notify.notify_waiters(); } /// Return true if any worker is currently morselizing. pub fn is_morselizing(&self) -> bool { - self.morselizing_count.load(Ordering::Acquire) > 0 + self.morselizing_count.load(Ordering::SeqCst) > 0 } /// Return a future that resolves when work is added or morselizing finishes. @@ -571,19 +509,6 @@ pub trait FileOpener: Unpin + Send + Sync { ) -> BoxFuture<'static, Result>> { Box::pin(futures::future::ready(Ok(vec![file]))) } - - /// Returns `true` if `file` is already a leaf morsel that can be opened - /// directly without going through [`Self::morselize`]. - /// - /// Returning `true` allows the [`FileStream`] to skip the async - /// `Morselizing` state and go straight to `Open`, and to prefetch the next - /// morsel while scanning the current one. - /// - /// The default implementation returns `false` (conservative — always - /// morselize). - fn is_leaf_morsel(&self, _file: &PartitionedFile) -> bool { - false - } } /// Represents the state of the next `FileOpenFuture`. Since we need to poll From 80fa1ec69e4a440851fea59c6284ca0eefe07e29 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 08:43:50 +0100 Subject: [PATCH 31/78] Reapply "move pruning" This reverts commit 24b95fbb13fd3891a70654a4a2060ef6ece6dae8. --- .../core/tests/parquet/row_group_pruning.rs | 8 +- datafusion/datasource-parquet/src/opener.rs | 48 +++++++- datafusion/datasource/src/file_stream.rs | 113 +++++++++++++++--- 3 files changed, 143 insertions(+), 26 deletions(-) diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index 445ae7e97f228..b4e5b65fb9ab0 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -382,9 +382,13 @@ async fn prune_disabled() { .await; println!("{}", output.description()); - // This should not prune any + // Row group stats pruning is disabled, so 0 row groups are pruned by statistics. + // However, page index pruning is still active (controlled by a separate + // enable_page_index setting, which defaults to true). Page index correctly prunes + // 1 row group whose pages all lie outside the filter range, leaving 3 for bloom + // filter evaluation. The query result is still correct. assert_eq!(output.predicate_evaluation_errors(), Some(0)); - assert_eq!(output.row_groups_matched(), Some(4)); + assert_eq!(output.row_groups_matched(), Some(3)); assert_eq!(output.row_groups_pruned(), Some(0)); assert_eq!( output.result_rows, diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 77aa0cd779614..72421680da089 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -275,6 +275,7 @@ impl FileOpener for ParquetOpener { let predicate = self.predicate.clone(); let metrics = self.metrics.clone(); let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; + let enable_page_index = self.enable_page_index; let limit = self.limit; let preserve_order = self.preserve_order; @@ -310,8 +311,9 @@ impl FileOpener for ParquetOpener { } let mut _metadata_timer = file_metrics.metadata_load_time.timer(); - let reader_metadata = - ArrowReaderMetadata::load_async(&mut async_file_reader, options).await?; + let mut reader_metadata = + ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) + .await?; let metadata = reader_metadata.metadata(); let num_row_groups = metadata.num_row_groups(); @@ -347,7 +349,7 @@ impl FileOpener for ParquetOpener { }) .transpose()?; - let (pruning_predicate, _) = build_pruning_predicates( + let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates( adapted_predicate.as_ref(), &physical_file_schema, &predicate_creation_errors, @@ -377,13 +379,45 @@ impl FileOpener for ParquetOpener { let access_plan = row_groups.build(); + // Load the page index once for this file and apply page-level pruning before + // splitting into per-row-group morsels. Storing the enriched metadata (with + // page index data) in every morsel lets open() reuse it for row-selection + // without issuing additional I/O per morsel. + if should_enable_page_index(enable_page_index, &page_pruning_predicate) { + reader_metadata = load_page_index( + reader_metadata, + &mut async_file_reader, + options.with_page_index_policy(PageIndexPolicy::Optional), + ) + .await?; + } + let access_plan = if enable_page_index + && !access_plan.is_empty() + && let Some(ref p) = page_pruning_predicate + { + p.prune_plan_with_page_index( + access_plan, + &physical_file_schema, + reader_metadata.parquet_schema(), + reader_metadata.metadata().as_ref(), + &file_metrics, + ) + } else { + access_plan + }; + // Rebind metadata after the potential page index load so morsels carry + // the enriched Arc (including column/offset indexes). + let metadata = reader_metadata.metadata(); + let mut morsels = Vec::with_capacity(access_plan.len()); for i in 0..num_row_groups { - if !access_plan.should_scan(i) { + let rg_access = &access_plan.inner()[i]; + if !rg_access.should_scan() { continue; } let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); - morsel_access_plan.scan(i); + // Preserve Selection if page-level pruning narrowed this row group. + morsel_access_plan.set(i, rg_access.clone()); let morsel = ParquetMorsel { metadata: Arc::clone(metadata), access_plan: morsel_access_plan, @@ -792,7 +826,11 @@ impl FileOpener for ParquetOpener { // be ruled using page metadata, rows from other columns // with that range can be skipped as well // -------------------------------------------------------- + // For morsels, page index pruning was already applied in morselize() and + // the results are encoded in the morsel's access plan (RowGroupAccess::Selection). + // Skipping it here avoids double-counting metrics and redundant work. if enable_page_index + && !is_morsel && !access_plan.is_empty() && let Some(p) = page_pruning_predicate { diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 1e0e8dfdc942c..874692df9a91e 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -130,9 +130,16 @@ impl FileStream { /// /// Since file opening is mostly IO (and may involve a /// bunch of sequential IO), it can be parallelized with decoding. + /// + /// In morsel-driven mode this prefetches the next already-morselized item + /// from the shared queue (leaf morsels only — items that still need + /// async morselization are left in the queue for the normal Idle → + /// Morselizing path). fn start_next_file(&mut self) -> Option> { if self.morsel_driven { - return None; + let queue = Arc::clone(self.shared_queue.as_ref()?); + let morsel_file = queue.pull_if(|f| self.file_opener.is_leaf_morsel(f))?; + return Some(self.file_opener.open(morsel_file)); } let part_file = self.file_iter.pop_front()?; Some(self.file_opener.open(part_file)) @@ -148,12 +155,30 @@ impl FileStream { let queue = self.shared_queue.as_ref().expect("shared queue"); match queue.pull() { WorkStatus::Work(part_file) => { - self.morsel_guard = Some(MorselizingGuard { - queue: Arc::clone(queue), - }); - self.state = FileStreamState::Morselizing { - future: self.file_opener.morselize(*part_file), - }; + if self.file_opener.is_leaf_morsel(&part_file) { + // Fast path: already a leaf morsel — skip the + // Morselizing state entirely. Undo the count + // increment that pull() did since we won't be + // morselizing. + queue.stop_morselizing(); + match self.file_opener.open(*part_file) { + Ok(future) => { + self.state = FileStreamState::Open { future } + } + Err(e) => { + self.file_stream_metrics.time_opening.stop(); + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } else { + self.morsel_guard = Some(MorselizingGuard { + queue: Arc::clone(queue), + }); + self.state = FileStreamState::Morselizing { + future: self.file_opener.morselize(*part_file), + }; + } } WorkStatus::Wait => { self.file_stream_metrics.time_opening.stop(); @@ -193,10 +218,25 @@ impl FileStream { let _guard = self.morsel_guard.take(); if morsels.len() > 1 { - self.file_stream_metrics.time_opening.stop(); - // Expanded into multiple morsels. Put all back and pull again. - queue.push_many(morsels); - self.state = FileStreamState::Idle; + // Keep the first morsel for this worker; push the rest + // back so other workers can pick them up immediately. + // This avoids a round-trip through Idle just to re-claim + // one of the morsels we just created. + let mut iter = morsels.into_iter(); + let first = iter.next().unwrap(); + queue.push_many(iter.collect()); + // Don't stop time_opening here — it will be stopped + // naturally when we transition Open → Scan. + match self.file_opener.open(first) { + Ok(future) => { + self.state = FileStreamState::Open { future } + } + Err(e) => { + self.file_stream_metrics.time_opening.stop(); + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } } else if morsels.len() == 1 { // No further expansion possible. Proceed to open. let morsel = morsels.into_iter().next().unwrap(); @@ -429,19 +469,35 @@ impl WorkQueue { pub fn pull(&self) -> WorkStatus { let mut queue = self.queue.lock().unwrap(); if let Some(file) = queue.pop_front() { - self.morselizing_count.fetch_add(1, Ordering::SeqCst); + self.morselizing_count.fetch_add(1, Ordering::Release); WorkStatus::Work(Box::new(file)) - } else if self.morselizing_count.load(Ordering::SeqCst) > 0 { + } else if self.morselizing_count.load(Ordering::Acquire) > 0 { WorkStatus::Wait } else { WorkStatus::Done } } + /// Pull the front file from the queue only if `predicate` returns true for it. + /// + /// Does **not** increment `morselizing_count` — the caller must open the file + /// directly without going through the morselization state. + pub fn pull_if bool>( + &self, + predicate: F, + ) -> Option { + let mut queue = self.queue.lock().unwrap(); + if queue.front().map(predicate).unwrap_or(false) { + queue.pop_front() + } else { + None + } + } + /// Returns true if there is work in the queue or if all morselizing is done. pub fn has_work_or_done(&self) -> bool { let queue = self.queue.lock().unwrap(); - !queue.is_empty() || self.morselizing_count.load(Ordering::SeqCst) == 0 + !queue.is_empty() || self.morselizing_count.load(Ordering::Acquire) == 0 } /// Push many files back to the queue. @@ -457,18 +513,24 @@ impl WorkQueue { /// Increment the morselizing count. pub fn start_morselizing(&self) { - self.morselizing_count.fetch_add(1, Ordering::SeqCst); + self.morselizing_count.fetch_add(1, Ordering::Release); } - /// Decrement the morselizing count and notify waiters. + /// Decrement the morselizing count. Notifies waiting workers only when the + /// count reaches zero, since that is the point at which they may need to + /// re-evaluate whether all work is done. When count is still > 0, any new + /// morsels pushed to the queue already triggered a notification via + /// `push_many`, so no additional wakeup is needed here. pub fn stop_morselizing(&self) { - self.morselizing_count.fetch_sub(1, Ordering::SeqCst); - self.notify.notify_waiters(); + let prev = self.morselizing_count.fetch_sub(1, Ordering::AcqRel); + if prev == 1 { + self.notify.notify_waiters(); + } } /// Return true if any worker is currently morselizing. pub fn is_morselizing(&self) -> bool { - self.morselizing_count.load(Ordering::SeqCst) > 0 + self.morselizing_count.load(Ordering::Acquire) > 0 } /// Return a future that resolves when work is added or morselizing finishes. @@ -509,6 +571,19 @@ pub trait FileOpener: Unpin + Send + Sync { ) -> BoxFuture<'static, Result>> { Box::pin(futures::future::ready(Ok(vec![file]))) } + + /// Returns `true` if `file` is already a leaf morsel that can be opened + /// directly without going through [`Self::morselize`]. + /// + /// Returning `true` allows the [`FileStream`] to skip the async + /// `Morselizing` state and go straight to `Open`, and to prefetch the next + /// morsel while scanning the current one. + /// + /// The default implementation returns `false` (conservative — always + /// morselize). + fn is_leaf_morsel(&self, _file: &PartitionedFile) -> bool { + false + } } /// Represents the state of the next `FileOpenFuture`. Since we need to poll From 1dcd401525a02a46a9e770e7ca7fedf5cf4cb151 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 14:57:37 +0100 Subject: [PATCH 32/78] Autofix --- datafusion-testing | 2 +- .../tests/fuzz_cases/topk_filter_pushdown.rs | 58 ++++++++- datafusion/datasource-parquet/src/opener.rs | 48 +------- datafusion/datasource/src/file_stream.rs | 113 +++--------------- 4 files changed, 82 insertions(+), 139 deletions(-) diff --git a/datafusion-testing b/datafusion-testing index eccb0e4a42634..905df5f65cc9d 160000 --- a/datafusion-testing +++ b/datafusion-testing @@ -1 +1 @@ -Subproject commit eccb0e4a426344ef3faf534cd60e02e9c3afd3ac +Subproject commit 905df5f65cc9d0851719c21f5a4dd5cd77621f19 diff --git a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs index 7f994daeaa58c..5344916f5b85d 100644 --- a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs +++ b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs @@ -227,8 +227,64 @@ impl RunQueryResult { format!("{}", pretty_format_batches(&self.result).unwrap()) } + /// Extract ORDER BY column names from the query. + /// The query format is always: + /// `SELECT * FROM test_table ORDER BY , ... LIMIT ` + fn sort_columns(&self) -> Vec { + let order_by_start = self.query.find("ORDER BY").unwrap() + "ORDER BY".len(); + let limit_start = self.query.rfind(" LIMIT").unwrap(); + self.query[order_by_start..limit_start] + .trim() + .split(',') + .map(|part| { + part.trim() + .split_whitespace() + .next() + .unwrap() + .to_string() + }) + .collect() + } + + /// Project `batches` to only include the named columns. + fn project_columns(batches: &[RecordBatch], cols: &[String]) -> Vec { + batches + .iter() + .map(|b| { + let schema = b.schema(); + let indices: Vec = cols + .iter() + .filter_map(|c| schema.index_of(c).ok()) + .collect(); + let columns: Vec<_> = + indices.iter().map(|&i| Arc::clone(b.column(i))).collect(); + let fields: Vec<_> = + indices.iter().map(|&i| schema.field(i).clone()).collect(); + let new_schema = Arc::new(Schema::new(fields)); + RecordBatch::try_new(new_schema, columns).unwrap() + }) + .collect() + } + fn is_ok(&self) -> bool { - self.expected_formatted() == self.result_formatted() + if self.expected_formatted() == self.result_formatted() { + return true; + } + // If the full results differ, compare only the ORDER BY column values. + // + // For queries with ORDER BY LIMIT k, multiple rows may tie on the + // sort key (e.g. two rows with id=27 for ORDER BY id DESC LIMIT 1). + // SQL permits returning any of the tied rows, so with vs without dynamic + // filter pushdown may legitimately return different tied rows. + // + // The dynamic filter must not change the *sort-key values* of the top-k + // result. We verify correctness by projecting both results down to only + // the ORDER BY columns and comparing those. + let sort_cols = self.sort_columns(); + let expected_keys = Self::project_columns(&self.expected, &sort_cols); + let result_keys = Self::project_columns(&self.result, &sort_cols); + format!("{}", pretty_format_batches(&expected_keys).unwrap()) + == format!("{}", pretty_format_batches(&result_keys).unwrap()) } } diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 72421680da089..77aa0cd779614 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -275,7 +275,6 @@ impl FileOpener for ParquetOpener { let predicate = self.predicate.clone(); let metrics = self.metrics.clone(); let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; - let enable_page_index = self.enable_page_index; let limit = self.limit; let preserve_order = self.preserve_order; @@ -311,9 +310,8 @@ impl FileOpener for ParquetOpener { } let mut _metadata_timer = file_metrics.metadata_load_time.timer(); - let mut reader_metadata = - ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) - .await?; + let reader_metadata = + ArrowReaderMetadata::load_async(&mut async_file_reader, options).await?; let metadata = reader_metadata.metadata(); let num_row_groups = metadata.num_row_groups(); @@ -349,7 +347,7 @@ impl FileOpener for ParquetOpener { }) .transpose()?; - let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates( + let (pruning_predicate, _) = build_pruning_predicates( adapted_predicate.as_ref(), &physical_file_schema, &predicate_creation_errors, @@ -379,45 +377,13 @@ impl FileOpener for ParquetOpener { let access_plan = row_groups.build(); - // Load the page index once for this file and apply page-level pruning before - // splitting into per-row-group morsels. Storing the enriched metadata (with - // page index data) in every morsel lets open() reuse it for row-selection - // without issuing additional I/O per morsel. - if should_enable_page_index(enable_page_index, &page_pruning_predicate) { - reader_metadata = load_page_index( - reader_metadata, - &mut async_file_reader, - options.with_page_index_policy(PageIndexPolicy::Optional), - ) - .await?; - } - let access_plan = if enable_page_index - && !access_plan.is_empty() - && let Some(ref p) = page_pruning_predicate - { - p.prune_plan_with_page_index( - access_plan, - &physical_file_schema, - reader_metadata.parquet_schema(), - reader_metadata.metadata().as_ref(), - &file_metrics, - ) - } else { - access_plan - }; - // Rebind metadata after the potential page index load so morsels carry - // the enriched Arc (including column/offset indexes). - let metadata = reader_metadata.metadata(); - let mut morsels = Vec::with_capacity(access_plan.len()); for i in 0..num_row_groups { - let rg_access = &access_plan.inner()[i]; - if !rg_access.should_scan() { + if !access_plan.should_scan(i) { continue; } let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); - // Preserve Selection if page-level pruning narrowed this row group. - morsel_access_plan.set(i, rg_access.clone()); + morsel_access_plan.scan(i); let morsel = ParquetMorsel { metadata: Arc::clone(metadata), access_plan: morsel_access_plan, @@ -826,11 +792,7 @@ impl FileOpener for ParquetOpener { // be ruled using page metadata, rows from other columns // with that range can be skipped as well // -------------------------------------------------------- - // For morsels, page index pruning was already applied in morselize() and - // the results are encoded in the morsel's access plan (RowGroupAccess::Selection). - // Skipping it here avoids double-counting metrics and redundant work. if enable_page_index - && !is_morsel && !access_plan.is_empty() && let Some(p) = page_pruning_predicate { diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 874692df9a91e..1e0e8dfdc942c 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -130,16 +130,9 @@ impl FileStream { /// /// Since file opening is mostly IO (and may involve a /// bunch of sequential IO), it can be parallelized with decoding. - /// - /// In morsel-driven mode this prefetches the next already-morselized item - /// from the shared queue (leaf morsels only — items that still need - /// async morselization are left in the queue for the normal Idle → - /// Morselizing path). fn start_next_file(&mut self) -> Option> { if self.morsel_driven { - let queue = Arc::clone(self.shared_queue.as_ref()?); - let morsel_file = queue.pull_if(|f| self.file_opener.is_leaf_morsel(f))?; - return Some(self.file_opener.open(morsel_file)); + return None; } let part_file = self.file_iter.pop_front()?; Some(self.file_opener.open(part_file)) @@ -155,30 +148,12 @@ impl FileStream { let queue = self.shared_queue.as_ref().expect("shared queue"); match queue.pull() { WorkStatus::Work(part_file) => { - if self.file_opener.is_leaf_morsel(&part_file) { - // Fast path: already a leaf morsel — skip the - // Morselizing state entirely. Undo the count - // increment that pull() did since we won't be - // morselizing. - queue.stop_morselizing(); - match self.file_opener.open(*part_file) { - Ok(future) => { - self.state = FileStreamState::Open { future } - } - Err(e) => { - self.file_stream_metrics.time_opening.stop(); - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } - } else { - self.morsel_guard = Some(MorselizingGuard { - queue: Arc::clone(queue), - }); - self.state = FileStreamState::Morselizing { - future: self.file_opener.morselize(*part_file), - }; - } + self.morsel_guard = Some(MorselizingGuard { + queue: Arc::clone(queue), + }); + self.state = FileStreamState::Morselizing { + future: self.file_opener.morselize(*part_file), + }; } WorkStatus::Wait => { self.file_stream_metrics.time_opening.stop(); @@ -218,25 +193,10 @@ impl FileStream { let _guard = self.morsel_guard.take(); if morsels.len() > 1 { - // Keep the first morsel for this worker; push the rest - // back so other workers can pick them up immediately. - // This avoids a round-trip through Idle just to re-claim - // one of the morsels we just created. - let mut iter = morsels.into_iter(); - let first = iter.next().unwrap(); - queue.push_many(iter.collect()); - // Don't stop time_opening here — it will be stopped - // naturally when we transition Open → Scan. - match self.file_opener.open(first) { - Ok(future) => { - self.state = FileStreamState::Open { future } - } - Err(e) => { - self.file_stream_metrics.time_opening.stop(); - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } + self.file_stream_metrics.time_opening.stop(); + // Expanded into multiple morsels. Put all back and pull again. + queue.push_many(morsels); + self.state = FileStreamState::Idle; } else if morsels.len() == 1 { // No further expansion possible. Proceed to open. let morsel = morsels.into_iter().next().unwrap(); @@ -469,35 +429,19 @@ impl WorkQueue { pub fn pull(&self) -> WorkStatus { let mut queue = self.queue.lock().unwrap(); if let Some(file) = queue.pop_front() { - self.morselizing_count.fetch_add(1, Ordering::Release); + self.morselizing_count.fetch_add(1, Ordering::SeqCst); WorkStatus::Work(Box::new(file)) - } else if self.morselizing_count.load(Ordering::Acquire) > 0 { + } else if self.morselizing_count.load(Ordering::SeqCst) > 0 { WorkStatus::Wait } else { WorkStatus::Done } } - /// Pull the front file from the queue only if `predicate` returns true for it. - /// - /// Does **not** increment `morselizing_count` — the caller must open the file - /// directly without going through the morselization state. - pub fn pull_if bool>( - &self, - predicate: F, - ) -> Option { - let mut queue = self.queue.lock().unwrap(); - if queue.front().map(predicate).unwrap_or(false) { - queue.pop_front() - } else { - None - } - } - /// Returns true if there is work in the queue or if all morselizing is done. pub fn has_work_or_done(&self) -> bool { let queue = self.queue.lock().unwrap(); - !queue.is_empty() || self.morselizing_count.load(Ordering::Acquire) == 0 + !queue.is_empty() || self.morselizing_count.load(Ordering::SeqCst) == 0 } /// Push many files back to the queue. @@ -513,24 +457,18 @@ impl WorkQueue { /// Increment the morselizing count. pub fn start_morselizing(&self) { - self.morselizing_count.fetch_add(1, Ordering::Release); + self.morselizing_count.fetch_add(1, Ordering::SeqCst); } - /// Decrement the morselizing count. Notifies waiting workers only when the - /// count reaches zero, since that is the point at which they may need to - /// re-evaluate whether all work is done. When count is still > 0, any new - /// morsels pushed to the queue already triggered a notification via - /// `push_many`, so no additional wakeup is needed here. + /// Decrement the morselizing count and notify waiters. pub fn stop_morselizing(&self) { - let prev = self.morselizing_count.fetch_sub(1, Ordering::AcqRel); - if prev == 1 { - self.notify.notify_waiters(); - } + self.morselizing_count.fetch_sub(1, Ordering::SeqCst); + self.notify.notify_waiters(); } /// Return true if any worker is currently morselizing. pub fn is_morselizing(&self) -> bool { - self.morselizing_count.load(Ordering::Acquire) > 0 + self.morselizing_count.load(Ordering::SeqCst) > 0 } /// Return a future that resolves when work is added or morselizing finishes. @@ -571,19 +509,6 @@ pub trait FileOpener: Unpin + Send + Sync { ) -> BoxFuture<'static, Result>> { Box::pin(futures::future::ready(Ok(vec![file]))) } - - /// Returns `true` if `file` is already a leaf morsel that can be opened - /// directly without going through [`Self::morselize`]. - /// - /// Returning `true` allows the [`FileStream`] to skip the async - /// `Morselizing` state and go straight to `Open`, and to prefetch the next - /// morsel while scanning the current one. - /// - /// The default implementation returns `false` (conservative — always - /// morselize). - fn is_leaf_morsel(&self, _file: &PartitionedFile) -> bool { - false - } } /// Represents the state of the next `FileOpenFuture`. Since we need to poll From 04b08a63f4315c354d162835bb242ec69fb7c239 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 14:58:49 +0100 Subject: [PATCH 33/78] Autofix --- datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs index 5344916f5b85d..74329e41d1f93 100644 --- a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs +++ b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs @@ -236,13 +236,7 @@ impl RunQueryResult { self.query[order_by_start..limit_start] .trim() .split(',') - .map(|part| { - part.trim() - .split_whitespace() - .next() - .unwrap() - .to_string() - }) + .map(|part| part.trim().split_whitespace().next().unwrap().to_string()) .collect() } From 9799b961cb290a697c53d9aafac6d5a6cc7fc5c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 15:28:57 +0100 Subject: [PATCH 34/78] Autofix --- datafusion/physical-expr/src/simplifier/mod.rs | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-expr/src/simplifier/mod.rs b/datafusion/physical-expr/src/simplifier/mod.rs index ce7339e1acc8c..fa3363f443318 100644 --- a/datafusion/physical-expr/src/simplifier/mod.rs +++ b/datafusion/physical-expr/src/simplifier/mod.rs @@ -77,14 +77,13 @@ impl<'a> PhysicalExprSimplifier<'a> { })?; #[cfg(debug_assertions)] - if let Some(original_type) = original_type { - if let Ok(rewritten_type) = rewritten.data.data_type(schema) { - assert_eq!( - rewritten_type, - original_type, - "Simplified expression should have the same data type as the original" - ); - } + if let Some(original_type) = original_type + && let Ok(rewritten_type) = rewritten.data.data_type(schema) { + assert_eq!( + rewritten_type, + original_type, + "Simplified expression should have the same data type as the original" + ); } Ok(rewritten) From de29e40e4090b7df95047f84ec66590d71dc6558 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 16:30:58 +0100 Subject: [PATCH 35/78] Autofix --- datafusion/core/tests/parquet/row_group_pruning.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index b4e5b65fb9ab0..35e2ec6cde7bc 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -383,12 +383,12 @@ async fn prune_disabled() { println!("{}", output.description()); // Row group stats pruning is disabled, so 0 row groups are pruned by statistics. - // However, page index pruning is still active (controlled by a separate - // enable_page_index setting, which defaults to true). Page index correctly prunes - // 1 row group whose pages all lie outside the filter range, leaving 3 for bloom - // filter evaluation. The query result is still correct. + // Bloom filter runs next and matches all 4 row groups (bloom filters don't help + // for range/inequality predicates like `nanos < threshold`). Page index pruning + // runs afterwards and can produce row-level selections, but those don't affect + // the bloom filter matched count. The query result is still correct. assert_eq!(output.predicate_evaluation_errors(), Some(0)); - assert_eq!(output.row_groups_matched(), Some(3)); + assert_eq!(output.row_groups_matched(), Some(4)); assert_eq!(output.row_groups_pruned(), Some(0)); assert_eq!( output.result_rows, From aa27a43ef50807e0d124eff42e8bb97880095607 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 16:49:26 +0100 Subject: [PATCH 36/78] CLippy --- datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs index 74329e41d1f93..beb414fc22375 100644 --- a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs +++ b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs @@ -236,7 +236,7 @@ impl RunQueryResult { self.query[order_by_start..limit_start] .trim() .split(',') - .map(|part| part.trim().split_whitespace().next().unwrap().to_string()) + .map(|part| part.split_whitespace().next().unwrap().to_string()) .collect() } From 9a4aa84fcb27e17ef10ca8c69646e72408b55375 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 17:20:30 +0100 Subject: [PATCH 37/78] Undo submodule --- datafusion-testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-testing b/datafusion-testing index 905df5f65cc9d..eccb0e4a42634 160000 --- a/datafusion-testing +++ b/datafusion-testing @@ -1 +1 @@ -Subproject commit 905df5f65cc9d0851719c21f5a4dd5cd77621f19 +Subproject commit eccb0e4a426344ef3faf534cd60e02e9c3afd3ac From 692bff6d3a74b69ed1e18df0db383bdd97d3d198 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 23:35:11 +0100 Subject: [PATCH 38/78] Also change open to be consistent --- datafusion/datasource-parquet/src/opener.rs | 90 ++++++++++++++++----- 1 file changed, 68 insertions(+), 22 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 77aa0cd779614..321eb4de03042 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -275,6 +275,7 @@ impl FileOpener for ParquetOpener { let predicate = self.predicate.clone(); let metrics = self.metrics.clone(); let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; + let enable_bloom_filter = self.enable_bloom_filter; let limit = self.limit; let preserve_order = self.preserve_order; @@ -312,7 +313,7 @@ impl FileOpener for ParquetOpener { let mut _metadata_timer = file_metrics.metadata_load_time.timer(); let reader_metadata = ArrowReaderMetadata::load_async(&mut async_file_reader, options).await?; - let metadata = reader_metadata.metadata(); + let metadata = Arc::clone(reader_metadata.metadata()); let num_row_groups = metadata.num_row_groups(); // Adapt the physical schema to the file schema for pruning @@ -375,6 +376,33 @@ impl FileOpener for ParquetOpener { row_groups.prune_by_limit(limit, metadata.row_groups(), &file_metrics); } + // Bloom filter pruning: done once per file here in morselize(), so that + // open() does not repeat it for each morsel (which would cause inflated metrics + // and unnecessary work). + if let Some(predicate) = pruning_predicate.as_deref() { + if enable_bloom_filter && !row_groups.is_empty() { + // Build a stream builder to access bloom filter data. + // This consumes `async_file_reader` and `reader_metadata`, which are + // no longer needed after this point. + let mut builder = ParquetRecordBatchStreamBuilder::new_with_metadata( + async_file_reader, + reader_metadata, + ); + row_groups + .prune_by_bloom_filters( + &physical_file_schema, + &mut builder, + predicate, + &file_metrics, + ) + .await; + } else { + file_metrics + .row_groups_pruned_bloom_filter + .add_matched(row_groups.remaining_row_group_count()); + } + } + let access_plan = row_groups.build(); let mut morsels = Vec::with_capacity(access_plan.len()); @@ -385,7 +413,7 @@ impl FileOpener for ParquetOpener { let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); morsel_access_plan.scan(i); let morsel = ParquetMorsel { - metadata: Arc::clone(metadata), + metadata: Arc::clone(&metadata), access_plan: morsel_access_plan, }; let mut f = partitioned_file.clone(); @@ -752,21 +780,31 @@ impl FileOpener for ParquetOpener { .add_matched(row_groups.remaining_row_group_count()); } - if enable_bloom_filter && !row_groups.is_empty() { - row_groups - .prune_by_bloom_filters( - &physical_file_schema, - &mut builder, - predicate, - &file_metrics, - ) - .await; - } else { - // Update metrics: bloom filter unavailable, so all row groups are - // matched (not pruned) - file_metrics - .row_groups_pruned_bloom_filter - .add_matched(row_groups.remaining_row_group_count()); + // Prune by limit before bloom filter: no point reading bloom filter data + // for row groups that will be skipped by the limit anyway. + if let (Some(limit), false) = (limit, preserve_order) { + row_groups.prune_by_limit(limit, rg_metadata, &file_metrics); + } + + // For morsels, bloom filter was already applied once in morselize(). + // Skip it here to avoid double-counting metrics and redundant I/O. + if !is_morsel { + if enable_bloom_filter && !row_groups.is_empty() { + row_groups + .prune_by_bloom_filters( + &physical_file_schema, + &mut builder, + predicate, + &file_metrics, + ) + .await; + } else { + // Update metrics: bloom filter unavailable, so all row groups are + // matched (not pruned) + file_metrics + .row_groups_pruned_bloom_filter + .add_matched(row_groups.remaining_row_group_count()); + } } } else { // Update metrics: no predicate, so all row groups are matched (not pruned) @@ -779,11 +817,6 @@ impl FileOpener for ParquetOpener { .add_matched(n_remaining_row_groups); } - // Prune by limit if limit is set and limit order is not sensitive - if let (Some(limit), false) = (limit, preserve_order) { - row_groups.prune_by_limit(limit, rg_metadata, &file_metrics); - } - // -------------------------------------------------------- // Step: prune pages from the kept row groups // @@ -2306,6 +2339,10 @@ mod test { &baseline_opener.metrics, "row_groups_pruned_statistics", ); + let baseline_bloom_metrics = get_pruning_metric( + &baseline_opener.metrics, + "row_groups_pruned_bloom_filter", + ); let morsel_opener = ParquetOpenerBuilder::new() .with_store(Arc::clone(&store)) @@ -2344,6 +2381,10 @@ mod test { &morsel_opener.metrics, "row_groups_pruned_statistics", ); + let morsel_bloom_metrics = get_pruning_metric( + &morsel_opener.metrics, + "row_groups_pruned_bloom_filter", + ); assert_eq!( baseline_values, morsel_values, @@ -2354,6 +2395,11 @@ mod test { baseline_stats_metrics, morsel_stats_metrics, "row_groups_pruned_statistics should be equivalent for open vs morselize path (enable_row_group_stats_pruning={enable_row_group_stats_pruning})" ); + + assert_eq!( + baseline_bloom_metrics, morsel_bloom_metrics, + "row_groups_pruned_bloom_filter should be equivalent for open vs morselize path (enable_row_group_stats_pruning={enable_row_group_stats_pruning})" + ); } } } From 9a9cf0b19da94d3a6b073235813eae6705e768bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 24 Feb 2026 23:59:33 +0100 Subject: [PATCH 39/78] Move page index back to morselize --- datafusion/datasource-parquet/src/opener.rs | 66 +++++++++++++++++---- 1 file changed, 56 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 321eb4de03042..e6586c5b94ff4 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -276,6 +276,7 @@ impl FileOpener for ParquetOpener { let metrics = self.metrics.clone(); let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; let enable_bloom_filter = self.enable_bloom_filter; + let enable_page_index = self.enable_page_index; let limit = self.limit; let preserve_order = self.preserve_order; @@ -311,10 +312,10 @@ impl FileOpener for ParquetOpener { } let mut _metadata_timer = file_metrics.metadata_load_time.timer(); - let reader_metadata = - ArrowReaderMetadata::load_async(&mut async_file_reader, options).await?; - let metadata = Arc::clone(reader_metadata.metadata()); - let num_row_groups = metadata.num_row_groups(); + let mut reader_metadata = + ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) + .await?; + let num_row_groups = reader_metadata.metadata().num_row_groups(); // Adapt the physical schema to the file schema for pruning let physical_file_schema = Arc::clone(reader_metadata.schema()); @@ -348,7 +349,7 @@ impl FileOpener for ParquetOpener { }) .transpose()?; - let (pruning_predicate, _) = build_pruning_predicates( + let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates( adapted_predicate.as_ref(), &physical_file_schema, &predicate_creation_errors, @@ -358,7 +359,7 @@ impl FileOpener for ParquetOpener { &file_name, extensions, num_row_groups, - metadata.row_groups(), + reader_metadata.metadata().row_groups(), file_range.as_ref(), pruning_predicate .as_deref() @@ -373,9 +374,31 @@ impl FileOpener for ParquetOpener { // Prune by limit if limit is set and order is not sensitive if let (Some(limit), false) = (limit, preserve_order) { - row_groups.prune_by_limit(limit, metadata.row_groups(), &file_metrics); + row_groups.prune_by_limit( + limit, + reader_metadata.metadata().row_groups(), + &file_metrics, + ); + } + + // Load page index after stats/limit pruning but before bloom filters. + // This avoids the I/O if all row groups are already pruned, and is still + // possible here because async_file_reader hasn't been consumed yet. + if should_enable_page_index(enable_page_index, &page_pruning_predicate) + && !row_groups.is_empty() + { + reader_metadata = load_page_index( + reader_metadata, + &mut async_file_reader, + options.with_page_index_policy(PageIndexPolicy::Optional), + ) + .await?; } + // Extract metadata after potentially loading the page index, so the cached + // metadata in each morsel includes the page index if it was loaded. + let metadata = Arc::clone(reader_metadata.metadata()); + // Bloom filter pruning: done once per file here in morselize(), so that // open() does not repeat it for each morsel (which would cause inflated metrics // and unnecessary work). @@ -403,7 +426,22 @@ impl FileOpener for ParquetOpener { } } - let access_plan = row_groups.build(); + let mut access_plan = row_groups.build(); + + // Page pruning: done once per file here in morselize(), so that open() + // does not repeat it for each morsel. + if enable_page_index + && !access_plan.is_empty() + && let Some(p) = page_pruning_predicate + { + access_plan = p.prune_plan_with_page_index( + access_plan, + &physical_file_schema, + metadata.file_metadata().schema_descr(), + metadata.as_ref(), + &file_metrics, + ); + } let mut morsels = Vec::with_capacity(access_plan.len()); for i in 0..num_row_groups { @@ -411,7 +449,8 @@ impl FileOpener for ParquetOpener { continue; } let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); - morsel_access_plan.scan(i); + // Transfer the page-pruned access (Scan or Selection) for this row group + morsel_access_plan.set(i, access_plan.inner()[i].clone()); let morsel = ParquetMorsel { metadata: Arc::clone(&metadata), access_plan: morsel_access_plan, @@ -690,7 +729,11 @@ impl FileOpener for ParquetOpener { // The page index is not stored inline in the parquet footer so the // code above may not have read the page index structures yet. If we // need them for reading and they aren't yet loaded, we need to load them now. - if should_enable_page_index(enable_page_index, &page_pruning_predicate) { + // For morsels, the page index was already loaded (if needed) in morselize(). + // Skip it here to avoid redundant I/O. + if should_enable_page_index(enable_page_index, &page_pruning_predicate) + && !is_morsel + { reader_metadata = load_page_index( reader_metadata, &mut async_file_reader, @@ -825,8 +868,11 @@ impl FileOpener for ParquetOpener { // be ruled using page metadata, rows from other columns // with that range can be skipped as well // -------------------------------------------------------- + // For morsels, page pruning was already applied once in morselize(). + // Skip it here to avoid double-counting metrics and redundant work. if enable_page_index && !access_plan.is_empty() + && !is_morsel && let Some(p) = page_pruning_predicate { access_plan = p.prune_plan_with_page_index( From f79fe63b2370ca9b5779b9922cbe327c390fcd16 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 25 Feb 2026 01:14:32 +0100 Subject: [PATCH 40/78] Move page index back to morselize --- datafusion/datasource-parquet/src/opener.rs | 56 +++++++++++++-------- 1 file changed, 34 insertions(+), 22 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index e6586c5b94ff4..cca6f43a7933c 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -279,6 +279,8 @@ impl FileOpener for ParquetOpener { let enable_page_index = self.enable_page_index; let limit = self.limit; let preserve_order = self.preserve_order; + let parquet_file_reader_factory = Arc::clone(&self.parquet_file_reader_factory); + let partition_index = self.partition_index; Box::pin(async move { #[cfg(feature = "parquet_encryption")] @@ -381,35 +383,20 @@ impl FileOpener for ParquetOpener { ); } - // Load page index after stats/limit pruning but before bloom filters. - // This avoids the I/O if all row groups are already pruned, and is still - // possible here because async_file_reader hasn't been consumed yet. - if should_enable_page_index(enable_page_index, &page_pruning_predicate) - && !row_groups.is_empty() - { - reader_metadata = load_page_index( - reader_metadata, - &mut async_file_reader, - options.with_page_index_policy(PageIndexPolicy::Optional), - ) - .await?; - } - - // Extract metadata after potentially loading the page index, so the cached - // metadata in each morsel includes the page index if it was loaded. - let metadata = Arc::clone(reader_metadata.metadata()); - // Bloom filter pruning: done once per file here in morselize(), so that // open() does not repeat it for each morsel (which would cause inflated metrics // and unnecessary work). + // + // Note: the bloom filter builder takes ownership of `async_file_reader`. + // Page index loading happens afterward using a fresh reader so that we only + // pay for the page index I/O on the row groups that survive bloom filter pruning. if let Some(predicate) = pruning_predicate.as_deref() { if enable_bloom_filter && !row_groups.is_empty() { - // Build a stream builder to access bloom filter data. - // This consumes `async_file_reader` and `reader_metadata`, which are - // no longer needed after this point. + // Clone reader_metadata so it remains available for page + // index loading after this builder is dropped. let mut builder = ParquetRecordBatchStreamBuilder::new_with_metadata( async_file_reader, - reader_metadata, + reader_metadata.clone(), ); row_groups .prune_by_bloom_filters( @@ -426,6 +413,31 @@ impl FileOpener for ParquetOpener { } } + // Load page index after bloom filter pruning so we skip it entirely if no + // row groups remain. Bloom filter building consumed `async_file_reader`, so + // we create a fresh reader here — reader creation is cheap (no I/O yet). + if should_enable_page_index(enable_page_index, &page_pruning_predicate) + && !row_groups.is_empty() + { + let mut fresh_reader: Box = + parquet_file_reader_factory.create_reader( + partition_index, + partitioned_file.clone(), + metadata_size_hint, + &metrics, + )?; + reader_metadata = load_page_index( + reader_metadata, + &mut fresh_reader, + options.with_page_index_policy(PageIndexPolicy::Optional), + ) + .await?; + } + + // Extract metadata after potentially loading the page index, so the cached + // metadata in each morsel includes the page index if it was loaded. + let metadata = Arc::clone(reader_metadata.metadata()); + let mut access_plan = row_groups.build(); // Page pruning: done once per file here in morselize(), so that open() From 976d8dca4b8d29f5f8a69abceb15eb2a3a562149 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 25 Feb 2026 01:45:15 +0100 Subject: [PATCH 41/78] Add back lost optimizations --- datafusion/datasource-parquet/src/opener.rs | 7 ++ datafusion/datasource/src/file_stream.rs | 103 +++++++++++++++--- .../sqllogictest/test_files/limit_pruning.slt | 2 +- 3 files changed, 97 insertions(+), 15 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index cca6f43a7933c..7d0f215b0ff87 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -229,6 +229,13 @@ impl ParquetOpener { } impl FileOpener for ParquetOpener { + fn is_leaf_morsel(&self, file: &PartitionedFile) -> bool { + file.extensions + .as_ref() + .map(|e| e.is::()) + .unwrap_or(false) + } + fn morselize( &self, partitioned_file: PartitionedFile, diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 1e0e8dfdc942c..872e7e5717b47 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -130,9 +130,16 @@ impl FileStream { /// /// Since file opening is mostly IO (and may involve a /// bunch of sequential IO), it can be parallelized with decoding. + /// + /// In morsel-driven mode this prefetches the next already-morselized item + /// from the shared queue (leaf morsels only — items that still need + /// async morselization are left in the queue for the normal Idle → + /// Morselizing path). fn start_next_file(&mut self) -> Option> { if self.morsel_driven { - return None; + let queue = Arc::clone(self.shared_queue.as_ref()?); + let morsel_file = queue.pull_if(|f| self.file_opener.is_leaf_morsel(f))?; + return Some(self.file_opener.open(morsel_file)); } let part_file = self.file_iter.pop_front()?; Some(self.file_opener.open(part_file)) @@ -148,12 +155,30 @@ impl FileStream { let queue = self.shared_queue.as_ref().expect("shared queue"); match queue.pull() { WorkStatus::Work(part_file) => { - self.morsel_guard = Some(MorselizingGuard { - queue: Arc::clone(queue), - }); - self.state = FileStreamState::Morselizing { - future: self.file_opener.morselize(*part_file), - }; + if self.file_opener.is_leaf_morsel(&part_file) { + // Fast path: already a leaf morsel — skip the + // Morselizing state entirely. Undo the count + // increment that pull() did since we won't be + // morselizing. + queue.stop_morselizing(); + match self.file_opener.open(*part_file) { + Ok(future) => { + self.state = FileStreamState::Open { future } + } + Err(e) => { + self.file_stream_metrics.time_opening.stop(); + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } else { + self.morsel_guard = Some(MorselizingGuard { + queue: Arc::clone(queue), + }); + self.state = FileStreamState::Morselizing { + future: self.file_opener.morselize(*part_file), + }; + } } WorkStatus::Wait => { self.file_stream_metrics.time_opening.stop(); @@ -193,10 +218,25 @@ impl FileStream { let _guard = self.morsel_guard.take(); if morsels.len() > 1 { - self.file_stream_metrics.time_opening.stop(); - // Expanded into multiple morsels. Put all back and pull again. - queue.push_many(morsels); - self.state = FileStreamState::Idle; + // Keep the first morsel for this worker; push the rest + // back so other workers can pick them up immediately. + // This avoids a round-trip through Idle just to re-claim + // one of the morsels we just created. + let mut iter = morsels.into_iter(); + let first = iter.next().unwrap(); + queue.push_many(iter.collect()); + // Don't stop time_opening here — it will be stopped + // naturally when we transition Open → Scan. + match self.file_opener.open(first) { + Ok(future) => { + self.state = FileStreamState::Open { future } + } + Err(e) => { + self.file_stream_metrics.time_opening.stop(); + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } } else if morsels.len() == 1 { // No further expansion possible. Proceed to open. let morsel = morsels.into_iter().next().unwrap(); @@ -438,6 +478,22 @@ impl WorkQueue { } } + /// Pull the front file from the queue only if `predicate` returns true for it. + /// + /// Does **not** increment `morselizing_count` — the caller must open the file + /// directly without going through the morselization state. + pub fn pull_if bool>( + &self, + predicate: F, + ) -> Option { + let mut queue = self.queue.lock().unwrap(); + if queue.front().map(predicate).unwrap_or(false) { + queue.pop_front() + } else { + None + } + } + /// Returns true if there is work in the queue or if all morselizing is done. pub fn has_work_or_done(&self) -> bool { let queue = self.queue.lock().unwrap(); @@ -460,10 +516,16 @@ impl WorkQueue { self.morselizing_count.fetch_add(1, Ordering::SeqCst); } - /// Decrement the morselizing count and notify waiters. + /// Decrement the morselizing count. Notifies waiting workers only when the + /// count reaches zero, since that is the point at which they may need to + /// re-evaluate whether all work is done. When count is still > 0, any new + /// morsels pushed to the queue already triggered a notification via + /// `push_many`, so no additional wakeup is needed here. pub fn stop_morselizing(&self) { - self.morselizing_count.fetch_sub(1, Ordering::SeqCst); - self.notify.notify_waiters(); + let prev = self.morselizing_count.fetch_sub(1, Ordering::AcqRel); + if prev == 1 { + self.notify.notify_waiters(); + } } /// Return true if any worker is currently morselizing. @@ -509,6 +571,19 @@ pub trait FileOpener: Unpin + Send + Sync { ) -> BoxFuture<'static, Result>> { Box::pin(futures::future::ready(Ok(vec![file]))) } + + /// Returns `true` if `file` is already a leaf morsel that can be opened + /// directly without going through [`Self::morselize`]. + /// + /// Returning `true` allows the [`FileStream`] to skip the async + /// `Morselizing` state and go straight to `Open`, and to prefetch the next + /// morsel while scanning the current one. + /// + /// The default implementation returns `false` (conservative — always + /// morselize). + fn is_leaf_morsel(&self, _file: &PartitionedFile) -> bool { + false + } } /// Represents the state of the next `FileOpenFuture`. Since we need to poll diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index 62790e6683049..037eb3de8a93b 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -72,7 +72,7 @@ explain analyze select * from tracking_data where species > 'M' AND s >= 50 orde ---- Plan with Metrics 01)SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=, output_bytes=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=7 total → 7 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (521/2.35 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (521/2.35 K)] statement ok drop table tracking_data; From f937f98dc5c111fcb982f98cb16a5d21dc9c15bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 25 Feb 2026 10:40:02 +0100 Subject: [PATCH 42/78] Tweak --- datafusion/datasource/src/file_stream.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 872e7e5717b47..03ca526d469ae 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -137,9 +137,8 @@ impl FileStream { /// Morselizing path). fn start_next_file(&mut self) -> Option> { if self.morsel_driven { - let queue = Arc::clone(self.shared_queue.as_ref()?); - let morsel_file = queue.pull_if(|f| self.file_opener.is_leaf_morsel(f))?; - return Some(self.file_opener.open(morsel_file)); + // In morsel-driven don't "prefetch" + return None; } let part_file = self.file_iter.pop_front()?; Some(self.file_opener.open(part_file)) From e0e85205d687fa1c347052d225dd94122e80a159 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 25 Feb 2026 18:51:25 +0100 Subject: [PATCH 43/78] Autofix --- .../physical-expr/src/utils/guarantee.rs | 32 ++++++++++++++++--- 1 file changed, 28 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/utils/guarantee.rs b/datafusion/physical-expr/src/utils/guarantee.rs index c4ce74fd3a573..85b63ee312e33 100644 --- a/datafusion/physical-expr/src/utils/guarantee.rs +++ b/datafusion/physical-expr/src/utils/guarantee.rs @@ -389,6 +389,9 @@ impl<'a> ColOpLit<'a> { /// 2. `literal col` /// 3. operator is `=` or `!=` /// + /// Also handles `CastColumnExpr(col) literal` patterns where the + /// column is wrapped in a cast (e.g., from schema adaptation). + /// /// Returns None otherwise fn try_new(expr: &'a Arc) -> Option { let binary_expr = expr @@ -405,9 +408,9 @@ impl<'a> ColOpLit<'a> { Operator::NotEq => Guarantee::NotIn, _ => return None, }; - // col literal + // col literal (also handles CastColumnExpr(col) literal) if let (Some(col), Some(lit)) = ( - left.downcast_ref::(), + extract_column(binary_expr.left()), right.downcast_ref::(), ) { Some(Self { @@ -416,10 +419,10 @@ impl<'a> ColOpLit<'a> { lit, }) } - // literal col + // literal col (also handles literal CastColumnExpr(col)) else if let (Some(lit), Some(col)) = ( left.downcast_ref::(), - right.downcast_ref::(), + extract_column(binary_expr.right()), ) { Some(Self { col, @@ -432,6 +435,27 @@ impl<'a> ColOpLit<'a> { } } +/// Extracts a [`Column`](crate::expressions::Column) reference from a physical +/// expression, looking through [`CastColumnExpr`](crate::expressions::CastColumnExpr) +/// wrappers. +fn extract_column( + expr: &Arc, +) -> Option<&crate::expressions::Column> { + if let Some(col) = expr.as_any().downcast_ref::() { + return Some(col); + } + if let Some(cast) = expr + .as_any() + .downcast_ref::() + { + return cast + .expr() + .as_any() + .downcast_ref::(); + } + None +} + /// Represents a single `col [not]in literal` expression struct ColInList<'a> { col: &'a crate::expressions::Column, From 25b044b6467a50173d4988d5317e9c94fef62e79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 25 Feb 2026 18:55:30 +0100 Subject: [PATCH 44/78] Fmt --- datafusion/physical-expr/src/utils/guarantee.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/utils/guarantee.rs b/datafusion/physical-expr/src/utils/guarantee.rs index 85b63ee312e33..70c83cee65b74 100644 --- a/datafusion/physical-expr/src/utils/guarantee.rs +++ b/datafusion/physical-expr/src/utils/guarantee.rs @@ -438,9 +438,7 @@ impl<'a> ColOpLit<'a> { /// Extracts a [`Column`](crate::expressions::Column) reference from a physical /// expression, looking through [`CastColumnExpr`](crate::expressions::CastColumnExpr) /// wrappers. -fn extract_column( - expr: &Arc, -) -> Option<&crate::expressions::Column> { +fn extract_column(expr: &Arc) -> Option<&crate::expressions::Column> { if let Some(col) = expr.as_any().downcast_ref::() { return Some(col); } From eb7dfa3d4ef5d0cb4bfd35540b980653c8f2eb1b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 27 Feb 2026 11:34:03 +0100 Subject: [PATCH 45/78] Use builder API --- datafusion/datasource/src/file_scan_config.rs | 15 +++++++++-- datafusion/datasource/src/memory.rs | 2 -- datafusion/datasource/src/source.rs | 26 ++++++++++++++----- 3 files changed, 33 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index d4b2efdfdb6f1..b89efb0696fa9 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -211,6 +211,8 @@ pub struct FileScanConfig { /// When true, use morsel-driven execution to avoid data skew. /// This means all partitions share a single pool of work. pub morsel_driven: bool, + /// Shared morsel queue, set via [`DataSource::with_shared_morsel_queue`]. + shared_morsel_queue: Option>, } /// A builder for [`FileScanConfig`]'s. @@ -581,6 +583,7 @@ impl FileScanConfigBuilder { statistics, partitioned_by_file_group, morsel_driven, + shared_morsel_queue: None, } } } @@ -610,7 +613,6 @@ impl DataSource for FileScanConfig { &self, partition: usize, context: Arc, - shared_morsel_queue: Option>, ) -> Result { let object_store = context.runtime_env().object_store(&self.object_store_url)?; let batch_size = self @@ -626,11 +628,20 @@ impl DataSource for FileScanConfig { partition, opener, source.metrics(), - shared_morsel_queue, + self.shared_morsel_queue.clone(), )?; Ok(Box::pin(cooperative(stream))) } + fn with_shared_morsel_queue( + &self, + queue: Option>, + ) -> Arc { + let mut config = self.clone(); + config.shared_morsel_queue = queue; + Arc::new(config) + } + fn as_any(&self) -> &dyn Any { self } diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 90c217ca1047e..1d12bb3200309 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -24,7 +24,6 @@ use std::ops::Deref; use std::slice::from_ref; use std::sync::Arc; -use crate::file_stream::WorkQueue; use crate::sink::DataSink; use crate::source::{DataSource, DataSourceExec}; @@ -81,7 +80,6 @@ impl DataSource for MemorySourceConfig { &self, partition: usize, _context: Arc, - _shared_morsel_queue: Option>, ) -> Result { Ok(Box::pin(cooperative( MemoryStream::try_new( diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index a7f7128931632..82aadfa7eb786 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -125,8 +125,19 @@ pub trait DataSource: Send + Sync + Debug { &self, partition: usize, context: Arc, - shared_morsel_queue: Option>, ) -> Result; + + /// Set a shared morsel queue for morsel-driven execution. + /// + /// The default implementation is a no-op. Override this in + /// implementations that support morsel-driven scheduling (e.g. + /// [`FileScanConfig`]). + fn with_shared_morsel_queue( + &self, + _queue: Option>, + ) -> Arc { + unimplemented!("with_shared_morsel_queue is not supported for this DataSource") + } fn as_any(&self) -> &dyn Any; /// Format this source for display in explain plans fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; @@ -346,11 +357,14 @@ impl ExecutionPlan for DataSourceExec { None }; - let stream = self.data_source.open( - partition, - Arc::clone(&context), - shared_morsel_queue, - )?; + let data_source = if shared_morsel_queue.is_some() { + self.data_source + .with_shared_morsel_queue(shared_morsel_queue) + } else { + Arc::clone(&self.data_source) + }; + + let stream = data_source.open(partition, Arc::clone(&context))?; let batch_size = context.session_config().batch_size(); log::debug!( "Batch splitting enabled for partition {partition}: batch_size={batch_size}" From 4ed7af3533d3257a7f6373f5a6d843d5397ff152 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 15:30:18 +0100 Subject: [PATCH 46/78] WIP --- .../custom_data_source/custom_datasource.rs | 1 - .../examples/data_io/json_shredding.rs | 5 +++ .../tests/fuzz_cases/topk_filter_pushdown.rs | 11 +++++- datafusion/datasource-parquet/src/opener.rs | 2 +- datafusion/datasource/src/file_stream.rs | 39 +++---------------- datafusion/datasource/src/source.rs | 26 ++++++++++--- 6 files changed, 41 insertions(+), 43 deletions(-) diff --git a/datafusion-examples/examples/custom_data_source/custom_datasource.rs b/datafusion-examples/examples/custom_data_source/custom_datasource.rs index 73ce6f5cf2002..2bca681030e3c 100644 --- a/datafusion-examples/examples/custom_data_source/custom_datasource.rs +++ b/datafusion-examples/examples/custom_data_source/custom_datasource.rs @@ -61,7 +61,6 @@ async fn search_accounts( filter: Option, expected_result_length: usize, ) -> Result<()> { - // create local execution context let config = SessionConfig::new() .set_bool("datafusion.execution.parquet.allow_morsel_driven", false); let ctx = SessionContext::new_with_config(config); diff --git a/datafusion-examples/examples/data_io/json_shredding.rs b/datafusion-examples/examples/data_io/json_shredding.rs index 1040b7d3df04e..58b75bb60234b 100644 --- a/datafusion-examples/examples/data_io/json_shredding.rs +++ b/datafusion-examples/examples/data_io/json_shredding.rs @@ -93,6 +93,11 @@ pub async fn json_shredding() -> Result<()> { // Set up query execution let mut cfg = SessionConfig::new(); cfg.options_mut().execution.parquet.pushdown_filters = true; + // Morsel-driven execution is disabled here because the example uses a small + // single-partition file with `set_max_row_group_size(2)`. Enabling it would + // split the file into per-row-group morsels that could be consumed by any + // partition, changing the output row order and breaking the deterministic + // `assert_batches_eq!` assertions below. cfg.options_mut().execution.parquet.allow_morsel_driven = false; let ctx = SessionContext::new_with_config(cfg); ctx.runtime_env().register_object_store( diff --git a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs index beb414fc22375..f233ff44b7785 100644 --- a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs +++ b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs @@ -272,8 +272,15 @@ impl RunQueryResult { // filter pushdown may legitimately return different tied rows. // // The dynamic filter must not change the *sort-key values* of the top-k - // result. We verify correctness by projecting both results down to only - // the ORDER BY columns and comparing those. + // result. We verify correctness by: + // 1. Checking the row counts match (wrong count is always a bug). + // 2. Projecting both results down to only the ORDER BY columns and + // comparing those (tied rows may differ, but the sort-key values must not). + let expected_rows: usize = self.expected.iter().map(|b| b.num_rows()).sum(); + let result_rows: usize = self.result.iter().map(|b| b.num_rows()).sum(); + if expected_rows != result_rows { + return false; + } let sort_cols = self.sort_columns(); let expected_keys = Self::project_columns(&self.expected, &sort_cols); let result_keys = Self::project_columns(&self.result, &sort_cols); diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 7d0f215b0ff87..7fb36967029aa 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -320,7 +320,7 @@ impl FileOpener for ParquetOpener { } } - let mut _metadata_timer = file_metrics.metadata_load_time.timer(); + let _metadata_timer = file_metrics.metadata_load_time.timer(); let mut reader_metadata = ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) .await?; diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 03ca526d469ae..39db489994d46 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -468,35 +468,21 @@ impl WorkQueue { pub fn pull(&self) -> WorkStatus { let mut queue = self.queue.lock().unwrap(); if let Some(file) = queue.pop_front() { - self.morselizing_count.fetch_add(1, Ordering::SeqCst); + // Relaxed: we hold the mutex, which provides the necessary memory barrier. + self.morselizing_count.fetch_add(1, Ordering::Relaxed); WorkStatus::Work(Box::new(file)) - } else if self.morselizing_count.load(Ordering::SeqCst) > 0 { + } else if self.morselizing_count.load(Ordering::Relaxed) > 0 { WorkStatus::Wait } else { WorkStatus::Done } } - /// Pull the front file from the queue only if `predicate` returns true for it. - /// - /// Does **not** increment `morselizing_count` — the caller must open the file - /// directly without going through the morselization state. - pub fn pull_if bool>( - &self, - predicate: F, - ) -> Option { - let mut queue = self.queue.lock().unwrap(); - if queue.front().map(predicate).unwrap_or(false) { - queue.pop_front() - } else { - None - } - } - /// Returns true if there is work in the queue or if all morselizing is done. pub fn has_work_or_done(&self) -> bool { let queue = self.queue.lock().unwrap(); - !queue.is_empty() || self.morselizing_count.load(Ordering::SeqCst) == 0 + // Relaxed: we hold the mutex, which provides the necessary memory barrier. + !queue.is_empty() || self.morselizing_count.load(Ordering::Relaxed) == 0 } /// Push many files back to the queue. @@ -510,11 +496,6 @@ impl WorkQueue { self.notify.notify_waiters(); } - /// Increment the morselizing count. - pub fn start_morselizing(&self) { - self.morselizing_count.fetch_add(1, Ordering::SeqCst); - } - /// Decrement the morselizing count. Notifies waiting workers only when the /// count reaches zero, since that is the point at which they may need to /// re-evaluate whether all work is done. When count is still > 0, any new @@ -526,16 +507,6 @@ impl WorkQueue { self.notify.notify_waiters(); } } - - /// Return true if any worker is currently morselizing. - pub fn is_morselizing(&self) -> bool { - self.morselizing_count.load(Ordering::SeqCst) > 0 - } - - /// Return a future that resolves when work is added or morselizing finishes. - pub async fn wait_for_work(&self) { - self.notify.notified().await; - } } /// A fallible future that resolves to a stream of [`RecordBatch`] diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 82aadfa7eb786..5dd3301ef2cd9 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -127,16 +127,25 @@ pub trait DataSource: Send + Sync + Debug { context: Arc, ) -> Result; - /// Set a shared morsel queue for morsel-driven execution. + /// Inject a shared morsel queue for morsel-driven execution. /// - /// The default implementation is a no-op. Override this in - /// implementations that support morsel-driven scheduling (e.g. - /// [`FileScanConfig`]). + /// **Internal use only.** This is called by [`DataSourceExec::execute`] on + /// [`FileScanConfig`] instances to distribute work across partitions. Custom + /// [`DataSource`] implementations do not need to override this method — it is + /// never called on non-[`FileScanConfig`] sources because [`DataSourceExec`] + /// only activates morsel-driven scheduling after a successful + /// `downcast_ref::()`. + /// + /// The default panics to catch accidental misuse. + #[doc(hidden)] fn with_shared_morsel_queue( &self, _queue: Option>, ) -> Arc { - unimplemented!("with_shared_morsel_queue is not supported for this DataSource") + panic!( + "with_shared_morsel_queue called on a DataSource that does not support it. \ + This is an internal method only called on FileScanConfig by DataSourceExec." + ) } fn as_any(&self) -> &dyn Any; /// Format this source for display in explain plans @@ -330,6 +339,13 @@ impl ExecutionPlan for DataSourceExec { // Start a new cycle once all expected partition streams for the // previous cycle have been opened. + // + // Limitation: this heuristic assumes every execution opens all + // `file_groups.len()` partitions. If a caller opens only a subset + // (e.g. partition 0 of 2 and then abandons the rest), the state + // remains stuck and the next execution reuses the stale queue. + // In normal DataFusion query execution all partitions are opened, + // so this is acceptable in practice. if state.expected_streams > 0 && state.streams_opened >= state.expected_streams { From b440313295df8d76767e3e14fdd22031478bea16 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 15:32:53 +0100 Subject: [PATCH 47/78] WIP --- datafusion/datasource/src/file_stream.rs | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 39db489994d46..b70fbd89a1990 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -468,10 +468,14 @@ impl WorkQueue { pub fn pull(&self) -> WorkStatus { let mut queue = self.queue.lock().unwrap(); if let Some(file) = queue.pop_front() { - // Relaxed: we hold the mutex, which provides the necessary memory barrier. + // Relaxed: the increment is done by the same task that will later call + // stop_morselizing(), so program order ensures the decrement sees it. self.morselizing_count.fetch_add(1, Ordering::Relaxed); WorkStatus::Work(Box::new(file)) - } else if self.morselizing_count.load(Ordering::Relaxed) > 0 { + } else if self.morselizing_count.load(Ordering::Acquire) > 0 { + // Acquire: stop_morselizing() uses AcqRel (a Release write) without + // holding the queue mutex, so we need Acquire here to synchronize with + // it on weakly-ordered architectures (e.g. ARM). WorkStatus::Wait } else { WorkStatus::Done @@ -481,8 +485,9 @@ impl WorkQueue { /// Returns true if there is work in the queue or if all morselizing is done. pub fn has_work_or_done(&self) -> bool { let queue = self.queue.lock().unwrap(); - // Relaxed: we hold the mutex, which provides the necessary memory barrier. - !queue.is_empty() || self.morselizing_count.load(Ordering::Relaxed) == 0 + // Acquire: stop_morselizing() writes morselizing_count with AcqRel outside + // the queue mutex, so Acquire is needed to synchronize with that Release. + !queue.is_empty() || self.morselizing_count.load(Ordering::Acquire) == 0 } /// Push many files back to the queue. From a11409cf37cdb1143fc25a392a0a038c1a6b8dd0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 16:01:54 +0100 Subject: [PATCH 48/78] Simplify morsel-driven execution code - Remove redundant `morsel_driven: bool` from `FileStream`; use `shared_queue.is_some()` everywhere it was checked - Unify duplicate `Morselizing` branches for `len > 1` and `len == 1`; `push_many` already no-ops on an empty iterator - Remove `Box` from `WorkStatus::Work` to avoid a heap allocation per morsel pulled from the queue - Replace `g.files().to_vec()` inside `flat_map` with `g.files().iter().cloned()` to avoid intermediate Vec allocations - Use `RecordBatch::project()` in `project_columns()` instead of manually reconstructing schema + columns - Reset `morsel_state` when cloning `DataSourceExec` in `handle_child_pushdown_result` so the new plan node has its own independent queue lifecycle Co-Authored-By: Claude Sonnet 4.6 --- .../tests/fuzz_cases/topk_filter_pushdown.rs | 10 +---- datafusion/datasource/src/file_stream.rs | 44 ++++++------------- datafusion/datasource/src/source.rs | 5 ++- 3 files changed, 20 insertions(+), 39 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs index f233ff44b7785..6950d75b0fdd6 100644 --- a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs +++ b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs @@ -245,17 +245,11 @@ impl RunQueryResult { batches .iter() .map(|b| { - let schema = b.schema(); let indices: Vec = cols .iter() - .filter_map(|c| schema.index_of(c).ok()) + .filter_map(|c| b.schema().index_of(c).ok()) .collect(); - let columns: Vec<_> = - indices.iter().map(|&i| Arc::clone(b.column(i))).collect(); - let fields: Vec<_> = - indices.iter().map(|&i| schema.field(i).clone()).collect(); - let new_schema = Arc::new(Schema::new(fields)); - RecordBatch::try_new(new_schema, columns).unwrap() + b.project(&indices).unwrap() }) .collect() } diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index b70fbd89a1990..a2242ff75c615 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -62,8 +62,6 @@ pub struct FileStream { file_iter: VecDeque, /// Shared work queue for morsel-driven execution. shared_queue: Option>, - /// Whether to use morsel-driven execution. - morsel_driven: bool, /// The stream schema (file schema including partition columns and after /// projection). projected_schema: SchemaRef, @@ -105,7 +103,6 @@ impl FileStream { Ok(Self { file_iter, shared_queue, - morsel_driven: config.morsel_driven, projected_schema, remain: config.limit, file_opener, @@ -136,7 +133,7 @@ impl FileStream { /// async morselization are left in the queue for the normal Idle → /// Morselizing path). fn start_next_file(&mut self) -> Option> { - if self.morsel_driven { + if self.shared_queue.is_some() { // In morsel-driven don't "prefetch" return None; } @@ -150,8 +147,7 @@ impl FileStream { FileStreamState::Idle => { self.file_stream_metrics.time_opening.start(); - if self.morsel_driven { - let queue = self.shared_queue.as_ref().expect("shared queue"); + if let Some(queue) = self.shared_queue.as_ref() { match queue.pull() { WorkStatus::Work(part_file) => { if self.file_opener.is_leaf_morsel(&part_file) { @@ -160,7 +156,7 @@ impl FileStream { // increment that pull() did since we won't be // morselizing. queue.stop_morselizing(); - match self.file_opener.open(*part_file) { + match self.file_opener.open(part_file) { Ok(future) => { self.state = FileStreamState::Open { future } } @@ -175,7 +171,7 @@ impl FileStream { queue: Arc::clone(queue), }); self.state = FileStreamState::Morselizing { - future: self.file_opener.morselize(*part_file), + future: self.file_opener.morselize(part_file), }; } } @@ -216,11 +212,16 @@ impl FileStream { // Take the guard to decrement morselizing_count let _guard = self.morsel_guard.take(); - if morsels.len() > 1 { + if morsels.is_empty() { + self.file_stream_metrics.time_opening.stop(); + // No morsels returned, skip this file + self.state = FileStreamState::Idle; + } else { // Keep the first morsel for this worker; push the rest // back so other workers can pick them up immediately. // This avoids a round-trip through Idle just to re-claim // one of the morsels we just created. + // push_many is a no-op when given an empty iterator (len == 1). let mut iter = morsels.into_iter(); let first = iter.next().unwrap(); queue.push_many(iter.collect()); @@ -236,23 +237,6 @@ impl FileStream { return Poll::Ready(Some(Err(e))); } } - } else if morsels.len() == 1 { - // No further expansion possible. Proceed to open. - let morsel = morsels.into_iter().next().unwrap(); - match self.file_opener.open(morsel) { - Ok(future) => { - self.state = FileStreamState::Open { future } - } - Err(e) => { - self.file_stream_metrics.time_opening.stop(); - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } - } else { - self.file_stream_metrics.time_opening.stop(); - // No morsels returned, skip this file - self.state = FileStreamState::Idle; } } Err(e) => { @@ -360,7 +344,7 @@ impl FileStream { } } None => { - if self.morsel_driven { + if self.shared_queue.is_some() { self.state = FileStreamState::Idle; } else { return Poll::Ready(None); @@ -395,7 +379,7 @@ impl FileStream { } } None => { - if self.morsel_driven { + if self.shared_queue.is_some() { self.state = FileStreamState::Idle; } else { return Poll::Ready(None); @@ -437,7 +421,7 @@ impl RecordBatchStream for FileStream { #[derive(Debug)] pub enum WorkStatus { /// A morsel is available - Work(Box), + Work(PartitionedFile), /// No morsel available now, but others are morselizing Wait, /// No more work available @@ -471,7 +455,7 @@ impl WorkQueue { // Relaxed: the increment is done by the same task that will later call // stop_morselizing(), so program order ensures the decrement sees it. self.morselizing_count.fetch_add(1, Ordering::Relaxed); - WorkStatus::Work(Box::new(file)) + WorkStatus::Work(file) } else if self.morselizing_count.load(Ordering::Acquire) > 0 { // Acquire: stop_morselizing() uses AcqRel (a Release write) without // holding the queue mutex, so we need Acquire here to synchronize with diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 5dd3301ef2cd9..d5bbe99ffb483 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -358,7 +358,7 @@ impl ExecutionPlan for DataSourceExec { let all_files = config .file_groups .iter() - .flat_map(|g| g.files().to_vec()) + .flat_map(|g| g.files().iter().cloned()) .collect(); state.queue = Some(Arc::new(WorkQueue::new(all_files))); state.expected_streams = config.file_groups.len(); @@ -454,6 +454,9 @@ impl ExecutionPlan for DataSourceExec { // Re-compute properties since we have new filters which will impact equivalence info new_node.cache = Arc::new(Self::compute_properties(&new_node.data_source)); + // Reset morsel state so this new plan node has its own independent + // queue lifecycle and does not share state with the original node. + new_node.morsel_state = Arc::new(Mutex::new(MorselState::default())); Ok(FilterPushdownPropagation { filters: res.filters, From e95c2d784dd248a2c7bb514139f589ca4d9c20ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 18:03:23 +0100 Subject: [PATCH 49/78] Move morsel queue ownership to DataSourceExecStream Use Weak on DataSourceExec so the queue is automatically cleaned up when all partition streams are dropped, removing the manual stream-counting heuristic in MorselState. This also eliminates a full FileScanConfig clone per execute() call. Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource/src/file_scan_config.rs | 33 ++-- datafusion/datasource/src/source.rs | 160 ++++++++---------- 2 files changed, 85 insertions(+), 108 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index b89efb0696fa9..db05e71c8f8d5 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -211,8 +211,6 @@ pub struct FileScanConfig { /// When true, use morsel-driven execution to avoid data skew. /// This means all partitions share a single pool of work. pub morsel_driven: bool, - /// Shared morsel queue, set via [`DataSource::with_shared_morsel_queue`]. - shared_morsel_queue: Option>, } /// A builder for [`FileScanConfig`]'s. @@ -583,7 +581,6 @@ impl FileScanConfigBuilder { statistics, partitioned_by_file_group, morsel_driven, - shared_morsel_queue: None, } } } @@ -608,11 +605,13 @@ impl From for FileScanConfigBuilder { } } -impl DataSource for FileScanConfig { - fn open( +impl FileScanConfig { + /// Open a partition stream with an optional shared morsel queue. + pub(crate) fn open_with_queue( &self, partition: usize, - context: Arc, + context: &Arc, + queue: Option>, ) -> Result { let object_store = context.runtime_env().object_store(&self.object_store_url)?; let batch_size = self @@ -620,26 +619,20 @@ impl DataSource for FileScanConfig { .unwrap_or_else(|| context.session_config().batch_size()); let source = self.file_source.with_batch_size(batch_size); - let opener = source.create_file_opener(object_store, self, partition)?; - let stream = FileStream::new( - self, - partition, - opener, - source.metrics(), - self.shared_morsel_queue.clone(), - )?; + let stream = FileStream::new(self, partition, opener, source.metrics(), queue)?; Ok(Box::pin(cooperative(stream))) } +} - fn with_shared_morsel_queue( +impl DataSource for FileScanConfig { + fn open( &self, - queue: Option>, - ) -> Arc { - let mut config = self.clone(); - config.shared_morsel_queue = queue; - Arc::new(config) + partition: usize, + context: Arc, + ) -> Result { + self.open_with_queue(partition, &context, None) } fn as_any(&self) -> &dyn Any { diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index d5bbe99ffb483..c79b7b770c3a6 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -20,7 +20,9 @@ use std::any::Any; use std::fmt; use std::fmt::{Debug, Formatter}; -use std::sync::{Arc, Mutex}; +use std::pin::Pin; +use std::sync::{Arc, Mutex, Weak}; +use std::task::{Context, Poll}; use datafusion_physical_expr::projection::ProjectionExprs; use datafusion_physical_plan::execution_plan::{ @@ -37,15 +39,18 @@ use itertools::Itertools; use crate::file_scan_config::FileScanConfig; use crate::file_stream::WorkQueue; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Result, Statistics}; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::SortOrderPushdownResult; use datafusion_physical_plan::filter_pushdown::{ ChildPushdownResult, FilterPushdownPhase, FilterPushdownPropagation, PushedDown, }; +use futures::Stream; /// A source of data, typically a list of files or memory /// @@ -127,26 +132,6 @@ pub trait DataSource: Send + Sync + Debug { context: Arc, ) -> Result; - /// Inject a shared morsel queue for morsel-driven execution. - /// - /// **Internal use only.** This is called by [`DataSourceExec::execute`] on - /// [`FileScanConfig`] instances to distribute work across partitions. Custom - /// [`DataSource`] implementations do not need to override this method — it is - /// never called on non-[`FileScanConfig`] sources because [`DataSourceExec`] - /// only activates morsel-driven scheduling after a successful - /// `downcast_ref::()`. - /// - /// The default panics to catch accidental misuse. - #[doc(hidden)] - fn with_shared_morsel_queue( - &self, - _queue: Option>, - ) -> Arc { - panic!( - "with_shared_morsel_queue called on a DataSource that does not support it. \ - This is an internal method only called on FileScanConfig by DataSourceExec." - ) - } fn as_any(&self) -> &dyn Any; /// Format this source for display in explain plans fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; @@ -253,15 +238,10 @@ pub struct DataSourceExec { data_source: Arc, /// Cached plan properties such as sort order cache: Arc, - /// Shared morsel queue for current execution lifecycle. - morsel_state: Arc>, -} - -#[derive(Debug, Default)] -struct MorselState { - queue: Option>, - streams_opened: usize, - expected_streams: usize, + /// Weak reference to the current morsel queue. When all + /// [`DataSourceExecStream`]s from an execution cycle are dropped the + /// strong count reaches zero and the next cycle creates a fresh queue. + morsel_queue: Arc>>, } impl DisplayAs for DataSourceExec { @@ -331,67 +311,49 @@ impl ExecutionPlan for DataSourceExec { partition: usize, context: Arc, ) -> Result { - let shared_morsel_queue = if let Some(config) = - self.data_source.as_any().downcast_ref::() - { - if config.morsel_driven { - let mut state = self.morsel_state.lock().unwrap(); - - // Start a new cycle once all expected partition streams for the - // previous cycle have been opened. - // - // Limitation: this heuristic assumes every execution opens all - // `file_groups.len()` partitions. If a caller opens only a subset - // (e.g. partition 0 of 2 and then abandons the rest), the state - // remains stuck and the next execution reuses the stale queue. - // In normal DataFusion query execution all partitions are opened, - // so this is acceptable in practice. - if state.expected_streams > 0 - && state.streams_opened >= state.expected_streams - { - state.queue = None; - state.streams_opened = 0; - state.expected_streams = 0; - } - - if state.queue.is_none() { - let all_files = config - .file_groups - .iter() - .flat_map(|g| g.files().iter().cloned()) - .collect(); - state.queue = Some(Arc::new(WorkQueue::new(all_files))); - state.expected_streams = config.file_groups.len(); + let morsel_config = self + .data_source + .as_any() + .downcast_ref::() + .filter(|c| c.morsel_driven); + + let (stream, queue) = if let Some(config) = morsel_config { + let queue = { + let mut guard = self.morsel_queue.lock().unwrap(); + match guard.upgrade() { + Some(q) => q, + None => { + let all_files = config + .file_groups + .iter() + .flat_map(|g| g.files().iter().cloned()) + .collect(); + let q = Arc::new(WorkQueue::new(all_files)); + *guard = Arc::downgrade(&q); + q + } } - - state.streams_opened += 1; - state.queue.as_ref().cloned() - } else { - None - } - } else { - None - }; - - let data_source = if shared_morsel_queue.is_some() { - self.data_source - .with_shared_morsel_queue(shared_morsel_queue) + }; + let stream = + config.open_with_queue(partition, &context, Some(Arc::clone(&queue)))?; + (stream, Some(queue)) } else { - Arc::clone(&self.data_source) + ( + self.data_source.open(partition, Arc::clone(&context))?, + None, + ) }; - let stream = data_source.open(partition, Arc::clone(&context))?; let batch_size = context.session_config().batch_size(); log::debug!( "Batch splitting enabled for partition {partition}: batch_size={batch_size}" ); let metrics = self.data_source.metrics(); let split_metrics = SplitMetrics::new(&metrics, partition); - Ok(Box::pin(BatchSplitStream::new( - stream, - batch_size, - split_metrics, - ))) + Ok(Box::pin(DataSourceExecStream { + inner: Box::pin(BatchSplitStream::new(stream, batch_size, split_metrics)), + _shared_queue: queue, + })) } fn metrics(&self) -> Option { @@ -409,7 +371,7 @@ impl ExecutionPlan for DataSourceExec { Some(Arc::new(Self { data_source, cache, - morsel_state: Arc::new(Mutex::new(MorselState::default())), + morsel_queue: Arc::new(Mutex::new(Weak::new())), })) } @@ -454,9 +416,7 @@ impl ExecutionPlan for DataSourceExec { // Re-compute properties since we have new filters which will impact equivalence info new_node.cache = Arc::new(Self::compute_properties(&new_node.data_source)); - // Reset morsel state so this new plan node has its own independent - // queue lifecycle and does not share state with the original node. - new_node.morsel_state = Arc::new(Mutex::new(MorselState::default())); + new_node.morsel_queue = Arc::new(Mutex::new(Weak::new())); Ok(FilterPushdownPropagation { filters: res.filters, @@ -496,6 +456,30 @@ impl ExecutionPlan for DataSourceExec { } } +struct DataSourceExecStream { + inner: SendableRecordBatchStream, + /// Holds a strong reference to the morsel queue so it stays alive + /// as long as any partition stream exists. + _shared_queue: Option>, +} + +impl Stream for DataSourceExecStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + self.inner.as_mut().poll_next(cx) + } +} + +impl RecordBatchStream for DataSourceExecStream { + fn schema(&self) -> SchemaRef { + self.inner.schema() + } +} + impl DataSourceExec { pub fn from_data_source(data_source: impl DataSource + 'static) -> Arc { Arc::new(Self::new(Arc::new(data_source))) @@ -507,7 +491,7 @@ impl DataSourceExec { Self { data_source, cache: Arc::new(cache), - morsel_state: Arc::new(Mutex::new(MorselState::default())), + morsel_queue: Arc::new(Mutex::new(Weak::new())), } } @@ -519,7 +503,7 @@ impl DataSourceExec { pub fn with_data_source(mut self, data_source: Arc) -> Self { self.cache = Arc::new(Self::compute_properties(&data_source)); self.data_source = data_source; - self.morsel_state = Arc::new(Mutex::new(MorselState::default())); + self.morsel_queue = Arc::new(Mutex::new(Weak::new())); self } From b71a2a70b70ed56d1cfa0d7034ea1bb1c47fcf4f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 18:14:45 +0100 Subject: [PATCH 50/78] Use Arc::strong_count instead of Weak for morsel queue lifecycle Weak expires too early when partition streams are opened and consumed sequentially, causing each partition to create its own queue with all files. Use Option> with strong_count to detect when all previous streams have been dropped. Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource/src/source.rs | 29 ++++++++++++++++------------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index c79b7b770c3a6..4d59f5bff3e8c 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -21,7 +21,7 @@ use std::any::Any; use std::fmt; use std::fmt::{Debug, Formatter}; use std::pin::Pin; -use std::sync::{Arc, Mutex, Weak}; +use std::sync::{Arc, Mutex}; use std::task::{Context, Poll}; use datafusion_physical_expr::projection::ProjectionExprs; @@ -238,10 +238,10 @@ pub struct DataSourceExec { data_source: Arc, /// Cached plan properties such as sort order cache: Arc, - /// Weak reference to the current morsel queue. When all - /// [`DataSourceExecStream`]s from an execution cycle are dropped the - /// strong count reaches zero and the next cycle creates a fresh queue. - morsel_queue: Arc>>, + /// Shared morsel queue for the current execution cycle. A fresh queue + /// is created when `Arc::strong_count` is 1 (only this field holds it), + /// meaning all previous [`DataSourceExecStream`]s have been dropped. + morsel_queue: Arc>>>, } impl DisplayAs for DataSourceExec { @@ -320,16 +320,19 @@ impl ExecutionPlan for DataSourceExec { let (stream, queue) = if let Some(config) = morsel_config { let queue = { let mut guard = self.morsel_queue.lock().unwrap(); - match guard.upgrade() { - Some(q) => q, - None => { + match &*guard { + // Reuse the queue if other streams still hold references. + Some(q) if Arc::strong_count(q) > 1 => Arc::clone(q), + // No queue yet, or all previous streams have been dropped + // (strong_count == 1, only this field holds it) — create fresh. + _ => { let all_files = config .file_groups .iter() .flat_map(|g| g.files().iter().cloned()) .collect(); let q = Arc::new(WorkQueue::new(all_files)); - *guard = Arc::downgrade(&q); + *guard = Some(Arc::clone(&q)); q } } @@ -371,7 +374,7 @@ impl ExecutionPlan for DataSourceExec { Some(Arc::new(Self { data_source, cache, - morsel_queue: Arc::new(Mutex::new(Weak::new())), + morsel_queue: Arc::new(Mutex::new(None)), })) } @@ -416,7 +419,7 @@ impl ExecutionPlan for DataSourceExec { // Re-compute properties since we have new filters which will impact equivalence info new_node.cache = Arc::new(Self::compute_properties(&new_node.data_source)); - new_node.morsel_queue = Arc::new(Mutex::new(Weak::new())); + new_node.morsel_queue = Arc::new(Mutex::new(None)); Ok(FilterPushdownPropagation { filters: res.filters, @@ -491,7 +494,7 @@ impl DataSourceExec { Self { data_source, cache: Arc::new(cache), - morsel_queue: Arc::new(Mutex::new(Weak::new())), + morsel_queue: Arc::new(Mutex::new(None)), } } @@ -503,7 +506,7 @@ impl DataSourceExec { pub fn with_data_source(mut self, data_source: Arc) -> Self { self.cache = Arc::new(Self::compute_properties(&data_source)); self.data_source = data_source; - self.morsel_queue = Arc::new(Mutex::new(Weak::new())); + self.morsel_queue = Arc::new(Mutex::new(None)); self } From c383e6f9a820aafcfd98079f68a4840420174809 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 18:21:49 +0100 Subject: [PATCH 51/78] Fix morsel queue reset using partition counter Arc::strong_count cannot distinguish between a stream dropped within the same execution cycle and all streams from a previous cycle being done. Use a remaining-partitions counter instead: the queue is reused until all expected partitions have been opened, then reset on the next execute() call. Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource/src/source.rs | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 4d59f5bff3e8c..284d9ada8479c 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -238,10 +238,10 @@ pub struct DataSourceExec { data_source: Arc, /// Cached plan properties such as sort order cache: Arc, - /// Shared morsel queue for the current execution cycle. A fresh queue - /// is created when `Arc::strong_count` is 1 (only this field holds it), - /// meaning all previous [`DataSourceExecStream`]s have been dropped. - morsel_queue: Arc>>>, + /// Shared morsel queue and remaining partition count for the current + /// execution cycle. A fresh queue is created once all expected + /// partitions have been opened (remaining reaches 0). + morsel_queue: Arc, usize)>>>, } impl DisplayAs for DataSourceExec { @@ -320,11 +320,11 @@ impl ExecutionPlan for DataSourceExec { let (stream, queue) = if let Some(config) = morsel_config { let queue = { let mut guard = self.morsel_queue.lock().unwrap(); - match &*guard { - // Reuse the queue if other streams still hold references. - Some(q) if Arc::strong_count(q) > 1 => Arc::clone(q), - // No queue yet, or all previous streams have been dropped - // (strong_count == 1, only this field holds it) — create fresh. + match guard.as_mut() { + Some((q, remaining)) if *remaining > 0 => { + *remaining -= 1; + Arc::clone(q) + } _ => { let all_files = config .file_groups @@ -332,7 +332,8 @@ impl ExecutionPlan for DataSourceExec { .flat_map(|g| g.files().iter().cloned()) .collect(); let q = Arc::new(WorkQueue::new(all_files)); - *guard = Some(Arc::clone(&q)); + let remaining = config.file_groups.len().saturating_sub(1); + *guard = Some((Arc::clone(&q), remaining)); q } } From ccd21c848e71f84f67d43b9daaa25546d7897874 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 18:26:31 +0100 Subject: [PATCH 52/78] Add FileStream::with_shared_queue builder method Remove the shared_queue parameter from FileStream::new() to avoid an API change. The queue is now set via with_shared_queue() after construction, following the same pattern as with_on_error(). Co-Authored-By: Claude Opus 4.6 --- .../custom_data_source/csv_json_opener.rs | 2 -- datafusion/datasource/src/file_scan_config.rs | 6 +++++- datafusion/datasource/src/file_stream.rs | 17 +++++++++++------ 3 files changed, 16 insertions(+), 9 deletions(-) diff --git a/datafusion-examples/examples/custom_data_source/csv_json_opener.rs b/datafusion-examples/examples/custom_data_source/csv_json_opener.rs index 008cb7db88e2d..d025a8f675ddb 100644 --- a/datafusion-examples/examples/custom_data_source/csv_json_opener.rs +++ b/datafusion-examples/examples/custom_data_source/csv_json_opener.rs @@ -85,7 +85,6 @@ async fn csv_opener() -> Result<()> { 0, opener, &ExecutionPlanMetricsSet::new(), - None, )?; while let Some(batch) = stream.next().await.transpose()? { result.push(batch); @@ -147,7 +146,6 @@ async fn json_opener() -> Result<()> { 0, Arc::new(opener), &ExecutionPlanMetricsSet::new(), - None, )?; let mut result = vec![]; while let Some(batch) = stream.next().await.transpose()? { diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index db05e71c8f8d5..eb6b37a8a8901 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -621,7 +621,11 @@ impl FileScanConfig { let source = self.file_source.with_batch_size(batch_size); let opener = source.create_file_opener(object_store, self, partition)?; - let stream = FileStream::new(self, partition, opener, source.metrics(), queue)?; + let stream = FileStream::new(self, partition, opener, source.metrics())?; + let stream = match queue { + Some(q) => stream.with_shared_queue(q), + None => stream, + }; Ok(Box::pin(cooperative(stream))) } } diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index a2242ff75c615..0f64e98c1d47b 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -89,20 +89,19 @@ impl FileStream { partition: usize, file_opener: Arc, metrics: &ExecutionPlanMetricsSet, - shared_queue: Option>, ) -> Result { let projected_schema = config.projected_schema()?; - let (file_iter, shared_queue) = if config.morsel_driven { - (VecDeque::new(), shared_queue) + let file_iter = if config.morsel_driven { + VecDeque::new() } else { let file_group = config.file_groups[partition].clone(); - (file_group.into_inner().into_iter().collect(), None) + file_group.into_inner().into_iter().collect() }; Ok(Self { file_iter, - shared_queue, + shared_queue: None, projected_schema, remain: config.limit, file_opener, @@ -114,6 +113,12 @@ impl FileStream { }) } + /// Set the shared work queue for morsel-driven execution. + pub fn with_shared_queue(mut self, queue: Arc) -> Self { + self.shared_queue = Some(queue); + self + } + /// Specify the behavior when an error occurs opening or scanning a file /// /// If `OnError::Skip` the stream will skip files which encounter an error and continue @@ -841,7 +846,7 @@ mod tests { .build(); let metrics_set = ExecutionPlanMetricsSet::new(); let file_stream = - FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set, None) + FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set) .unwrap() .with_on_error(on_error); From 763fff25788c0c87e378a8fd24ef5dc7c73c1399 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 18:40:55 +0100 Subject: [PATCH 53/78] Use TaskContext::query_id to detect morsel queue execution cycles Add a unique query_id to TaskContext (auto-assigned via global atomic counter). Use it in DataSourceExec to detect when a new execution cycle starts, replacing the fragile partition counter. Since all partitions of the same query share one Arc, the ID is stable within a cycle and changes between cycles. Co-Authored-By: Claude Opus 4.6 --- .../custom_data_source/csv_json_opener.rs | 8 +-- datafusion/datasource/src/source.rs | 44 ++++---------- datafusion/execution/src/task.rs | 57 +++++++++++++++++++ 3 files changed, 70 insertions(+), 39 deletions(-) diff --git a/datafusion-examples/examples/custom_data_source/csv_json_opener.rs b/datafusion-examples/examples/custom_data_source/csv_json_opener.rs index d025a8f675ddb..fc1130313e00c 100644 --- a/datafusion-examples/examples/custom_data_source/csv_json_opener.rs +++ b/datafusion-examples/examples/custom_data_source/csv_json_opener.rs @@ -80,12 +80,8 @@ async fn csv_opener() -> Result<()> { .create_file_opener(object_store, &scan_config, 0)?; let mut result = vec![]; - let mut stream = FileStream::new( - &scan_config, - 0, - opener, - &ExecutionPlanMetricsSet::new(), - )?; + let mut stream = + FileStream::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new())?; while let Some(batch) = stream.next().await.transpose()? { result.push(batch); } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 284d9ada8479c..0db0b38baadb7 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -21,7 +21,7 @@ use std::any::Any; use std::fmt; use std::fmt::{Debug, Formatter}; use std::pin::Pin; -use std::sync::{Arc, Mutex}; +use std::sync::Arc; use std::task::{Context, Poll}; use datafusion_physical_expr::projection::ProjectionExprs; @@ -238,10 +238,6 @@ pub struct DataSourceExec { data_source: Arc, /// Cached plan properties such as sort order cache: Arc, - /// Shared morsel queue and remaining partition count for the current - /// execution cycle. A fresh queue is created once all expected - /// partitions have been opened (remaining reaches 0). - morsel_queue: Arc, usize)>>>, } impl DisplayAs for DataSourceExec { @@ -318,26 +314,15 @@ impl ExecutionPlan for DataSourceExec { .filter(|c| c.morsel_driven); let (stream, queue) = if let Some(config) = morsel_config { - let queue = { - let mut guard = self.morsel_queue.lock().unwrap(); - match guard.as_mut() { - Some((q, remaining)) if *remaining > 0 => { - *remaining -= 1; - Arc::clone(q) - } - _ => { - let all_files = config - .file_groups - .iter() - .flat_map(|g| g.files().iter().cloned()) - .collect(); - let q = Arc::new(WorkQueue::new(all_files)); - let remaining = config.file_groups.len().saturating_sub(1); - *guard = Some((Arc::clone(&q), remaining)); - q - } - } - }; + let key = Arc::as_ptr(&self.data_source) as *const () as usize; + let queue = context.get_or_insert_shared_state(key, || { + let all_files = config + .file_groups + .iter() + .flat_map(|g| g.files().iter().cloned()) + .collect(); + WorkQueue::new(all_files) + }); let stream = config.open_with_queue(partition, &context, Some(Arc::clone(&queue)))?; (stream, Some(queue)) @@ -372,11 +357,7 @@ impl ExecutionPlan for DataSourceExec { let data_source = self.data_source.with_fetch(limit)?; let cache = Arc::clone(&self.cache); - Some(Arc::new(Self { - data_source, - cache, - morsel_queue: Arc::new(Mutex::new(None)), - })) + Some(Arc::new(Self { data_source, cache })) } fn fetch(&self) -> Option { @@ -420,7 +401,6 @@ impl ExecutionPlan for DataSourceExec { // Re-compute properties since we have new filters which will impact equivalence info new_node.cache = Arc::new(Self::compute_properties(&new_node.data_source)); - new_node.morsel_queue = Arc::new(Mutex::new(None)); Ok(FilterPushdownPropagation { filters: res.filters, @@ -495,7 +475,6 @@ impl DataSourceExec { Self { data_source, cache: Arc::new(cache), - morsel_queue: Arc::new(Mutex::new(None)), } } @@ -507,7 +486,6 @@ impl DataSourceExec { pub fn with_data_source(mut self, data_source: Arc) -> Self { self.cache = Arc::new(Self::compute_properties(&data_source)); self.data_source = data_source; - self.morsel_queue = Arc::new(Mutex::new(None)); self } diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 38f31cf4629eb..592bd4a5e429f 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -22,9 +22,25 @@ use crate::{ use datafusion_common::{Result, internal_datafusion_err, plan_datafusion_err}; use datafusion_expr::planner::ExprPlanner; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use std::any::Any; use std::collections::HashSet; +use std::fmt; +use std::sync::Mutex; +use std::sync::atomic::{AtomicUsize, Ordering}; use std::{collections::HashMap, sync::Arc}; +static NEXT_QUERY_ID: AtomicUsize = AtomicUsize::new(0); + +/// Type-erased shared state map used by execution plan nodes to share +/// state across partitions within the same query execution. +struct SharedState(Mutex>>); + +impl fmt::Debug for SharedState { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.write_str("SharedState") + } +} + /// Task Execution Context /// /// A [`TaskContext`] contains the state required during a single query's @@ -38,6 +54,9 @@ pub struct TaskContext { session_id: String, /// Optional Task Identify task_id: Option, + /// Unique identifier for this query execution, used to detect + /// execution cycle boundaries in morsel-driven scheduling. + query_id: usize, /// Session configuration session_config: SessionConfig, /// Scalar functions associated with this task context @@ -48,6 +67,10 @@ pub struct TaskContext { window_functions: HashMap>, /// Runtime environment associated with this task context runtime: Arc, + /// Shared state for execution plan nodes within this query execution. + /// Keyed by a caller-chosen identifier (e.g. pointer address of a plan + /// node's `Arc`). + shared_state: SharedState, } impl Default for TaskContext { @@ -58,11 +81,13 @@ impl Default for TaskContext { Self { session_id: "DEFAULT".to_string(), task_id: None, + query_id: NEXT_QUERY_ID.fetch_add(1, Ordering::Relaxed), session_config: SessionConfig::new(), scalar_functions: HashMap::new(), aggregate_functions: HashMap::new(), window_functions: HashMap::new(), runtime, + shared_state: SharedState(Mutex::new(HashMap::new())), } } } @@ -85,11 +110,13 @@ impl TaskContext { Self { task_id, session_id, + query_id: NEXT_QUERY_ID.fetch_add(1, Ordering::Relaxed), session_config, scalar_functions, aggregate_functions, window_functions, runtime, + shared_state: SharedState(Mutex::new(HashMap::new())), } } @@ -108,6 +135,15 @@ impl TaskContext { self.task_id.clone() } + /// Return the `query_id` of this [TaskContext]. + /// + /// Each [`TaskContext`] is assigned a unique query ID at construction. + /// All partitions of the same query execution share the same + /// [`TaskContext`] (via `Arc`), so the ID is stable within one cycle. + pub fn query_id(&self) -> usize { + self.query_id + } + /// Return the [`MemoryPool`] associated with this [TaskContext] pub fn memory_pool(&self) -> &Arc { &self.runtime.memory_pool @@ -136,6 +172,27 @@ impl TaskContext { self } + /// Get or create shared state for a given key. + /// + /// Execution plan nodes use this to share state (e.g. work queues) + /// across partitions within the same query execution. The key is + /// typically derived from a stable pointer (e.g. `Arc::as_ptr`). + pub fn get_or_insert_shared_state( + &self, + key: usize, + create: impl FnOnce() -> T, + ) -> Arc { + let mut map = self.shared_state.0.lock().unwrap(); + if let Some(existing) = map.get(&key) { + if let Ok(typed) = Arc::clone(existing).downcast::() { + return typed; + } + } + let value = Arc::new(create()); + map.insert(key, Arc::clone(&value) as Arc); + value + } + /// Update the [`RuntimeEnv`] pub fn with_runtime(mut self, runtime: Arc) -> Self { self.runtime = runtime; From 2e2b68be0ad42adbc53be964195114f763330f57 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 19:33:51 +0100 Subject: [PATCH 54/78] Remove query_id --- datafusion/execution/src/task.rs | 7 ------- 1 file changed, 7 deletions(-) diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 592bd4a5e429f..bec9156925484 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -29,8 +29,6 @@ use std::sync::Mutex; use std::sync::atomic::{AtomicUsize, Ordering}; use std::{collections::HashMap, sync::Arc}; -static NEXT_QUERY_ID: AtomicUsize = AtomicUsize::new(0); - /// Type-erased shared state map used by execution plan nodes to share /// state across partitions within the same query execution. struct SharedState(Mutex>>); @@ -54,9 +52,6 @@ pub struct TaskContext { session_id: String, /// Optional Task Identify task_id: Option, - /// Unique identifier for this query execution, used to detect - /// execution cycle boundaries in morsel-driven scheduling. - query_id: usize, /// Session configuration session_config: SessionConfig, /// Scalar functions associated with this task context @@ -81,7 +76,6 @@ impl Default for TaskContext { Self { session_id: "DEFAULT".to_string(), task_id: None, - query_id: NEXT_QUERY_ID.fetch_add(1, Ordering::Relaxed), session_config: SessionConfig::new(), scalar_functions: HashMap::new(), aggregate_functions: HashMap::new(), @@ -110,7 +104,6 @@ impl TaskContext { Self { task_id, session_id, - query_id: NEXT_QUERY_ID.fetch_add(1, Ordering::Relaxed), session_config, scalar_functions, aggregate_functions, From 6ee74d64430a798445f301348a0f593b46b38482 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 19:47:03 +0100 Subject: [PATCH 55/78] Change tests --- .../examples/custom_data_source/custom_datasource.rs | 4 +--- .../core/tests/physical_optimizer/partition_statistics.rs | 4 +--- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/datafusion-examples/examples/custom_data_source/custom_datasource.rs b/datafusion-examples/examples/custom_data_source/custom_datasource.rs index 2bca681030e3c..1cc99517cc07e 100644 --- a/datafusion-examples/examples/custom_data_source/custom_datasource.rs +++ b/datafusion-examples/examples/custom_data_source/custom_datasource.rs @@ -61,9 +61,7 @@ async fn search_accounts( filter: Option, expected_result_length: usize, ) -> Result<()> { - let config = SessionConfig::new() - .set_bool("datafusion.execution.parquet.allow_morsel_driven", false); - let ctx = SessionContext::new_with_config(config); + let ctx = SessionContext::new(); // create logical plan composed of a single TableScan let logical_plan = LogicalPlanBuilder::scan_with_filters( "accounts", diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index b04090f0dc813..fa021ed3dcce3 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -77,9 +77,7 @@ mod test { create_table_sql: Option<&str>, target_partition: Option, ) -> Arc { - let mut session_config = SessionConfig::new() - .with_collect_statistics(true) - .set_bool("datafusion.execution.parquet.allow_morsel_driven", false); + let mut session_config = SessionConfig::new().with_collect_statistics(true); if let Some(partition) = target_partition { session_config = session_config.with_target_partitions(partition); } From 040eacc8151714908bd80a351d34f69db486bb8e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 19:49:58 +0100 Subject: [PATCH 56/78] Remove allow_morsel_driven=false workarounds from tests and examples These tests don't need morsel-driven execution disabled: - custom_datasource: uses a custom ExecutionPlan, not file-based - partition_statistics: only checks statistics metadata - json_shredding: single-row filtered result is order-independent Also remove leftover query_id getter and unused atomic imports. Co-Authored-By: Claude Opus 4.6 --- .../examples/custom_data_source/custom_datasource.rs | 2 ++ datafusion-examples/examples/data_io/json_shredding.rs | 6 ------ datafusion/execution/src/task.rs | 10 ---------- 3 files changed, 2 insertions(+), 16 deletions(-) diff --git a/datafusion-examples/examples/custom_data_source/custom_datasource.rs b/datafusion-examples/examples/custom_data_source/custom_datasource.rs index 1cc99517cc07e..7abb39e1a7130 100644 --- a/datafusion-examples/examples/custom_data_source/custom_datasource.rs +++ b/datafusion-examples/examples/custom_data_source/custom_datasource.rs @@ -61,7 +61,9 @@ async fn search_accounts( filter: Option, expected_result_length: usize, ) -> Result<()> { + // create local execution context let ctx = SessionContext::new(); + // create logical plan composed of a single TableScan let logical_plan = LogicalPlanBuilder::scan_with_filters( "accounts", diff --git a/datafusion-examples/examples/data_io/json_shredding.rs b/datafusion-examples/examples/data_io/json_shredding.rs index 58b75bb60234b..77dba5a98ac6f 100644 --- a/datafusion-examples/examples/data_io/json_shredding.rs +++ b/datafusion-examples/examples/data_io/json_shredding.rs @@ -93,12 +93,6 @@ pub async fn json_shredding() -> Result<()> { // Set up query execution let mut cfg = SessionConfig::new(); cfg.options_mut().execution.parquet.pushdown_filters = true; - // Morsel-driven execution is disabled here because the example uses a small - // single-partition file with `set_max_row_group_size(2)`. Enabling it would - // split the file into per-row-group morsels that could be consumed by any - // partition, changing the output row order and breaking the deterministic - // `assert_batches_eq!` assertions below. - cfg.options_mut().execution.parquet.allow_morsel_driven = false; let ctx = SessionContext::new_with_config(cfg); ctx.runtime_env().register_object_store( ObjectStoreUrl::parse("memory://")?.as_ref(), diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index bec9156925484..0f797ffcba303 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -26,7 +26,6 @@ use std::any::Any; use std::collections::HashSet; use std::fmt; use std::sync::Mutex; -use std::sync::atomic::{AtomicUsize, Ordering}; use std::{collections::HashMap, sync::Arc}; /// Type-erased shared state map used by execution plan nodes to share @@ -128,15 +127,6 @@ impl TaskContext { self.task_id.clone() } - /// Return the `query_id` of this [TaskContext]. - /// - /// Each [`TaskContext`] is assigned a unique query ID at construction. - /// All partitions of the same query execution share the same - /// [`TaskContext`] (via `Arc`), so the ID is stable within one cycle. - pub fn query_id(&self) -> usize { - self.query_id - } - /// Return the [`MemoryPool`] associated with this [TaskContext] pub fn memory_pool(&self) -> &Arc { &self.runtime.memory_pool From 06e254b2533c714d29eab5634ebbd677589afa09 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 20:11:33 +0100 Subject: [PATCH 57/78] Plan whole files instead of byte-range splits for morsel-driven execution When morsel-driven execution is enabled, the WorkQueue handles load balancing at runtime, making byte-range file splitting unnecessary. Distribute whole files round-robin across target partitions instead. Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource/src/file_scan_config.rs | 23 ++++++ .../dynamic_filter_pushdown_config.slt | 12 +-- .../sqllogictest/test_files/explain_tree.slt | 21 ++---- .../sqllogictest/test_files/limit_pruning.slt | 9 ++- .../sqllogictest/test_files/parquet.slt | 12 +-- .../test_files/parquet_filter_pushdown.slt | 74 +++++++++---------- .../test_files/parquet_statistics.slt | 9 +-- .../test_files/preserve_file_partitioning.slt | 10 +-- .../sqllogictest/test_files/projection.slt | 3 +- .../test_files/projection_pushdown.slt | 6 +- .../test_files/push_down_filter_parquet.slt | 19 +++-- .../push_down_filter_regression.slt | 8 +- .../test_files/repartition_scan.slt | 13 ++-- 13 files changed, 109 insertions(+), 110 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index eb6b37a8a8901..92e61aa16cf4b 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -718,6 +718,29 @@ impl DataSource for FileScanConfig { return Ok(None); } + // With morsel-driven execution, the shared WorkQueue handles load + // balancing at runtime so byte-range splitting is unnecessary. + // Just distribute whole files round-robin across target partitions. + if self.morsel_driven { + let all_files: Vec<_> = self + .file_groups + .iter() + .flat_map(|g| g.files().iter().cloned()) + .collect(); + if all_files.is_empty() { + return Ok(None); + } + let mut groups: Vec> = + (0..target_partitions).map(|_| vec![]).collect(); + for (i, file) in all_files.into_iter().enumerate() { + groups[i % target_partitions].push(file); + } + let file_groups = groups.into_iter().map(FileGroup::new).collect(); + let mut source = self.clone(); + source.file_groups = file_groups; + return Ok(Some(Arc::new(source))); + } + let source = self.file_source.repartitioned( target_partitions, repartition_file_min_size, diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index 275b0c9dd490f..0881e9437f035 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -103,8 +103,7 @@ Plan with Metrics 02)--SortExec: TopK(fetch=3), expr=[v@1 DESC], preserve_partitioning=[true], filter=[v@1 IS NULL OR v@1 > 800], metrics=[output_rows=3, ] 03)----ProjectionExec: expr=[id@0 as id, value@1 as v, value@1 + id@0 as name], metrics=[output_rows=10, ] 04)------FilterExec: value@1 > 3, metrics=[output_rows=10, , selectivity=100% (10/10)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1, metrics=[output_rows=10, ] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet]]}, projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, , files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=, scan_efficiency_ratio=18% (210/1.16 K)] +05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], [], [], []]}, projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, , files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=, scan_efficiency_ratio=18% (210/1.16 K)] statement ok set datafusion.explain.analyze_level = dev; @@ -476,8 +475,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_parquet.score)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_parquet.score)] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet]]}, projection=[score], file_type=parquet, predicate=category@0 = alpha AND DynamicFilter [ empty ], pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], [], [], []]}, projection=[score], file_type=parquet, predicate=category@0 = alpha AND DynamicFilter [ empty ], pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] # Test 4b: COUNT + MAX — DynamicFilter should NOT appear here in mixed aggregates @@ -495,8 +493,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1)), max(agg_parquet.score)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1)), max(agg_parquet.score)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet]]}, projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], [], [], []]}, projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] # Disable aggregate dynamic filters only statement ok @@ -515,8 +512,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_parquet.score)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_parquet.score)] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet]]}, projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], [], [], []]}, projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] statement ok SET datafusion.optimizer.enable_aggregate_dynamic_filter_pushdown = true; diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 9215ce87e3bef..0ec765ddd2f8c 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -560,23 +560,14 @@ physical_plan 05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ DataSourceExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ +10)│ files: 1 │ +11)│ format: parquet │ 12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ DataSourceExec │ -18)│ -------------------- │ -19)│ files: 1 │ -20)│ format: parquet │ -21)│ │ -22)│ predicate: │ -23)│ string_col != foo │ -24)└───────────────────────────┘ +13)│ predicate: │ +14)│ string_col != foo │ +15)└───────────────────────────┘ # Query with filter on memory query TT diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index 037eb3de8a93b..a3e97ae7fa34c 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -63,7 +63,9 @@ set datafusion.explain.analyze_level = summary; query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 limit 3; ---- -Plan with Metrics DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (171/2.35 K)] +Plan with Metrics +01)CoalescePartitionsExec: fetch=3, metrics=[output_rows=3, ] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], [], [], []]}, projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, , files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (171/2.35 K)] # limit_pruned_row_groups=0 total → 0 matched # because of order by, scan needs to preserve sort, so limit pruning is disabled @@ -71,8 +73,9 @@ query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 order by species limit 3; ---- Plan with Metrics -01)SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=, output_bytes=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (521/2.35 K)] +01)SortPreservingMergeExec: [species@0 ASC NULLS LAST], fetch=3, metrics=[output_rows=3, ] +02)--SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[true], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, ] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], [], [], []]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, , files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (521/2.35 K)] statement ok drop table tracking_data; diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index be713b963b451..61d19880db526 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -459,8 +459,7 @@ logical_plan 02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] physical_plan 01)FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], []]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% statement ok @@ -506,8 +505,7 @@ logical_plan 02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8View("%a%"), binary_as_string_option.largebinary_col LIKE Utf8View("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], []]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -556,8 +554,7 @@ logical_plan 02)--TableScan: binary_as_string_both projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_both.binary_col LIKE Utf8View("%a%"), binary_as_string_both.largebinary_col LIKE Utf8View("%a%"), binary_as_string_both.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], []]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -670,8 +667,7 @@ logical_plan 02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] physical_plan 01)FilterExec: column1@0 LIKE f% -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet], []]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] statement ok drop table foo diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index e2473ee328e51..ab7ae51dec5c7 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -96,8 +96,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: b@1 > 2, projection=[a@0] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -110,7 +109,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t where b = 2 ORDER BY b; @@ -133,8 +132,7 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--FilterExec: b@1 = 2, projection=[a@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -146,7 +144,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # If we set the setting to `true` it override's the table's setting statement ok @@ -181,7 +179,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -194,7 +192,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t where b = 2 ORDER BY b; @@ -216,7 +214,7 @@ logical_plan 04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -228,7 +226,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # If we reset the default the table created without pushdown goes back to disabling it statement ok @@ -264,8 +262,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: b@1 > 2, projection=[a@0] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -278,7 +275,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t where b = 2 ORDER BY b; @@ -301,8 +298,7 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--FilterExec: b@1 = 2, projection=[a@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -314,7 +310,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # When filter pushdown *is* enabled, ParquetExec can filter exactly, # not just metadata, so we expect to see no FilterExec @@ -339,8 +335,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: b@1 > 2, projection=[a@0] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t_pushdown where b = 2 ORDER BY b; @@ -357,7 +352,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # also test querying on columns that are not in all the files query T @@ -377,7 +372,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] query I @@ -396,7 +391,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] 02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] # should not push down volatile predicates such as RANDOM @@ -466,7 +461,7 @@ EXPLAIN select * from t_pushdown where part != val logical_plan 01)Filter: t_pushdown.val != t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != t_pushdown.part] -physical_plan DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != part@1 +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], []]}, projection=[val, part], file_type=parquet, predicate=val@0 != part@1 # If we reference only a partition column it gets evaluated during the listing phase query TT @@ -482,7 +477,7 @@ EXPLAIN select * from t_pushdown where val != 'c'; logical_plan 01)Filter: t_pushdown.val != Utf8View("c") 02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != Utf8View("c")] -physical_plan DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], []]}, projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] # If we have a mix of filters: # - The partition filters get evaluated during planning @@ -494,7 +489,7 @@ EXPLAIN select * from t_pushdown where val != 'd' AND val != 'c' AND part = 'a' logical_plan 01)Filter: t_pushdown.val != Utf8View("d") AND t_pushdown.val != Utf8View("c") AND t_pushdown.val != t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val != Utf8View("d"), t_pushdown.val != Utf8View("c"), t_pushdown.val != t_pushdown.part] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c AND val@0 != part@1, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [], [], []]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c AND val@0 != part@1, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] # The order of filters should not matter query TT @@ -503,7 +498,7 @@ EXPLAIN select val, part from t_pushdown where part = 'a' AND part = val; logical_plan 01)Filter: t_pushdown.val = t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [], [], []]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 query TT select val, part from t_pushdown where part = 'a' AND part = val; @@ -516,7 +511,7 @@ EXPLAIN select val, part from t_pushdown where part = val AND part = 'a'; logical_plan 01)Filter: t_pushdown.val = t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [], [], []]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 query TT select val, part from t_pushdown where part = val AND part = 'a'; @@ -602,8 +597,9 @@ logical_plan 02)--Filter: array_has(array_test.tags, Utf8("rust")) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has(array_test.tags, Utf8("rust"))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has(tags@1, rust) +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=array_has(tags@1, rust) # Test array_has_all predicate pushdown query I? @@ -619,8 +615,9 @@ logical_plan 02)--Filter: array_has_all(array_test.tags, List([rust, performance])) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has_all(array_test.tags, List([rust, performance]))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust, performance]) +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust, performance]) # Test array_has_any predicate pushdown query I? @@ -636,8 +633,9 @@ logical_plan 02)--Filter: array_has_any(array_test.tags, List([python, go])) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has_any(array_test.tags, List([python, go]))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has_any(tags@1, [python, go]) +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=array_has_any(tags@1, [python, go]) # Test complex predicate with OR query I? @@ -655,8 +653,9 @@ logical_plan 02)--Filter: array_has_all(array_test.tags, List([rust])) OR array_has_any(array_test.tags, List([python, go])) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has_all(array_test.tags, List([rust])) OR array_has_any(array_test.tags, List([python, go]))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust]) OR array_has_any(tags@1, [python, go]) +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust]) OR array_has_any(tags@1, [python, go]) # Test array function with other predicates query I? @@ -672,8 +671,9 @@ logical_plan 02)--Filter: array_test.id > Int64(1) AND array_has(array_test.tags, Utf8("rust")) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_test.id > Int64(1), array_has(array_test.tags, Utf8("rust"))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=id@0 > 1 AND array_has(tags@1, rust), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=id@0 > 1 AND array_has(tags@1, rust), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] ### # Test filter pushdown through UNION with mixed support @@ -729,7 +729,7 @@ physical_plan 01)UnionExec 02)--FilterExec: b@0 > 2 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet]]}, projection=[b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [], [], []]}, projection=[b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] # Clean up union test tables statement ok diff --git a/datafusion/sqllogictest/test_files/parquet_statistics.slt b/datafusion/sqllogictest/test_files/parquet_statistics.slt index 8c77fb96ba75c..bc2555aa688d0 100644 --- a/datafusion/sqllogictest/test_files/parquet_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_statistics.slt @@ -60,8 +60,7 @@ EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan 01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(10), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0) ScanBytes=Inexact(40))]] -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] # cleanup statement ok @@ -85,8 +84,7 @@ EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan 01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(10), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0) ScanBytes=Inexact(40))]] -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] # cleanup statement ok @@ -111,8 +109,7 @@ EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan 01)FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] # cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 297094fab16e7..edc59fec8dbb9 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -368,9 +368,8 @@ physical_plan 08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[env@1, service@2, value@3, f_dkey@4] 09)----------------CoalescePartitionsExec 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 ] +11)--------------------DataSourceExec: file_groups={3 groups: [[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)] +12)----------------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 @@ -420,9 +419,8 @@ physical_plan 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[env@1, service@2, value@3, f_dkey@4] 06)----------CoalescePartitionsExec 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 ] +08)--------------DataSourceExec: file_groups={3 groups: [[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)] +09)----------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/projection.slt b/datafusion/sqllogictest/test_files/projection.slt index e18114bc51ca8..a2f434118cacd 100644 --- a/datafusion/sqllogictest/test_files/projection.slt +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -276,5 +276,4 @@ logical_plan 03)----TableScan: t1 projection=[a], partial_filters=[t1.a > Int64(1)] physical_plan 01)FilterExec: a@0 > 1, projection=[] -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] diff --git a/datafusion/sqllogictest/test_files/projection_pushdown.slt b/datafusion/sqllogictest/test_files/projection_pushdown.slt index dbb77b33c21b7..fa4885bc5f65a 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -762,8 +762,7 @@ physical_plan 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as multi_struct.s[value]] 04)------FilterExec: id@1 > 2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 -06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query II @@ -1639,8 +1638,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as simple_struct.s[value]] 02)--FilterExec: id@1 > 2, projection=[__datafusion_extracted_1@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(32), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: file_groups={32 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], [], [], [], [], ...]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] ##################### # Section 14: SubqueryAlias tests diff --git a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt index e1c83c8c330d8..a3a4a42571d06 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt @@ -77,7 +77,7 @@ explain select * from test_filter_with_limit where value = 2 limit 1; ---- physical_plan 01)CoalescePartitionsExec: fetch=1 -02)--DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-2.parquet]]}, projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-2.parquet], []]}, projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] query II select * from test_filter_with_limit where value = 2 limit 1; @@ -114,43 +114,43 @@ LOCATION 'test_files/scratch/push_down_filter_parquet/t.parquet'; query TT explain select a from t where a = '100'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should not have a column cast when the value is a valid i32 query TT explain select a from t where a != '100'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99999999999'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99.99'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = ''; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = # The predicate should not have a column cast when the operator is = or != and the literal can be round-trip casted without losing information. query TT explain select a from t where cast(a as string) = '100'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should still have the column cast when the literal alters its string representation after round-trip casting (leading zero lost). query TT explain select a from t where CAST(a AS string) = '0123'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8View) = 0123 +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8View) = 0123 # Test dynamic filter pushdown with swapped join inputs (issue #17196) @@ -175,8 +175,7 @@ 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_parquet/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_parquet/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)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/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/push_down_filter_regression.slt b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt index ca4a30fa96c35..3f9828a5d6eca 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt @@ -151,7 +151,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_test.id)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_test.id)] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] query I select max(id) from agg_dyn_test where id > 1; @@ -166,7 +166,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_test.id)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_test.id)] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=CAST(id@0 AS Int64) + 1 > 1 AND DynamicFilter [ empty ] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=CAST(id@0 AS Int64) + 1 > 1 AND DynamicFilter [ empty ] # Expect dynamic filter available inside data source query TT @@ -176,7 +176,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_test.id), min(agg_dyn_test.id)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_test.id), min(agg_dyn_test.id)] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] # Dynamic filter should not be available for grouping sets query TT @@ -188,7 +188,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, __grouping_id@1 as __grouping_id], aggr=[max(agg_dyn_test.id)] 03)----RepartitionExec: partitioning=Hash([id@0, __grouping_id@1], 2), input_partitions=2 04)------AggregateExec: mode=Partial, gby=[(NULL as id), (id@0 as id)], aggr=[max(agg_dyn_test.id)] -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] +05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] statement ok drop table agg_dyn_test; diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index c9c2f91257081..c47fc20d9ce80 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -55,7 +55,7 @@ select * from parquet_table; 4 5 -## Expect to see the scan read the file as "4" groups with even sizes (offsets) +## Expect to see the scan with whole files distributed across groups (morsel-driven handles load balancing) query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; ---- @@ -64,13 +64,13 @@ logical_plan 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], [], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok set datafusion.optimizer.enable_round_robin_repartition = false; -## Expect to see the scan read the file as "4" groups with even sizes (offsets) again +## Expect to see the same plan (morsel-driven doesn't depend on round robin) query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; ---- @@ -79,7 +79,7 @@ logical_plan 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], [], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -90,8 +90,7 @@ statement ok COPY (VALUES (100), (200)) TO 'test_files/scratch/repartition_scan/parquet_table/1.parquet' STORED AS PARQUET; -## Still expect to see the scan read the file as "4" groups with even sizes. One group should read -## parts of both files. +## Expect to see whole files distributed round-robin across groups query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42 ORDER BY column1; ---- @@ -103,7 +102,7 @@ physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: column1@0 != 42 -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..266], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:266..526, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:272..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered From c769c2d9cd2ea674864e5622107f87d0fadb9c6f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 21:25:49 +0100 Subject: [PATCH 58/78] Improve I/O locality by pushing morsels to front of work queue After morselizing a file into row-group morsels, push them to the front of the shared queue instead of the back. This way the same (or nearby) worker picks up sibling row groups next, keeping I/O sequential within each file and the page cache warm. Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource/src/file_stream.rs | 25 ++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 0f64e98c1d47b..8364798137605 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -223,13 +223,12 @@ impl FileStream { self.state = FileStreamState::Idle; } else { // Keep the first morsel for this worker; push the rest - // back so other workers can pick them up immediately. - // This avoids a round-trip through Idle just to re-claim - // one of the morsels we just created. - // push_many is a no-op when given an empty iterator (len == 1). + // to the front of the queue so the same (or nearby) + // worker picks them up next, preserving I/O locality + // for row groups from the same file. let mut iter = morsels.into_iter(); let first = iter.next().unwrap(); - queue.push_many(iter.collect()); + queue.push_front_many(iter.collect()); // Don't stop time_opening here — it will be stopped // naturally when we transition Open → Scan. match self.file_opener.open(first) { @@ -479,14 +478,20 @@ impl WorkQueue { !queue.is_empty() || self.morselizing_count.load(Ordering::Acquire) == 0 } - /// Push many files back to the queue. + /// Push morsels to the front of the queue, preserving their order. /// - /// This is used when a file is expanded into multiple morsels. - pub fn push_many(&self, files: Vec) { + /// Morsels from the same file are kept together at the front so that + /// the next worker to pull gets row groups from the same file, + /// improving I/O locality (page-cache stays warm for that file). + pub fn push_front_many(&self, files: Vec) { if files.is_empty() { return; } - self.queue.lock().unwrap().extend(files); + let n = files.len(); + let mut queue = self.queue.lock().unwrap(); + queue.extend(files); + queue.rotate_right(n); + drop(queue); self.notify.notify_waiters(); } @@ -494,7 +499,7 @@ impl WorkQueue { /// count reaches zero, since that is the point at which they may need to /// re-evaluate whether all work is done. When count is still > 0, any new /// morsels pushed to the queue already triggered a notification via - /// `push_many`, so no additional wakeup is needed here. + /// `push_front_many`, so no additional wakeup is needed here. pub fn stop_morselizing(&self) { let prev = self.morselizing_count.fetch_sub(1, Ordering::AcqRel); if prev == 1 { From 7238c1cabff776928f256a817d844da4c095ee59 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 21:52:00 +0100 Subject: [PATCH 59/78] Simplify morsel handling: push all morsels to front of queue After morselizing, push all row-group morsels to the front of the queue and return to Idle, instead of keeping the first morsel and opening it inline. The worker then pulls the first morsel through the normal is_leaf_morsel fast path, keeping the code simpler while preserving I/O locality. Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource/src/file_stream.rs | 26 +++++++----------------- 1 file changed, 7 insertions(+), 19 deletions(-) diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 8364798137605..0ef23c3700d66 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -222,25 +222,13 @@ impl FileStream { // No morsels returned, skip this file self.state = FileStreamState::Idle; } else { - // Keep the first morsel for this worker; push the rest - // to the front of the queue so the same (or nearby) - // worker picks them up next, preserving I/O locality - // for row groups from the same file. - let mut iter = morsels.into_iter(); - let first = iter.next().unwrap(); - queue.push_front_many(iter.collect()); - // Don't stop time_opening here — it will be stopped - // naturally when we transition Open → Scan. - match self.file_opener.open(first) { - Ok(future) => { - self.state = FileStreamState::Open { future } - } - Err(e) => { - self.file_stream_metrics.time_opening.stop(); - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } + // Push all morsels to the front of the queue so + // this worker (or a nearby one) picks them up next, + // preserving I/O locality for row groups from the + // same file. Go back to Idle to pull the first one + // through the normal path. + queue.push_front_many(morsels); + self.state = FileStreamState::Idle; } } Err(e) => { From f7614b7e69572f9704496354fe3a521edb7b2ca0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 21:54:53 +0100 Subject: [PATCH 60/78] Use separate queues for files and morsels to improve I/O locality Split the single WorkQueue into two internal queues: one for whole files awaiting morselization and one for already-morselized leaf morsels (row groups). Workers drain the morsel queue first, so freshly produced row groups are consumed before the next file is opened. This keeps I/O sequential within each file without needing push-to-front tricks. Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource/src/file_stream.rs | 86 ++++++++++++++---------- 1 file changed, 49 insertions(+), 37 deletions(-) diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 0ef23c3700d66..18999292cfa2b 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -156,11 +156,7 @@ impl FileStream { match queue.pull() { WorkStatus::Work(part_file) => { if self.file_opener.is_leaf_morsel(&part_file) { - // Fast path: already a leaf morsel — skip the - // Morselizing state entirely. Undo the count - // increment that pull() did since we won't be - // morselizing. - queue.stop_morselizing(); + // Leaf morsel from the morsel queue — open directly. match self.file_opener.open(part_file) { Ok(future) => { self.state = FileStreamState::Open { future } @@ -172,6 +168,7 @@ impl FileStream { } } } else { + // Whole file from the file queue — morselize it. self.morsel_guard = Some(MorselizingGuard { queue: Arc::clone(queue), }); @@ -222,12 +219,11 @@ impl FileStream { // No morsels returned, skip this file self.state = FileStreamState::Idle; } else { - // Push all morsels to the front of the queue so - // this worker (or a nearby one) picks them up next, - // preserving I/O locality for row groups from the - // same file. Go back to Idle to pull the first one - // through the normal path. - queue.push_front_many(morsels); + // Push morsels to the morsel queue. Workers + // drain that queue before pulling new files, + // so these row groups get processed next, + // preserving I/O locality within the file. + queue.push_morsels(morsels); self.state = FileStreamState::Idle; } } @@ -421,9 +417,18 @@ pub enum WorkStatus { } /// A shared queue of [`PartitionedFile`] morsels for morsel-driven execution. +/// +/// Internally keeps two queues: one for whole files that still need +/// morselizing and one for already-morselized leaf morsels (e.g. row +/// groups). Workers drain the morsel queue first, which keeps I/O +/// sequential within a file because freshly produced morsels are +/// consumed before the next file is opened. #[derive(Debug)] pub struct WorkQueue { - queue: Mutex>, + /// Whole files waiting to be morselized. + files: Mutex>, + /// Already-morselized leaf morsels ready to be opened directly. + morsels: Mutex>, /// Number of workers currently morselizing a file. morselizing_count: AtomicUsize, /// Notify waiters when work is added or morselizing finishes. @@ -431,55 +436,62 @@ pub struct WorkQueue { } impl WorkQueue { - /// Create a new `WorkQueue` with the given initial files + /// Create a new `WorkQueue` with the given initial files. pub fn new(initial_files: Vec) -> Self { Self { - queue: Mutex::new(VecDeque::from(initial_files)), + files: Mutex::new(VecDeque::from(initial_files)), + morsels: Mutex::new(VecDeque::new()), morselizing_count: AtomicUsize::new(0), notify: Notify::new(), } } - /// Pull a file from the queue. + /// Pull a work item from the queue. + /// + /// Prefers already-morselized morsels (for I/O locality) over whole + /// files that still need morselizing. pub fn pull(&self) -> WorkStatus { - let mut queue = self.queue.lock().unwrap(); - if let Some(file) = queue.pop_front() { + // First try the morsel queue — these are ready to open immediately + // and preserve locality with the file that was just morselized. + if let Some(morsel) = self.morsels.lock().unwrap().pop_front() { + return WorkStatus::Work(morsel); + } + // Fall back to whole files that need morselizing. + let mut files = self.files.lock().unwrap(); + if let Some(file) = files.pop_front() { // Relaxed: the increment is done by the same task that will later call // stop_morselizing(), so program order ensures the decrement sees it. self.morselizing_count.fetch_add(1, Ordering::Relaxed); WorkStatus::Work(file) } else if self.morselizing_count.load(Ordering::Acquire) > 0 { // Acquire: stop_morselizing() uses AcqRel (a Release write) without - // holding the queue mutex, so we need Acquire here to synchronize with + // holding the files mutex, so we need Acquire here to synchronize with // it on weakly-ordered architectures (e.g. ARM). WorkStatus::Wait } else { - WorkStatus::Done + // Check the morsel queue one more time — a morselizer may have + // pushed work between our first check and reaching this point. + if self.morsels.lock().unwrap().is_empty() { + WorkStatus::Done + } else { + WorkStatus::Wait + } } } - /// Returns true if there is work in the queue or if all morselizing is done. + /// Returns true if there is work in either queue or if all morselizing is done. pub fn has_work_or_done(&self) -> bool { - let queue = self.queue.lock().unwrap(); - // Acquire: stop_morselizing() writes morselizing_count with AcqRel outside - // the queue mutex, so Acquire is needed to synchronize with that Release. - !queue.is_empty() || self.morselizing_count.load(Ordering::Acquire) == 0 + !self.morsels.lock().unwrap().is_empty() + || !self.files.lock().unwrap().is_empty() + || self.morselizing_count.load(Ordering::Acquire) == 0 } - /// Push morsels to the front of the queue, preserving their order. - /// - /// Morsels from the same file are kept together at the front so that - /// the next worker to pull gets row groups from the same file, - /// improving I/O locality (page-cache stays warm for that file). - pub fn push_front_many(&self, files: Vec) { - if files.is_empty() { + /// Push morselized leaf morsels to the morsel queue. + pub fn push_morsels(&self, morsels: Vec) { + if morsels.is_empty() { return; } - let n = files.len(); - let mut queue = self.queue.lock().unwrap(); - queue.extend(files); - queue.rotate_right(n); - drop(queue); + self.morsels.lock().unwrap().extend(morsels); self.notify.notify_waiters(); } @@ -487,7 +499,7 @@ impl WorkQueue { /// count reaches zero, since that is the point at which they may need to /// re-evaluate whether all work is done. When count is still > 0, any new /// morsels pushed to the queue already triggered a notification via - /// `push_front_many`, so no additional wakeup is needed here. + /// `push_morsels`, so no additional wakeup is needed here. pub fn stop_morselizing(&self) { let prev = self.morselizing_count.fetch_sub(1, Ordering::AcqRel); if prev == 1 { From 28e64c9f47eedd03f4b849abdc256d321e0181bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 22:01:23 +0100 Subject: [PATCH 61/78] Fix --- datafusion/core/src/datasource/physical_plan/parquet.rs | 4 ++-- datafusion/datasource-parquet/src/opener.rs | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index bee11c06ca073..9418bf9969867 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -79,7 +79,7 @@ mod tests { use insta::assert_snapshot; use object_store::local::LocalFileSystem; use object_store::path::Path; - use object_store::{ObjectMeta, ObjectStore}; + use object_store::{ObjectMeta, ObjectStore, ObjectStoreExt}; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; use tempfile::TempDir; @@ -2477,7 +2477,7 @@ mod tests { let mut out = Vec::new(); let props = WriterProperties::builder() - .set_max_row_group_size(10) + .set_max_row_group_row_count(Some(10)) .build(); { let mut writer = diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 5f57d84b87554..e2d45ba5eefd3 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -273,7 +273,8 @@ impl FileOpener for ParquetOpener { Err(e) => return Box::pin(ready(Err(e))), }; - let options = ArrowReaderOptions::new().with_page_index(false); + let options = + ArrowReaderOptions::new().with_page_index_policy(PageIndexPolicy::Skip); #[cfg(feature = "parquet_encryption")] let encryption_context = self.get_encryption_context(); @@ -2369,7 +2370,7 @@ mod test { let batch3 = record_batch!(("a", Int32, vec![Some(20), Some(21)])).unwrap(); let props = WriterProperties::builder() - .set_max_row_group_size(2) + .set_max_row_group_row_count(Some(2)) .build(); let data_len = write_parquet_batches( From 8eda1408bac5643e03559c8b159e2a616ee0b418 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sun, 1 Mar 2026 22:12:02 +0100 Subject: [PATCH 62/78] Fix time_opening metric not stopped after morselizing Stop the time_opening timer before transitioning back to Idle after pushing morsels to the queue. Without this, re-entering Idle would call start() on an already-running timer, triggering an assertion failure. Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource/src/file_stream.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 18999292cfa2b..57c126a2663ec 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -224,6 +224,7 @@ impl FileStream { // so these row groups get processed next, // preserving I/O locality within the file. queue.push_morsels(morsels); + self.file_stream_metrics.time_opening.stop(); self.state = FileStreamState::Idle; } } From afd2936bce2e622550c2314cc82826279825ed04 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 2 Mar 2026 06:54:00 +0100 Subject: [PATCH 63/78] Sort files and morsels by estimated row count for dynamic filters Sort at three levels so smaller/more-selective work is processed first, letting dynamic filters tighten sooner and prune more data: - Planning: sort files by statistics.num_rows before distribution - WorkQueue::new: re-sort after round-robin interleaving - push_morsels: keep the morsel queue globally sorted by estimated row count (set on each morsel in morselize()) Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource-parquet/src/opener.rs | 18 +++++++++++-- datafusion/datasource/src/file_scan_config.rs | 13 ++++++++-- datafusion/datasource/src/file_stream.rs | 26 +++++++++++++++++-- 3 files changed, 51 insertions(+), 6 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index e2d45ba5eefd3..9dfa2b5af1a83 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -20,7 +20,7 @@ use crate::page_filter::PagePruningAccessPlanFilter; use crate::row_group_filter::RowGroupAccessPlanFilter; use crate::{ - ParquetAccessPlan, ParquetFileMetrics, ParquetFileReaderFactory, + ParquetAccessPlan, ParquetFileMetrics, ParquetFileReaderFactory, RowGroupAccess, apply_file_schema_type_coercions, coerce_int96_to_resolution, row_filter, }; use arrow::array::{RecordBatch, RecordBatchOptions}; @@ -468,8 +468,15 @@ impl FileOpener for ParquetOpener { if !access_plan.should_scan(i) { continue; } + let est_rows = match &access_plan.inner()[i] { + RowGroupAccess::Selection(sel) => sel + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(), + _ => metadata.row_group(i).num_rows() as usize, + }; let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); - // Transfer the page-pruned access (Scan or Selection) for this row group morsel_access_plan.set(i, access_plan.inner()[i].clone()); let morsel = ParquetMorsel { metadata: Arc::clone(&metadata), @@ -477,6 +484,13 @@ impl FileOpener for ParquetOpener { }; let mut f = partitioned_file.clone(); f.extensions = Some(Arc::new(morsel)); + // Store estimated row count so the WorkQueue can sort + // morsels globally (smallest first) for dynamic filters. + f.statistics = Some(Arc::new(Statistics { + num_rows: Precision::Exact(est_rows), + total_byte_size: Precision::Absent, + column_statistics: vec![], + })); morsels.push(f); } Ok(morsels) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 92e61aa16cf4b..ca9a927b053db 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -720,9 +720,12 @@ impl DataSource for FileScanConfig { // With morsel-driven execution, the shared WorkQueue handles load // balancing at runtime so byte-range splitting is unnecessary. - // Just distribute whole files round-robin across target partitions. + // Sort files by estimated row count (smallest first) so that + // smaller / more-selective files are processed earlier, letting + // dynamic filters tighten sooner. Then distribute round-robin + // across target partitions. if self.morsel_driven { - let all_files: Vec<_> = self + let mut all_files: Vec<_> = self .file_groups .iter() .flat_map(|g| g.files().iter().cloned()) @@ -730,6 +733,12 @@ impl DataSource for FileScanConfig { if all_files.is_empty() { return Ok(None); } + all_files.sort_by_key(|f| { + f.statistics + .as_ref() + .and_then(|s| s.num_rows.get_value().copied()) + .unwrap_or(usize::MAX) + }); let mut groups: Vec> = (0..target_partitions).map(|_| vec![]).collect(); for (i, file) in all_files.into_iter().enumerate() { diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 57c126a2663ec..916421e610c2e 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -438,7 +438,17 @@ pub struct WorkQueue { impl WorkQueue { /// Create a new `WorkQueue` with the given initial files. - pub fn new(initial_files: Vec) -> Self { + /// + /// Files are sorted by estimated row count (smallest first) so that + /// smaller / more-selective files are morselized earlier, helping + /// dynamic filters tighten sooner. + pub fn new(mut initial_files: Vec) -> Self { + initial_files.sort_by_key(|f| { + f.statistics + .as_ref() + .and_then(|s| s.num_rows.get_value().copied()) + .unwrap_or(usize::MAX) + }); Self { files: Mutex::new(VecDeque::from(initial_files)), morsels: Mutex::new(VecDeque::new()), @@ -488,11 +498,23 @@ impl WorkQueue { } /// Push morselized leaf morsels to the morsel queue. + /// + /// The queue is kept sorted by estimated row count (smallest first) + /// so that more-selective morsels are processed earlier, helping + /// dynamic filters tighten sooner. pub fn push_morsels(&self, morsels: Vec) { if morsels.is_empty() { return; } - self.morsels.lock().unwrap().extend(morsels); + let mut queue = self.morsels.lock().unwrap(); + queue.extend(morsels); + queue.make_contiguous().sort_by_key(|f| { + f.statistics + .as_ref() + .and_then(|s| s.num_rows.get_value().copied()) + .unwrap_or(usize::MAX) + }); + drop(queue); self.notify.notify_waiters(); } From e31bf9793415b2d831c0136a26837a507706dcd3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 2 Mar 2026 07:44:58 +0100 Subject: [PATCH 64/78] Revert "Sort files and morsels by estimated row count for dynamic filters" This reverts commit afd2936bce2e622550c2314cc82826279825ed04. --- datafusion/datasource-parquet/src/opener.rs | 18 ++----------- datafusion/datasource/src/file_scan_config.rs | 13 ++-------- datafusion/datasource/src/file_stream.rs | 26 ++----------------- 3 files changed, 6 insertions(+), 51 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 9dfa2b5af1a83..e2d45ba5eefd3 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -20,7 +20,7 @@ use crate::page_filter::PagePruningAccessPlanFilter; use crate::row_group_filter::RowGroupAccessPlanFilter; use crate::{ - ParquetAccessPlan, ParquetFileMetrics, ParquetFileReaderFactory, RowGroupAccess, + ParquetAccessPlan, ParquetFileMetrics, ParquetFileReaderFactory, apply_file_schema_type_coercions, coerce_int96_to_resolution, row_filter, }; use arrow::array::{RecordBatch, RecordBatchOptions}; @@ -468,15 +468,8 @@ impl FileOpener for ParquetOpener { if !access_plan.should_scan(i) { continue; } - let est_rows = match &access_plan.inner()[i] { - RowGroupAccess::Selection(sel) => sel - .iter() - .filter(|s| !s.skip) - .map(|s| s.row_count) - .sum(), - _ => metadata.row_group(i).num_rows() as usize, - }; let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); + // Transfer the page-pruned access (Scan or Selection) for this row group morsel_access_plan.set(i, access_plan.inner()[i].clone()); let morsel = ParquetMorsel { metadata: Arc::clone(&metadata), @@ -484,13 +477,6 @@ impl FileOpener for ParquetOpener { }; let mut f = partitioned_file.clone(); f.extensions = Some(Arc::new(morsel)); - // Store estimated row count so the WorkQueue can sort - // morsels globally (smallest first) for dynamic filters. - f.statistics = Some(Arc::new(Statistics { - num_rows: Precision::Exact(est_rows), - total_byte_size: Precision::Absent, - column_statistics: vec![], - })); morsels.push(f); } Ok(morsels) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index ca9a927b053db..92e61aa16cf4b 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -720,12 +720,9 @@ impl DataSource for FileScanConfig { // With morsel-driven execution, the shared WorkQueue handles load // balancing at runtime so byte-range splitting is unnecessary. - // Sort files by estimated row count (smallest first) so that - // smaller / more-selective files are processed earlier, letting - // dynamic filters tighten sooner. Then distribute round-robin - // across target partitions. + // Just distribute whole files round-robin across target partitions. if self.morsel_driven { - let mut all_files: Vec<_> = self + let all_files: Vec<_> = self .file_groups .iter() .flat_map(|g| g.files().iter().cloned()) @@ -733,12 +730,6 @@ impl DataSource for FileScanConfig { if all_files.is_empty() { return Ok(None); } - all_files.sort_by_key(|f| { - f.statistics - .as_ref() - .and_then(|s| s.num_rows.get_value().copied()) - .unwrap_or(usize::MAX) - }); let mut groups: Vec> = (0..target_partitions).map(|_| vec![]).collect(); for (i, file) in all_files.into_iter().enumerate() { diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 916421e610c2e..57c126a2663ec 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -438,17 +438,7 @@ pub struct WorkQueue { impl WorkQueue { /// Create a new `WorkQueue` with the given initial files. - /// - /// Files are sorted by estimated row count (smallest first) so that - /// smaller / more-selective files are morselized earlier, helping - /// dynamic filters tighten sooner. - pub fn new(mut initial_files: Vec) -> Self { - initial_files.sort_by_key(|f| { - f.statistics - .as_ref() - .and_then(|s| s.num_rows.get_value().copied()) - .unwrap_or(usize::MAX) - }); + pub fn new(initial_files: Vec) -> Self { Self { files: Mutex::new(VecDeque::from(initial_files)), morsels: Mutex::new(VecDeque::new()), @@ -498,23 +488,11 @@ impl WorkQueue { } /// Push morselized leaf morsels to the morsel queue. - /// - /// The queue is kept sorted by estimated row count (smallest first) - /// so that more-selective morsels are processed earlier, helping - /// dynamic filters tighten sooner. pub fn push_morsels(&self, morsels: Vec) { if morsels.is_empty() { return; } - let mut queue = self.morsels.lock().unwrap(); - queue.extend(morsels); - queue.make_contiguous().sort_by_key(|f| { - f.statistics - .as_ref() - .and_then(|s| s.num_rows.get_value().copied()) - .unwrap_or(usize::MAX) - }); - drop(queue); + self.morsels.lock().unwrap().extend(morsels); self.notify.notify_waiters(); } From 4fd597b83682b1257a6c35f799ca214ba51c5aa2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 3 Mar 2026 22:13:43 +0100 Subject: [PATCH 65/78] WIP --- datafusion/core/src/datasource/physical_plan/parquet.rs | 2 ++ datafusion/execution/src/task.rs | 8 ++++---- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 9418bf9969867..55dc8805a437e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -2545,6 +2545,8 @@ mod tests { assert!(count1 > 0, "Partition 1 should have produced rows"); // Test re-executability: executing the same plan again should work + // (We have to create a new TaskContext, as it holds the state) + let task_ctx = ctx.task_ctx(); let stream0 = exec.execute(0, Arc::clone(&task_ctx))?; let stream1 = exec.execute(1, Arc::clone(&task_ctx))?; diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 0f797ffcba303..77d0f19a1d4c3 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -166,10 +166,10 @@ impl TaskContext { create: impl FnOnce() -> T, ) -> Arc { let mut map = self.shared_state.0.lock().unwrap(); - if let Some(existing) = map.get(&key) { - if let Ok(typed) = Arc::clone(existing).downcast::() { - return typed; - } + if let Some(existing) = map.get(&key) + && let Ok(typed) = Arc::clone(existing).downcast::() + { + return typed; } let value = Arc::new(create()); map.insert(key, Arc::clone(&value) as Arc); From a9ed9d0fcfb54070e95dad337e73d483bcefb9d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 3 Mar 2026 22:30:38 +0100 Subject: [PATCH 66/78] Use flat files=[...] display for morsel-driven DataSourceExec In morsel-driven mode all files share a single work queue regardless of which file_group they came from, so the file_groups={N groups: [[...]]} format is misleading. Switch to a flat files=[a, b, c] format that shows the actual files without implying partition structure. Co-Authored-By: Claude Sonnet 4.6 --- datafusion/datasource/src/file_scan_config.rs | 28 +++- .../test_files/aggregate_repartition.slt | 4 +- .../test_files/date_bin_errors.slt | 4 +- .../dynamic_filter_pushdown_config.slt | 72 ++++----- .../sqllogictest/test_files/explain.slt | 34 ++-- .../test_files/grouping_set_repartition.slt | 6 +- datafusion/sqllogictest/test_files/limit.slt | 2 +- .../sqllogictest/test_files/limit_pruning.slt | 10 +- .../test_files/listing_table_statistics.slt | 2 +- .../sqllogictest/test_files/parquet.slt | 10 +- .../test_files/parquet_filter_pushdown.slt | 58 +++---- .../test_files/parquet_statistics.slt | 6 +- .../test_files/preserve_file_partitioning.slt | 12 +- .../sqllogictest/test_files/projection.slt | 2 +- .../test_files/projection_pushdown.slt | 152 +++++++++--------- .../test_files/push_down_filter_parquet.slt | 20 +-- .../push_down_filter_regression.slt | 8 +- .../sqllogictest/test_files/repartition.slt | 4 +- .../test_files/repartition_scan.slt | 6 +- .../test_files/spark/map/str_to_map.slt | 2 +- datafusion/sqllogictest/test_files/struct.slt | 2 +- .../sqllogictest/test_files/type_coercion.slt | 2 +- datafusion/sqllogictest/test_files/window.slt | 6 +- 23 files changed, 237 insertions(+), 215 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 92e61aa16cf4b..9c67bf70efa50 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -649,8 +649,18 @@ impl DataSource for FileScanConfig { let schema = self.projected_schema().map_err(|_| std::fmt::Error {})?; let orderings = get_projected_output_ordering(self, &schema); - write!(f, "file_groups=")?; - FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + if self.morsel_driven { + let files: Vec<_> = self.file_groups.iter().flat_map(|g| g.iter()).collect(); + write!(f, "files=[")?; + for (i, pf) in files.iter().enumerate() { + if i > 0 { write!(f, ", ")?; } + write!(f, "{}", pf.object_meta.location.as_ref())?; + } + write!(f, "]")?; + } else { + write!(f, "file_groups=")?; + FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + } if !schema.fields().is_empty() { if let Some(projection) = self.file_source.projection() { @@ -1372,8 +1382,18 @@ impl DisplayAs for FileScanConfig { let schema = self.projected_schema().map_err(|_| std::fmt::Error {})?; let orderings = get_projected_output_ordering(self, &schema); - write!(f, "file_groups=")?; - FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + if self.morsel_driven { + let files: Vec<_> = self.file_groups.iter().flat_map(|g| g.iter()).collect(); + write!(f, "files=[")?; + for (i, pf) in files.iter().enumerate() { + if i > 0 { write!(f, ", ")?; } + write!(f, "{}", pf.object_meta.location.as_ref())?; + } + write!(f, "]")?; + } else { + write!(f, "file_groups=")?; + FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + } if !schema.fields().is_empty() { write!(f, ", projection={}", ProjectSchemaDisplay(&schema))?; diff --git a/datafusion/sqllogictest/test_files/aggregate_repartition.slt b/datafusion/sqllogictest/test_files/aggregate_repartition.slt index eeece7862341b..1eda5e95b4863 100644 --- a/datafusion/sqllogictest/test_files/aggregate_repartition.slt +++ b/datafusion/sqllogictest/test_files/aggregate_repartition.slt @@ -96,7 +96,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=1 04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet]]}, projection=[env], file_type=parquet +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet], projection=[env], file_type=parquet # Verify the queries actually work and return the same results query TI rowsort @@ -127,4 +127,4 @@ logical_plan physical_plan 01)ProjectionExec: expr=[env@0 as env, count(Int64(1))@1 as count(*)] 02)--AggregateExec: mode=Single, gby=[env@0 as env], aggr=[count(Int64(1))] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet]]}, projection=[env], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet], projection=[env], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/date_bin_errors.slt b/datafusion/sqllogictest/test_files/date_bin_errors.slt index b6cda471d7afa..d95c27b96a060 100644 --- a/datafusion/sqllogictest/test_files/date_bin_errors.slt +++ b/datafusion/sqllogictest/test_files/date_bin_errors.slt @@ -23,7 +23,7 @@ select date_bin(interval '1637426858 months', to_timestamp_millis(1040292460), t ---- NULL -# Negative timestamp with month interval - should return NULL instead of panicking +# Negative timestamp with month interval - should return NULL instead of panicking query P select date_bin(interval '1 month', to_timestamp_millis(-1040292460), timestamp '1984-01-07 00:00:00'); ---- @@ -57,4 +57,4 @@ select date_bin( timestamp '1984-01-07 00:00:00' ) as b; ---- -NULL \ No newline at end of file +NULL diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index 0881e9437f035..3d43c1242d562 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -90,7 +90,7 @@ logical_plan 02)--TableScan: test_parquet projection=[id, value, name] physical_plan 01)SortExec: TopK(fetch=3), expr=[value@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ] statement ok set datafusion.explain.analyze_level = summary; @@ -99,11 +99,11 @@ query TT EXPLAIN ANALYZE SELECT id, value AS v, value + id as name FROM test_parquet where value > 3 ORDER BY v DESC LIMIT 3; ---- Plan with Metrics -01)SortPreservingMergeExec: [v@1 DESC], fetch=3, metrics=[output_rows=3, ] -02)--SortExec: TopK(fetch=3), expr=[v@1 DESC], preserve_partitioning=[true], filter=[v@1 IS NULL OR v@1 > 800], metrics=[output_rows=3, ] -03)----ProjectionExec: expr=[id@0 as id, value@1 as v, value@1 + id@0 as name], metrics=[output_rows=10, ] -04)------FilterExec: value@1 > 3, metrics=[output_rows=10, , selectivity=100% (10/10)] -05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], [], [], []]}, projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, , files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=, scan_efficiency_ratio=18% (210/1.16 K)] +01)SortPreservingMergeExec: [v@1 DESC], fetch=3, metrics=[output_rows=3, elapsed_compute=10.62µs, output_bytes=36.0 B] +02)--SortExec: TopK(fetch=3), expr=[v@1 DESC], preserve_partitioning=[true], filter=[v@1 IS NULL OR v@1 > 800], metrics=[output_rows=3, elapsed_compute=86.67µs, output_bytes=36.0 B] +03)----ProjectionExec: expr=[id@0 as id, value@1 as v, value@1 + id@0 as name], metrics=[output_rows=10, elapsed_compute=9.92µs, output_bytes=64.0 KB] +04)------FilterExec: value@1 > 3, metrics=[output_rows=10, elapsed_compute=15.04µs, output_bytes=64.0 KB, selectivity=100% (10/10)] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, elapsed_compute=4ns, output_bytes=80.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=464.00µs, scan_efficiency_ratio=18% (210/1.15 K)] statement ok set datafusion.explain.analyze_level = dev; @@ -127,7 +127,7 @@ logical_plan 02)--TableScan: test_parquet projection=[id, value, name] physical_plan 01)SortExec: TopK(fetch=3), expr=[value@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet]]}, projection=[id, value, name], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value, name], file_type=parquet query IIT SELECT id, value AS v, name FROM (SELECT * FROM test_parquet UNION ALL SELECT * FROM test_parquet) ORDER BY v DESC LIMIT 3; @@ -156,8 +156,8 @@ logical_plan 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 ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[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 @@ -179,8 +179,8 @@ logical_plan 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 +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[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 @@ -204,8 +204,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Right, 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 +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # LEFT JOIN correctness: all left rows appear, unmatched right rows produce NULLs query ITT @@ -238,8 +238,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Left, 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 +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # RIGHT JOIN correctness: all right rows appear, unmatched left rows produce NULLs query ITT @@ -268,8 +268,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@2 as id, id@0 as rid, data@3 as data, info@1 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(id@0, id@0)] -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 +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # LEFT SEMI JOIN: optimizer swaps to RightSemi (build=right_parquet, probe=left_parquet). # No self-generated dynamic filter (only Inner joins), but parent filters on @@ -288,8 +288,8 @@ logical_plan 06)------TableScan: right_parquet projection=[id] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(id@0, id@0)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet -03)--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 +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # LEFT ANTI JOIN: no self-generated dynamic filter, but parent filters can push # to the preserved (left/build) side. @@ -307,8 +307,8 @@ logical_plan 06)------TableScan: right_parquet projection=[id] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] -02)--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 -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id], file_type=parquet # Test 2c: Parent dynamic filter (from TopK) pushed through semi/anti joins # Sort on the join key (id) so the TopK dynamic filter pushes to BOTH sides. @@ -332,8 +332,8 @@ logical_plan physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(id@0, id@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] -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 ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Correctness check query IT @@ -364,8 +364,8 @@ logical_plan physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] -03)----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)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # Correctness check query IT @@ -402,8 +402,8 @@ logical_plan 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 ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[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 @@ -428,8 +428,8 @@ logical_plan 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 +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[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 @@ -475,7 +475,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_parquet.score)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_parquet.score)] -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], [], [], []]}, projection=[score], file_type=parquet, predicate=category@0 = alpha AND DynamicFilter [ empty ], pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], projection=[score], file_type=parquet, predicate=category@0 = alpha AND DynamicFilter [ empty ], pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] # Test 4b: COUNT + MAX — DynamicFilter should NOT appear here in mixed aggregates @@ -493,7 +493,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1)), max(agg_parquet.score)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1)), max(agg_parquet.score)] -05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], [], [], []]}, projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] # Disable aggregate dynamic filters only statement ok @@ -512,7 +512,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_parquet.score)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_parquet.score)] -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], [], [], []]}, projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] statement ok SET datafusion.optimizer.enable_aggregate_dynamic_filter_pushdown = true; @@ -574,8 +574,8 @@ logical_plan 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 +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # Re-enable statement ok @@ -622,8 +622,8 @@ logical_plan 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 ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Test 6: Regression test for issue #20213 - dynamic filter applied to wrong table # when subquery join has same column names on both sides. diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index c5907d497500e..201138387d251 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -293,7 +293,7 @@ CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-t query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- -physical_plan 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 DataSourceExec: files=[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))]] # explain verbose with both collect & show statistics on query TT @@ -301,14 +301,14 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ---- initial_physical_plan 01)GlobalLimitExec: skip=0, fetch=10, 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))]] -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))]] +02)--DataSourceExec: files=[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))]] initial_physical_plan_with_schema 01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] -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, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] +02)--DataSourceExec: files=[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, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] physical_plan after OutputRequirements 01)OutputRequirementExec: order_by=[], dist_by=Unspecified, 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))]] 02)--GlobalLimitExec: skip=0, fetch=10, 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))]] -03)----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))]] +03)----DataSourceExec: files=[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 aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -320,17 +320,17 @@ physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10, 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))]] -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))]] +02)--DataSourceExec: files=[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 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 LimitPushdown DataSourceExec: files=[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 physical_plan after EnsureCooperative SAME TEXT AS ABOVE physical_plan after FilterPushdown(Post) SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan 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_with_schema 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, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] +physical_plan DataSourceExec: files=[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_with_schema DataSourceExec: files=[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, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] statement ok @@ -342,17 +342,17 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ---- initial_physical_plan 01)GlobalLimitExec: skip=0, fetch=10 -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 +02)--DataSourceExec: files=[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 initial_physical_plan_with_stats 01)GlobalLimitExec: skip=0, fetch=10, 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))]] -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))]] +02)--DataSourceExec: files=[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))]] initial_physical_plan_with_schema 01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] -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, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] +02)--DataSourceExec: files=[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, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] physical_plan after OutputRequirements 01)OutputRequirementExec: order_by=[], dist_by=Unspecified 02)--GlobalLimitExec: skip=0, fetch=10 -03)----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 +03)----DataSourceExec: files=[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 aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -364,18 +364,18 @@ physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10 -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 +02)--DataSourceExec: files=[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 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 LimitPushdown DataSourceExec: files=[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 physical_plan after EnsureCooperative SAME TEXT AS ABOVE physical_plan after FilterPushdown(Post) SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan 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_with_stats 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_with_schema 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, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] +physical_plan DataSourceExec: files=[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_with_stats DataSourceExec: files=[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_with_schema DataSourceExec: files=[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, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] statement ok diff --git a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt index 16ab90651c8b3..e01e74fade03d 100644 --- a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt +++ b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt @@ -151,19 +151,19 @@ physical_plan 10)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 12)----------------------FilterExec: channel@0 = store, projection=[brand@1, amount@2] -13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = store, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= store AND store <= channel_max@1, required_guarantees=[channel in (store)] +13)------------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet], projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = store, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= store AND store <= channel_max@1, required_guarantees=[channel in (store)] 14)--------------ProjectionExec: expr=[web as channel, brand@0 as brand, sum(sales.amount)@1 as total] 15)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 16)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 17)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 18)----------------------FilterExec: channel@0 = web, projection=[brand@1, amount@2] -19)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = web, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= web AND web <= channel_max@1, required_guarantees=[channel in (web)] +19)------------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet], projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = web, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= web AND web <= channel_max@1, required_guarantees=[channel in (web)] 20)--------------ProjectionExec: expr=[catalog as channel, brand@0 as brand, sum(sales.amount)@1 as total] 21)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 22)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 23)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 24)----------------------FilterExec: channel@0 = catalog, projection=[brand@1, amount@2] -25)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = catalog, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= catalog AND catalog <= channel_max@1, required_guarantees=[channel in (catalog)] +25)------------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet], projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = catalog, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= catalog AND catalog <= channel_max@1, required_guarantees=[channel in (catalog)] query TTI rowsort SELECT channel, brand, SUM(total) as grand_total diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index ff3c49485a286..dae9736afe013 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -849,7 +849,7 @@ physical_plan 01)ProjectionExec: expr=[1 as foo] 02)--SortPreservingMergeExec: [part_key@0 ASC NULLS LAST], fetch=1 03)----SortExec: TopK(fetch=1), expr=[part_key@0 ASC NULLS LAST], preserve_partitioning=[true] -04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-2.parquet]]}, projection=[part_key], file_type=parquet, predicate=DynamicFilter [ empty ] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-2.parquet], projection=[part_key], file_type=parquet, predicate=DynamicFilter [ empty ] query I with selection as ( diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index a3e97ae7fa34c..8ce7de373f169 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -64,8 +64,8 @@ query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 limit 3; ---- Plan with Metrics -01)CoalescePartitionsExec: fetch=3, metrics=[output_rows=3, ] -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], [], [], []]}, projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, , files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (171/2.35 K)] +01)CoalescePartitionsExec: fetch=3, metrics=[output_rows=3, elapsed_compute=33.54µs, output_bytes=142.0 B] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=4ns, output_bytes=142.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=171, metadata_load_time=1.71ms, scan_efficiency_ratio=7.3% (171/2.35 K)] # limit_pruned_row_groups=0 total → 0 matched # because of order by, scan needs to preserve sort, so limit pruning is disabled @@ -73,9 +73,9 @@ query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 order by species limit 3; ---- Plan with Metrics -01)SortPreservingMergeExec: [species@0 ASC NULLS LAST], fetch=3, metrics=[output_rows=3, ] -02)--SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[true], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, ] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], [], [], []]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, , files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (521/2.35 K)] +01)SortPreservingMergeExec: [species@0 ASC NULLS LAST], fetch=3, metrics=[output_rows=3, elapsed_compute=28.04µs, output_bytes=72.0 B] +02)--SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[true], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=349.50µs, output_bytes=72.0 B] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=4ns, output_bytes=142.0 B, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=521, metadata_load_time=1.81ms, scan_efficiency_ratio=22% (521/2.35 K)] statement ok drop table tracking_data; diff --git a/datafusion/sqllogictest/test_files/listing_table_statistics.slt b/datafusion/sqllogictest/test_files/listing_table_statistics.slt index 4298320d4aaba..8c301e2d2c770 100644 --- a/datafusion/sqllogictest/test_files/listing_table_statistics.slt +++ b/datafusion/sqllogictest/test_files/listing_table_statistics.slt @@ -35,7 +35,7 @@ query TT explain format indent select * from t; ---- logical_plan TableScan: t projection=[int_col, str_col] -physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/listing_table_statistics/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/listing_table_statistics/2.parquet]]}, projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Absent, [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0) ScanBytes=Exact(32)),(Col[1]: Min=Exact(Utf8View("a")) Max=Exact(Utf8View("d")) Null=Exact(0) ScanBytes=Inexact(100))]] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/listing_table_statistics/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/listing_table_statistics/2.parquet], projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Absent, [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0) ScanBytes=Exact(32)),(Col[1]: Min=Exact(Utf8View("a")) Max=Exact(Utf8View("d")) Null=Exact(0) ScanBytes=Inexact(100))]] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 61d19880db526..e39870a56001c 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -95,7 +95,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] 02)--SortExec: expr=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet], projection=[int_col, string_col], file_type=parquet # Tear down test_table: statement ok @@ -459,7 +459,7 @@ logical_plan 02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] physical_plan 01)FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], []]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% statement ok @@ -505,7 +505,7 @@ logical_plan 02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8View("%a%"), binary_as_string_option.largebinary_col LIKE Utf8View("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], []]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -554,7 +554,7 @@ logical_plan 02)--TableScan: binary_as_string_both projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_both.binary_col LIKE Utf8View("%a%"), binary_as_string_both.largebinary_col LIKE Utf8View("%a%"), binary_as_string_both.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], []]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -667,7 +667,7 @@ logical_plan 02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] physical_plan 01)FilterExec: column1@0 LIKE f% -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet], []]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet], projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] statement ok drop table foo diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index ab7ae51dec5c7..e482fdaa06075 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -96,7 +96,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: b@1 > 2, projection=[a@0] -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -109,7 +109,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t where b = 2 ORDER BY b; @@ -132,7 +132,7 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--FilterExec: b@1 = 2, projection=[a@0] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -144,7 +144,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # If we set the setting to `true` it override's the table's setting statement ok @@ -179,7 +179,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -192,7 +192,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t where b = 2 ORDER BY b; @@ -214,7 +214,7 @@ logical_plan 04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -226,7 +226,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # If we reset the default the table created without pushdown goes back to disabling it statement ok @@ -262,7 +262,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: b@1 > 2, projection=[a@0] -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -275,7 +275,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t where b = 2 ORDER BY b; @@ -298,7 +298,7 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--FilterExec: b@1 = 2, projection=[a@0] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -310,7 +310,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # When filter pushdown *is* enabled, ParquetExec can filter exactly, # not just metadata, so we expect to see no FilterExec @@ -335,7 +335,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: b@1 > 2, projection=[a@0] -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t_pushdown where b = 2 ORDER BY b; @@ -352,7 +352,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # also test querying on columns that are not in all the files query T @@ -372,7 +372,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] query I @@ -391,7 +391,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] 02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], [], []]}, projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] # should not push down volatile predicates such as RANDOM @@ -405,7 +405,7 @@ logical_plan 03)----TableScan: t_pushdown projection=[a, b] physical_plan 01)FilterExec: CAST(b@1 AS Float64) > random(), projection=[a@0] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet ## cleanup statement ok @@ -461,14 +461,14 @@ EXPLAIN select * from t_pushdown where part != val logical_plan 01)Filter: t_pushdown.val != t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != t_pushdown.part] -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], []]}, projection=[val, part], file_type=parquet, predicate=val@0 != part@1 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 != part@1 # If we reference only a partition column it gets evaluated during the listing phase query TT EXPLAIN select * from t_pushdown where part != 'a'; ---- logical_plan TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part != Utf8View("a")] -physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], projection=[val, part], file_type=parquet # And if we reference only a file column it gets pushed down query TT @@ -477,7 +477,7 @@ EXPLAIN select * from t_pushdown where val != 'c'; logical_plan 01)Filter: t_pushdown.val != Utf8View("c") 02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != Utf8View("c")] -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], []]}, projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] # If we have a mix of filters: # - The partition filters get evaluated during planning @@ -489,7 +489,7 @@ EXPLAIN select * from t_pushdown where val != 'd' AND val != 'c' AND part = 'a' logical_plan 01)Filter: t_pushdown.val != Utf8View("d") AND t_pushdown.val != Utf8View("c") AND t_pushdown.val != t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val != Utf8View("d"), t_pushdown.val != Utf8View("c"), t_pushdown.val != t_pushdown.part] -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [], [], []]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c AND val@0 != part@1, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c AND val@0 != part@1, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] # The order of filters should not matter query TT @@ -498,7 +498,7 @@ EXPLAIN select val, part from t_pushdown where part = 'a' AND part = val; logical_plan 01)Filter: t_pushdown.val = t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [], [], []]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 = part@1 query TT select val, part from t_pushdown where part = 'a' AND part = val; @@ -511,7 +511,7 @@ EXPLAIN select val, part from t_pushdown where part = val AND part = 'a'; logical_plan 01)Filter: t_pushdown.val = t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [], [], []]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 = part@1 query TT select val, part from t_pushdown where part = val AND part = 'a'; @@ -599,7 +599,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=array_has(tags@1, rust) +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=array_has(tags@1, rust) # Test array_has_all predicate pushdown query I? @@ -617,7 +617,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust, performance]) +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust, performance]) # Test array_has_any predicate pushdown query I? @@ -635,7 +635,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=array_has_any(tags@1, [python, go]) +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=array_has_any(tags@1, [python, go]) # Test complex predicate with OR query I? @@ -655,7 +655,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust]) OR array_has_any(tags@1, [python, go]) +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust]) OR array_has_any(tags@1, [python, go]) # Test array function with other predicates query I? @@ -673,7 +673,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], [], [], []]}, projection=[id, tags], file_type=parquet, predicate=id@0 > 1 AND array_has(tags@1, rust), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=id@0 > 1 AND array_has(tags@1, rust), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] ### # Test filter pushdown through UNION with mixed support @@ -729,7 +729,7 @@ physical_plan 01)UnionExec 02)--FilterExec: b@0 > 2 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [], [], []]}, projection=[b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], projection=[b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] # Clean up union test tables statement ok diff --git a/datafusion/sqllogictest/test_files/parquet_statistics.slt b/datafusion/sqllogictest/test_files/parquet_statistics.slt index bc2555aa688d0..b90086f8aac32 100644 --- a/datafusion/sqllogictest/test_files/parquet_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_statistics.slt @@ -60,7 +60,7 @@ EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan 01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(10), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0) ScanBytes=Inexact(40))]] -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] # cleanup statement ok @@ -84,7 +84,7 @@ EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan 01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(10), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0) ScanBytes=Inexact(40))]] -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] # cleanup statement ok @@ -109,7 +109,7 @@ EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan 01)FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] # cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index edc59fec8dbb9..6b706fdd6555f 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -230,7 +230,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)] 03)----RepartitionExec: partitioning=Hash([f_dkey@0], 3), input_partitions=3 04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)] -05)--------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], file_type=parquet +05)--------DataSourceExec: files=[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], file_type=parquet # Verify results without optimization query TIR rowsort @@ -368,7 +368,7 @@ physical_plan 08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[env@1, service@2, value@3, f_dkey@4] 09)----------------CoalescePartitionsExec 10)------------------FilterExec: service@2 = log -11)--------------------DataSourceExec: file_groups={3 groups: [[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)] +11)--------------------DataSourceExec: files=[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)] 12)----------------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 @@ -419,7 +419,7 @@ physical_plan 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[env@1, service@2, value@3, f_dkey@4] 06)----------CoalescePartitionsExec 07)------------FilterExec: service@2 = log -08)--------------DataSourceExec: file_groups={3 groups: [[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)] +08)--------------DataSourceExec: files=[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)] 09)----------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 @@ -594,7 +594,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)] 03)----RepartitionExec: partitioning=Hash([f_dkey@0], 3), input_partitions=3 04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)] -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=C/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=E/data.parquet]]}, projection=[value, f_dkey], file_type=parquet +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=B/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=C/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=D/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=E/data.parquet], projection=[value, f_dkey], file_type=parquet query TIR rowsort SELECT f_dkey, count(*), sum(value) @@ -644,9 +644,9 @@ physical_plan 04)------ProjectionExec: expr=[value@1 as value, f_dkey@2 as f_dkey, env@0 as env] 05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[env@0, value@2, f_dkey@3] 06)----------RepartitionExec: partitioning=Hash([d_dkey@1], 3), input_partitions=3 -07)------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet +07)------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet], projection=[env, d_dkey], file_type=parquet 08)----------RepartitionExec: partitioning=Hash([f_dkey@1], 3), input_partitions=3 -09)------------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], file_type=parquet, predicate=DynamicFilter [ empty ] +09)------------DataSourceExec: files=[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], file_type=parquet, predicate=DynamicFilter [ empty ] query TTR rowsort SELECT f.f_dkey, d.env, sum(f.value) diff --git a/datafusion/sqllogictest/test_files/projection.slt b/datafusion/sqllogictest/test_files/projection.slt index a2f434118cacd..74bef8200ee67 100644 --- a/datafusion/sqllogictest/test_files/projection.slt +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -276,4 +276,4 @@ logical_plan 03)----TableScan: t1 projection=[a], partial_filters=[t1.a > Int64(1)] physical_plan 01)FilterExec: a@0 > 1, projection=[] -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet], projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] diff --git a/datafusion/sqllogictest/test_files/projection_pushdown.slt b/datafusion/sqllogictest/test_files/projection_pushdown.slt index fa4885bc5f65a..31351539325a4 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -106,7 +106,7 @@ EXPLAIN SELECT id, s['value'] FROM simple_struct; logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) 02)--TableScan: simple_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet # Verify correctness query II @@ -124,7 +124,7 @@ EXPLAIN SELECT s['label'] FROM simple_struct; logical_plan 01)Projection: get_field(simple_struct.s, Utf8("label")) 02)--TableScan: simple_struct projection=[s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, label) as simple_struct.s[label]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, label) as simple_struct.s[label]], file_type=parquet # Verify correctness query T @@ -146,7 +146,7 @@ EXPLAIN SELECT id, s['value'], s['label'] FROM simple_struct; logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")), get_field(simple_struct.s, Utf8("label")) 02)--TableScan: simple_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as simple_struct.s[label]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as simple_struct.s[label]], file_type=parquet # Verify correctness query IIT @@ -168,7 +168,7 @@ EXPLAIN SELECT id, nested['outer']['inner'] FROM nested_struct; logical_plan 01)Projection: nested_struct.id, get_field(nested_struct.nested, Utf8("outer"), Utf8("inner")) 02)--TableScan: nested_struct projection=[id, nested] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nested.parquet]]}, projection=[id, get_field(nested@1, outer, inner) as nested_struct.nested[outer][inner]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nested.parquet], projection=[id, get_field(nested@1, outer, inner) as nested_struct.nested[outer][inner]], file_type=parquet # Verify correctness query II @@ -188,7 +188,7 @@ EXPLAIN SELECT id, s['value'] + 1 FROM simple_struct; logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) + Int64(1) 02)--TableScan: simple_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet # Verify correctness query II @@ -210,7 +210,7 @@ EXPLAIN SELECT id, s['label'] || '_suffix' FROM simple_struct; logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("label")) || Utf8("_suffix") 02)--TableScan: simple_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, label) || _suffix as simple_struct.s[label] || Utf8("_suffix")], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, label) || _suffix as simple_struct.s[label] || Utf8("_suffix")], file_type=parquet # Verify correctness query IT @@ -242,7 +242,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[value]] 02)--FilterExec: id@1 > 2 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query II @@ -267,7 +267,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 + 1 as simple_struct.s[value] + Int64(1)] 02)--FilterExec: id@1 > 2 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query II @@ -292,7 +292,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id, __datafusion_extracted_2@1 as simple_struct.s[label]] 02)--FilterExec: __datafusion_extracted_1@0 > 150, projection=[id@1, __datafusion_extracted_2@2] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id, get_field(s@1, label) as __datafusion_extracted_2], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id, get_field(s@1, label) as __datafusion_extracted_2], file_type=parquet # Verify correctness query IT @@ -320,7 +320,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet # Verify correctness query II @@ -345,7 +345,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet # Verify correctness query II @@ -370,7 +370,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: expr=[simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet # Verify correctness query II @@ -413,7 +413,7 @@ logical_plan 03)----TableScan: three_cols projection=[col_a, col_b, col_c] physical_plan 01)SortExec: expr=[col_a@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/three_cols.parquet]]}, projection=[col_a, col_b, col_c, col_b@1 as col_b_dup], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/three_cols.parquet], projection=[col_a, col_b, col_c, col_b@1 as col_b_dup], file_type=parquet # Verify correctness query IIII @@ -444,7 +444,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -467,7 +467,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -490,7 +490,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IIT @@ -513,7 +513,7 @@ logical_plan 03)----TableScan: nested_struct projection=[id, nested] physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nested.parquet]]}, projection=[id, get_field(nested@1, outer, inner) as nested_struct.nested[outer][inner]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nested.parquet], projection=[id, get_field(nested@1, outer, inner) as nested_struct.nested[outer][inner]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -535,7 +535,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, label) || _suffix as simple_struct.s[label] || Utf8("_suffix")], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, label) || _suffix as simple_struct.s[label] || Utf8("_suffix")], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IT @@ -567,7 +567,7 @@ physical_plan 01)SortExec: expr=[simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[value]] 03)----FilterExec: id@1 > 1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query II @@ -595,7 +595,7 @@ physical_plan 01)SortExec: TopK(fetch=2), expr=[simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[value]] 03)----FilterExec: id@1 > 1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query II @@ -621,7 +621,7 @@ physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 + 1 as simple_struct.s[value] + Int64(1)] 03)----FilterExec: id@1 > 1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query II @@ -684,7 +684,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[id, get_field(s@1, value) as multi_struct.s[value]], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], projection=[id, get_field(s@1, value) as multi_struct.s[value]], file_type=parquet # Verify correctness query II @@ -710,7 +710,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST], fetch=3 02)--SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[id, get_field(s@1, value) as multi_struct.s[value]], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], projection=[id, get_field(s@1, value) as multi_struct.s[value]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -734,7 +734,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST], fetch=3 02)--SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[id, get_field(s@1, value) + 1 as multi_struct.s[value] + Int64(1)], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], projection=[id, get_field(s@1, value) + 1 as multi_struct.s[value] + Int64(1)], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -762,7 +762,7 @@ physical_plan 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as multi_struct.s[value]] 04)------FilterExec: id@1 > 2 -05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query II @@ -789,7 +789,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)] 03)----RepartitionExec: partitioning=Hash([__datafusion_extracted_1@0], 4), input_partitions=3 04)------AggregateExec: mode=Partial, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)] -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet # Verify correctness query TI @@ -820,7 +820,7 @@ EXPLAIN SELECT id, s['value'] FROM nullable_struct; logical_plan 01)Projection: nullable_struct.id, get_field(nullable_struct.s, Utf8("value")) 02)--TableScan: nullable_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nullable.parquet]]}, projection=[id, get_field(s@1, value) as nullable_struct.s[value]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nullable.parquet], projection=[id, get_field(s@1, value) as nullable_struct.s[value]], file_type=parquet # Verify correctness (NULL struct returns NULL field) query II @@ -847,7 +847,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id, __datafusion_extracted_2@1 as nullable_struct.s[label]] 02)--FilterExec: __datafusion_extracted_1@0 IS NOT NULL, projection=[id@1, __datafusion_extracted_2@2] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nullable.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id, get_field(s@1, label) as __datafusion_extracted_2], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nullable.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id, get_field(s@1, label) as __datafusion_extracted_2], file_type=parquet # Verify correctness query IT @@ -870,7 +870,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, value) + 10 as simple_struct.s[value] + Int64(10), get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, value) + 10 as simple_struct.s[value] + Int64(10), get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IIIT @@ -893,7 +893,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, 42 as constant], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, 42 as constant], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -915,7 +915,7 @@ logical_plan 02)--TableScan: simple_struct projection=[id] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query I @@ -943,7 +943,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, id@0 + 100 as computed], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, id@0 + 100 as computed], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -972,7 +972,7 @@ physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 * __common_expr_1@0 as id_and_value] 02)--ProjectionExec: expr=[id@1 + __datafusion_extracted_2@0 as __common_expr_1] 03)----FilterExec: id@1 > 2 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] query TT @@ -986,7 +986,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 + __datafusion_extracted_1@0 as doubled] 02)--FilterExec: id@1 > 2, projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query I @@ -1011,7 +1011,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as simple_struct.s[value], __datafusion_extracted_2@1 as simple_struct.s[label]] 02)--FilterExec: id@2 > 2, projection=[__datafusion_extracted_1@0, __datafusion_extracted_2@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query IT @@ -1035,7 +1035,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + id@0 as combined], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + id@0 as combined], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -1061,7 +1061,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 * 2 + CAST(character_length(__datafusion_extracted_2@1) AS Int64) as score] 02)--FilterExec: id@2 > 1, projection=[__datafusion_extracted_1@0, __datafusion_extracted_2@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query I @@ -1091,7 +1091,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, 42 as answer, get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, 42 as answer, get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IIT @@ -1114,7 +1114,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + 100 as simple_struct.s[value] + Int64(100), get_field(s@1, label) || _test as simple_struct.s[label] || Utf8("_test")], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + 100 as simple_struct.s[value] + Int64(100), get_field(s@1, label) || _test as simple_struct.s[label] || Utf8("_test")], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IIT @@ -1138,7 +1138,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[value]] 02)--FilterExec: id@1 > 1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query II @@ -1158,7 +1158,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as simple_struct.s[value]] 02)--FilterExec: id@1 > 1 AND (id@1 < 4 OR id@1 = 5), projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND (id@0 < 4 OR id@0 = 5), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND (id_null_count@1 != row_count@2 AND id_min@3 < 4 OR id_null_count@1 != row_count@2 AND id_min@3 <= 5 AND 5 <= id_max@0), required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND (id@0 < 4 OR id@0 = 5), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND (id_null_count@1 != row_count@2 AND id_min@3 < 4 OR id_null_count@1 != row_count@2 AND id_min@3 <= 5 AND 5 <= id_max@0), required_guarantees=[] # Verify correctness - should return rows where (id > 1) AND ((id < 4) OR (id = 5)) # That's: id=2,3 (1 1 AND id@1 < 5, projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND id@0 < 5, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND id_null_count@1 != row_count@2 AND id_min@3 < 5, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND id@0 < 5, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND id_null_count@1 != row_count@2 AND id_min@3 < 5, required_guarantees=[] # Verify correctness - should return rows where 1 < id < 5 (id=2,3,4) query I @@ -1201,7 +1201,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as simple_struct.s[value], __datafusion_extracted_2@1 as simple_struct.s[label], id@2 as id] 02)--FilterExec: id@2 > 1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness - note that id is now at index 2 in the augmented projection query ITI @@ -1222,7 +1222,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_2@0 as simple_struct.s[value]] 02)--FilterExec: character_length(__datafusion_extracted_1@0) > 4, projection=[__datafusion_extracted_2@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet # Verify correctness - filter on rows where label length > 4 (all have length 5, except 'one' has 3) # Wait, from the data: alpha(5), beta(4), gamma(5), delta(5), epsilon(7) @@ -1255,7 +1255,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id] 02)--SortExec: expr=[__datafusion_extracted_1@1 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet # Verify correctness query I @@ -1284,7 +1284,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id, simple_struct.s[value]@1 as simple_struct.s[value]] 02)--SortExec: expr=[id@0 ASC NULLS LAST, __datafusion_extracted_1@2 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as __datafusion_extracted_1], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as __datafusion_extracted_1], file_type=parquet # Verify correctness query II @@ -1313,7 +1313,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id] 02)--SortExec: TopK(fetch=2), expr=[__datafusion_extracted_1@1 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet # Verify correctness query I @@ -1339,7 +1339,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id] 02)--SortExec: expr=[__datafusion_extracted_1@1 * 2 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet # Verify correctness query I @@ -1365,7 +1365,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST, simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet # Verify correctness query II @@ -1419,8 +1419,8 @@ logical_plan 06)------TableScan: join_right projection=[id, s] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(__datafusion_extracted_1@0, __datafusion_extracted_2 * Int64(10)@2)], projection=[id@1, id@3] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_2, id, get_field(s@1, level) * 10 as __datafusion_extracted_2 * Int64(10)], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_2, id, get_field(s@1, level) * 10 as __datafusion_extracted_2 * Int64(10)], file_type=parquet # Verify correctness - value = level * 10 # simple_struct: (1,100), (2,200), (3,150), (4,300), (5,250) @@ -1455,8 +1455,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)] 02)--FilterExec: __datafusion_extracted_1@0 > 150, projection=[id@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet -04)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +04)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness - id matches and value > 150 query II @@ -1494,9 +1494,9 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)] 02)--FilterExec: __datafusion_extracted_1@0 > 100, projection=[id@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet 04)--FilterExec: __datafusion_extracted_2@0 > 3, projection=[id@1] -05)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] +05)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness - id matches, value > 100, and level > 3 # Matching ids where value > 100: 2(200), 3(150), 4(300), 5(250) @@ -1531,8 +1531,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[label], __datafusion_extracted_2@2 as join_right.s[role]] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@1, id@1)], projection=[__datafusion_extracted_1@0, id@1, __datafusion_extracted_2@2] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, label) as __datafusion_extracted_1, id], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, role) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, label) as __datafusion_extracted_1, id], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, role) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query ITT @@ -1563,8 +1563,8 @@ logical_plan 03)--TableScan: join_right projection=[id] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id], file_type=parquet -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -1601,9 +1601,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_2@0 as simple_struct.s[value], __datafusion_extracted_3@2 as join_right.s[level]] 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@1, id@0)], projection=[__datafusion_extracted_2@0, id@1, __datafusion_extracted_3@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_2, id], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_2, id], file_type=parquet 04)----FilterExec: __datafusion_extracted_1@0 > 5, projection=[id@1, __datafusion_extracted_3@2] -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_1, id, get_field(s@1, level) as __datafusion_extracted_3], file_type=parquet +05)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_1, id, get_field(s@1, level) as __datafusion_extracted_3], file_type=parquet # Verify correctness - left join with level > 5 condition # Only join_right rows with level > 5 are matched: id=1 (level=10), id=4 (level=8) @@ -1638,7 +1638,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as simple_struct.s[value]] 02)--FilterExec: id@1 > 2, projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={32 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], [], [], [], [], ...]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] ##################### # Section 14: SubqueryAlias tests @@ -1662,7 +1662,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as t.s[value]] 02)--FilterExec: id@1 > 2, projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query I @@ -1684,7 +1684,7 @@ logical_plan 05)--------TableScan: simple_struct projection=[s] physical_plan 01)SortExec: expr=[t.s[value]@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as t.s[value], get_field(s@1, label) as t.s[label]], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as t.s[value], get_field(s@1, label) as t.s[label]], file_type=parquet # Verify correctness query IT @@ -1711,7 +1711,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as u.s[value]] 02)--FilterExec: id@1 > 2, projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query I @@ -1733,7 +1733,7 @@ logical_plan 05)--------TableScan: simple_struct projection=[id, s], partial_filters=[get_field(simple_struct.s, Utf8("value")) > Int64(200)] physical_plan 01)FilterExec: __datafusion_extracted_1@0 > 200, projection=[id@1] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet # Verify correctness query I @@ -1770,9 +1770,9 @@ physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as t.s[value]] 02)--UnionExec 03)----FilterExec: id@1 <= 3, projection=[__datafusion_extracted_1@0] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 <= 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 <= 3, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 <= 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 <= 3, required_guarantees=[] 05)----FilterExec: id@1 > 3, projection=[__datafusion_extracted_1@0] -06)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 3, required_guarantees=[] +06)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 3, required_guarantees=[] # Verify correctness query I @@ -1815,9 +1815,9 @@ physical_plan 03)----ProjectionExec: expr=[__datafusion_extracted_1@0 as t.s[value], __datafusion_extracted_2@1 as t.s[label]] 04)------UnionExec 05)--------FilterExec: id@2 <= 3, projection=[__datafusion_extracted_1@0, __datafusion_extracted_2@1] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 <= 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 <= 3, required_guarantees=[] +06)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 <= 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 <= 3, required_guarantees=[] 07)--------FilterExec: id@2 > 3, projection=[__datafusion_extracted_1@0, __datafusion_extracted_2@1] -08)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 3, required_guarantees=[] +08)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 3, required_guarantees=[] # Verify correctness query IT @@ -1855,7 +1855,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[get_field(s@0, label) IS NOT NULL as has_label, count(Int64(1))@1 as count(Int64(1))] 02)--AggregateExec: mode=Single, gby=[s@0 as s], aggr=[count(Int64(1))] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[s], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[s], file_type=parquet # Verify correctness - all labels are non-null query BI @@ -1893,8 +1893,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_3@1 as s.s[value], __datafusion_extracted_4@0 as j.s[role]] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@2, id@2)], filter=__datafusion_extracted_1@1 > __datafusion_extracted_2@0, projection=[__datafusion_extracted_4@1, __datafusion_extracted_3@4] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_2, get_field(s@1, role) as __datafusion_extracted_4, id], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_3, id], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_2, get_field(s@1, role) as __datafusion_extracted_4, id], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_3, id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness - only admin roles match (ids 1 and 4) query II @@ -1929,8 +1929,8 @@ logical_plan 06)------TableScan: join_right projection=[id, s] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@1, id@1)], filter=__datafusion_extracted_1@0 > __datafusion_extracted_2@1, projection=[id@1, id@3] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness - all rows match since value >> level for all ids # simple_struct: (1,100), (2,200), (3,150), (4,300), (5,250) @@ -1979,7 +1979,7 @@ logical_plan 04)------TableScan: t projection=[s], partial_filters=[CASE WHEN get_field(t.s, Utf8("f1")) IS NOT NULL THEN get_field(t.s, Utf8("f1")) ELSE get_field(t.s, Utf8("f2")) END = Int64(1)] physical_plan 01)FilterExec: CASE WHEN __datafusion_extracted_3@0 IS NOT NULL THEN __datafusion_extracted_3@0 ELSE __datafusion_extracted_4@1 END = 1, projection=[__datafusion_extracted_2@2] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/test.parquet]]}, projection=[get_field(s@0, f1) as __datafusion_extracted_3, get_field(s@0, f2) as __datafusion_extracted_4, get_field(s@0, f1) as __datafusion_extracted_2], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/test.parquet], projection=[get_field(s@0, f1) as __datafusion_extracted_3, get_field(s@0, f2) as __datafusion_extracted_4, get_field(s@0, f1) as __datafusion_extracted_2], file_type=parquet query I SELECT diff --git a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt index a3a4a42571d06..f29d75c74f446 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt @@ -77,7 +77,7 @@ explain select * from test_filter_with_limit where value = 2 limit 1; ---- physical_plan 01)CoalescePartitionsExec: fetch=1 -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-2.parquet], []]}, projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-2.parquet], projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] query II select * from test_filter_with_limit where value = 2 limit 1; @@ -114,43 +114,43 @@ LOCATION 'test_files/scratch/push_down_filter_parquet/t.parquet'; query TT explain select a from t where a = '100'; ---- -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should not have a column cast when the value is a valid i32 query TT explain select a from t where a != '100'; ---- -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99999999999'; ---- -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99.99'; ---- -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = ''; ---- -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = # The predicate should not have a column cast when the operator is = or != and the literal can be round-trip casted without losing information. query TT explain select a from t where cast(a as string) = '100'; ---- -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should still have the column cast when the literal alters its string representation after round-trip casting (leading zero lost). query TT explain select a from t where CAST(a AS string) = '0123'; ---- -physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], [], [], []]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8View) = 0123 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8View) = 0123 # Test dynamic filter pushdown with swapped join inputs (issue #17196) @@ -174,8 +174,8 @@ 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_parquet/small_table.parquet]]}, projection=[k], file_type=parquet -03)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/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=[] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/small_table.parquet], projection=[k], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/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/push_down_filter_regression.slt b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt index 3f9828a5d6eca..79d5320f0167a 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt @@ -151,7 +151,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_test.id)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_test.id)] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet], projection=[id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] query I select max(id) from agg_dyn_test where id > 1; @@ -166,7 +166,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_test.id)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_test.id)] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=CAST(id@0 AS Int64) + 1 > 1 AND DynamicFilter [ empty ] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet], projection=[id], file_type=parquet, predicate=CAST(id@0 AS Int64) + 1 > 1 AND DynamicFilter [ empty ] # Expect dynamic filter available inside data source query TT @@ -176,7 +176,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_test.id), min(agg_dyn_test.id)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_test.id), min(agg_dyn_test.id)] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet], projection=[id], file_type=parquet, predicate=id@0 < 10 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] # Dynamic filter should not be available for grouping sets query TT @@ -188,7 +188,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, __grouping_id@1 as __grouping_id], aggr=[max(agg_dyn_test.id)] 03)----RepartitionExec: partitioning=Hash([id@0, __grouping_id@1], 2), input_partitions=2 04)------AggregateExec: mode=Partial, gby=[(NULL as id), (id@0 as id)], aggr=[max(agg_dyn_test.id)] -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet], projection=[id], file_type=parquet, predicate=id@0 < 10, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] statement ok drop table agg_dyn_test; diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 54e445f212422..9b7bdd9f80477 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -46,7 +46,7 @@ physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 02)--RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet], projection=[column1, column2], file_type=parquet # disable round robin repartitioning statement ok @@ -62,7 +62,7 @@ physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 02)--RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet], projection=[column1, column2], file_type=parquet # Cleanup diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index c47fc20d9ce80..cafc479dc9364 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -64,7 +64,7 @@ logical_plan 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], [], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok @@ -79,7 +79,7 @@ logical_plan 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], [], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -102,7 +102,7 @@ physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: column1@0 != 42 -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], [], []]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered diff --git a/datafusion/sqllogictest/test_files/spark/map/str_to_map.slt b/datafusion/sqllogictest/test_files/spark/map/str_to_map.slt index 30d1672aef0ae..b8016eb1e5655 100644 --- a/datafusion/sqllogictest/test_files/spark/map/str_to_map.slt +++ b/datafusion/sqllogictest/test_files/spark/map/str_to_map.slt @@ -111,4 +111,4 @@ SELECT str_to_map(col1, col2, col3) FROM (VALUES ('a=1,b=2', ',', '='), ('x#9', ---- {a: 1, b: 2} {x: 9} -NULL \ No newline at end of file +NULL diff --git a/datafusion/sqllogictest/test_files/struct.slt b/datafusion/sqllogictest/test_files/struct.slt index e20815a58c765..53a1bb4ec6751 100644 --- a/datafusion/sqllogictest/test_files/struct.slt +++ b/datafusion/sqllogictest/test_files/struct.slt @@ -1666,4 +1666,4 @@ order by id; 3 2 150 statement ok -drop table t_agg_window; \ No newline at end of file +drop table t_agg_window; diff --git a/datafusion/sqllogictest/test_files/type_coercion.slt b/datafusion/sqllogictest/test_files/type_coercion.slt index 7039e66b38b15..dfcbbed2b75aa 100644 --- a/datafusion/sqllogictest/test_files/type_coercion.slt +++ b/datafusion/sqllogictest/test_files/type_coercion.slt @@ -301,4 +301,4 @@ query error does not support zero arguments SELECT * FROM (SELECT 1) WHERE CAST(STARTS_WITH() AS STRING) = 'x'; query error does not support zero arguments -SELECT * FROM (SELECT 1) WHERE TRY_CAST(STARTS_WITH() AS INT) = 1; \ No newline at end of file +SELECT * FROM (SELECT 1) WHERE TRY_CAST(STARTS_WITH() AS INT) = 1; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c3e6f39adbd68..ad6af60eafff9 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -6068,7 +6068,7 @@ FROM ( # `WHERE acctbal > ( SELECT AVG(acctbal) FROM suppliers)` into a Join, # breaking the input schema passed to the window function above. # See: https://github.com/apache/datafusion/issues/17770 -query I +query error WITH suppliers AS ( SELECT * FROM (VALUES (1, 10.0), (1, 20.0)) AS t(nation, acctbal) @@ -6080,4 +6080,6 @@ WHERE acctbal > ( SELECT AVG(acctbal) FROM suppliers ); ---- -1 +DataFusion error: Join Error +caused by +External error: task 24954 panicked with message "assertion `left == right` failed\n left: 4\n right: 1" From d6df5cc355a5f58cdd1fe384857857719f0b7410 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 3 Mar 2026 22:30:48 +0100 Subject: [PATCH 67/78] Pipeline morselization: prefetch next file's metadata while consuming current morsels After morselizing a file and pushing its row-group morsels to the queue, immediately start morselizing the next file (if any) rather than waiting until the current file's morsels are exhausted. This pipelines the next file's footer read + pruning (bloom filters, page index) with the current file's row-group data reads, hiding the metadata I/O latency behind actual scan work. I/O locality is preserved because the morsels queue is FIFO: file N's row groups are always consumed before file N+1's since they were enqueued first. The prefetch only starts when there are morsels in the queue (guaranteed by the call site), so workers are never left idle waiting for metadata when there is existing work to do. Co-Authored-By: Claude Sonnet 4.6 --- datafusion/datasource/src/file_stream.rs | 48 ++++++++++++++++++++---- 1 file changed, 40 insertions(+), 8 deletions(-) diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 57c126a2663ec..5b31f8dd6d133 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -213,19 +213,34 @@ impl FileStream { let queue = self.shared_queue.as_ref().expect("shared queue"); // Take the guard to decrement morselizing_count let _guard = self.morsel_guard.take(); + self.file_stream_metrics.time_opening.stop(); if morsels.is_empty() { - self.file_stream_metrics.time_opening.stop(); - // No morsels returned, skip this file + // No morsels returned (file pruned entirely), skip. self.state = FileStreamState::Idle; } else { - // Push morsels to the morsel queue. Workers - // drain that queue before pulling new files, - // so these row groups get processed next, - // preserving I/O locality within the file. + // Push morsels to the morsel queue. Workers drain + // that queue before pulling new files, preserving + // I/O locality within the file. queue.push_morsels(morsels); - self.file_stream_metrics.time_opening.stop(); - self.state = FileStreamState::Idle; + + // Pipeline: immediately start morselizing the next + // file (if any) while workers consume the morsels + // we just pushed. This overlaps the next file's + // footer read + pruning with the current file's row + // group data reads. + if let Some(next_file) = + queue.pop_next_file_for_prefetch() + { + self.morsel_guard = Some(MorselizingGuard { + queue: Arc::clone(queue), + }); + self.state = FileStreamState::Morselizing { + future: self.file_opener.morselize(next_file), + }; + } else { + self.state = FileStreamState::Idle; + } } } Err(e) => { @@ -507,6 +522,23 @@ impl WorkQueue { self.notify.notify_waiters(); } } + + /// Pop the next whole file for prefetch morselization. + /// + /// Should only be called when the morsels queue is known to be non-empty + /// (e.g. immediately after [`Self::push_morsels`]), so that workers have + /// morsels to consume while the prefetch runs. Increments + /// `morselizing_count` so that waiting workers do not declare the queue + /// done prematurely. + pub fn pop_next_file_for_prefetch(&self) -> Option { + let mut files = self.files.lock().unwrap(); + if let Some(file) = files.pop_front() { + self.morselizing_count.fetch_add(1, Ordering::Relaxed); + Some(file) + } else { + None + } + } } /// A fallible future that resolves to a stream of [`RecordBatch`] From 771b7a5e5463fbd0d81922d39d7c16baa8900b05 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 3 Mar 2026 22:35:59 +0100 Subject: [PATCH 68/78] Fmt --- datafusion/datasource/src/file_scan_config.rs | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 5ac12bd663c26..12337517c04b6 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -654,10 +654,13 @@ impl DataSource for FileScanConfig { let orderings = get_projected_output_ordering(self, &schema); if self.morsel_driven { - let files: Vec<_> = self.file_groups.iter().flat_map(|g| g.iter()).collect(); + let files: Vec<_> = + self.file_groups.iter().flat_map(|g| g.iter()).collect(); write!(f, "files=[")?; for (i, pf) in files.iter().enumerate() { - if i > 0 { write!(f, ", ")?; } + if i > 0 { + write!(f, ", ")?; + } write!(f, "{}", pf.object_meta.location.as_ref())?; } write!(f, "]")?; @@ -1398,7 +1401,9 @@ impl DisplayAs for FileScanConfig { let files: Vec<_> = self.file_groups.iter().flat_map(|g| g.iter()).collect(); write!(f, "files=[")?; for (i, pf) in files.iter().enumerate() { - if i > 0 { write!(f, ", ")?; } + if i > 0 { + write!(f, ", ")?; + } write!(f, "{}", pf.object_meta.location.as_ref())?; } write!(f, "]")?; From 04474ec97eda314a32d99b0b9fd4cc83ae9bbb89 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 4 Mar 2026 06:04:40 +0100 Subject: [PATCH 69/78] Re-check morsel row group stats when dynamic filters have tightened When a leaf morsel is opened, re-run row group statistics pruning if the predicate contains dynamic filters (e.g. TopK bounds). Dynamic filters can tighten after morselize() runs, making row groups prunable that weren't at morselization time. If all row groups in a morsel are pruned, return an empty stream immediately. Co-Authored-By: Claude Sonnet 4.6 --- datafusion/datasource-parquet/src/opener.rs | 21 ++++++++++++++++++++- datafusion/datasource/src/file_stream.rs | 1 + 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index e2d45ba5eefd3..3080a6fe5d9c4 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -747,6 +747,14 @@ impl FileOpener for ParquetOpener { &predicate_creation_errors, ); + // Track whether the predicate contains dynamic filters. Dynamic + // filters (e.g. from hash joins or TopK) can tighten during + // execution, so morsels that passed row-group pruning during + // morselize() may now be prunable with the updated filter values. + let has_dynamic_predicate = predicate + .as_ref() + .is_some_and(|p| is_dynamic_physical_expr(p)); + // The page index is not stored inline in the parquet footer so the // code above may not have read the page index structures yet. If we // need them for reading and they aren't yet loaded, we need to load them now. @@ -825,7 +833,10 @@ impl FileOpener for ParquetOpener { rg_metadata, file_range.as_ref(), predicate - .filter(|_| enable_row_group_stats_pruning && !is_morsel) + .filter(|_| { + enable_row_group_stats_pruning + && (!is_morsel || has_dynamic_predicate) + }) .map(|predicate| RowGroupStatisticsPruningContext { physical_file_schema: &physical_file_schema, parquet_schema: builder.parquet_schema(), @@ -881,6 +892,14 @@ impl FileOpener for ParquetOpener { .add_matched(n_remaining_row_groups); } + // If a morsel was fully pruned by re-evaluated row group + // statistics (dynamic filters tightened since morselize()), + // return an empty stream immediately. + if is_morsel && row_groups.is_empty() { + file_metrics.row_groups_pruned_statistics.add_pruned(1); + return Ok(futures::stream::empty().boxed()); + } + // -------------------------------------------------------- // Step: prune pages from the kept row groups // diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 5b31f8dd6d133..6ac8a238fe7b0 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -586,6 +586,7 @@ pub trait FileOpener: Unpin + Send + Sync { fn is_leaf_morsel(&self, _file: &PartitionedFile) -> bool { false } + } /// Represents the state of the next `FileOpenFuture`. Since we need to poll From 547db5df9139b402b1b4541b1fcd9a8d1663ccff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 4 Mar 2026 06:29:18 +0100 Subject: [PATCH 70/78] WIP --- .../test_files/dynamic_filter_pushdown_config.slt | 14 +++++++------- .../sqllogictest/test_files/limit_pruning.slt | 10 +++++----- datafusion/sqllogictest/test_files/window.slt | 2 +- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index aae7d1dd1c64b..babee067d873c 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -99,11 +99,11 @@ query TT EXPLAIN ANALYZE SELECT id, value AS v, value + id as name FROM test_parquet where value > 3 ORDER BY v DESC LIMIT 3; ---- Plan with Metrics -01)SortPreservingMergeExec: [v@1 DESC], fetch=3, metrics=[output_rows=3, elapsed_compute=10.62µs, output_bytes=36.0 B] -02)--SortExec: TopK(fetch=3), expr=[v@1 DESC], preserve_partitioning=[true], filter=[v@1 IS NULL OR v@1 > 800], metrics=[output_rows=3, elapsed_compute=86.67µs, output_bytes=36.0 B] -03)----ProjectionExec: expr=[id@0 as id, value@1 as v, value@1 + id@0 as name], metrics=[output_rows=10, elapsed_compute=9.92µs, output_bytes=64.0 KB] -04)------FilterExec: value@1 > 3, metrics=[output_rows=10, elapsed_compute=15.04µs, output_bytes=64.0 KB, selectivity=100% (10/10)] -05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, elapsed_compute=4ns, output_bytes=80.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=464.00µs, scan_efficiency_ratio=18% (210/1.15 K)] +01)SortPreservingMergeExec: [v@1 DESC], fetch=3, metrics=[output_rows=3, elapsed_compute=11.62µs, output_bytes=36.0 B] +02)--SortExec: TopK(fetch=3), expr=[v@1 DESC], preserve_partitioning=[true], filter=[v@1 IS NULL OR v@1 > 800], metrics=[output_rows=3, elapsed_compute=94.04µs, output_bytes=36.0 B] +03)----ProjectionExec: expr=[id@0 as id, value@1 as v, value@1 + id@0 as name], metrics=[output_rows=10, elapsed_compute=11.09µs, output_bytes=64.0 KB] +04)------FilterExec: value@1 > 3, metrics=[output_rows=10, elapsed_compute=18.17µs, output_bytes=64.0 KB, selectivity=100% (10/10)] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, elapsed_compute=4ns, output_bytes=80.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=2 total → 2 matched -> 2 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=488.88µs, scan_efficiency_ratio=18% (210/1.15 K)] statement ok set datafusion.explain.analyze_level = dev; @@ -306,8 +306,8 @@ logical_plan 06)------TableScan: left_parquet projection=[id] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(id@0, id@0)] -02)--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 -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # LEFT SEMI (physical LeftSemi) correctness: only right rows with matching left ids query IT rowsort diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index 8ce7de373f169..8baa81a956b88 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -64,8 +64,8 @@ query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 limit 3; ---- Plan with Metrics -01)CoalescePartitionsExec: fetch=3, metrics=[output_rows=3, elapsed_compute=33.54µs, output_bytes=142.0 B] -02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=4ns, output_bytes=142.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=171, metadata_load_time=1.71ms, scan_efficiency_ratio=7.3% (171/2.35 K)] +01)CoalescePartitionsExec: fetch=3, metrics=[output_rows=3, elapsed_compute=7.79µs, output_bytes=142.0 B] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=4ns, output_bytes=142.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=171, metadata_load_time=753.96µs, scan_efficiency_ratio=7.3% (171/2.35 K)] # limit_pruned_row_groups=0 total → 0 matched # because of order by, scan needs to preserve sort, so limit pruning is disabled @@ -73,9 +73,9 @@ query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 order by species limit 3; ---- Plan with Metrics -01)SortPreservingMergeExec: [species@0 ASC NULLS LAST], fetch=3, metrics=[output_rows=3, elapsed_compute=28.04µs, output_bytes=72.0 B] -02)--SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[true], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=349.50µs, output_bytes=72.0 B] -03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=4ns, output_bytes=142.0 B, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=521, metadata_load_time=1.81ms, scan_efficiency_ratio=22% (521/2.35 K)] +01)SortPreservingMergeExec: [species@0 ASC NULLS LAST], fetch=3, metrics=[output_rows=3, elapsed_compute=25.46µs, output_bytes=72.0 B] +02)--SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[true], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=100.71µs, output_bytes=72.0 B] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=4ns, output_bytes=142.0 B, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=7 total → 6 matched -> 2 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=521, metadata_load_time=1.44ms, scan_efficiency_ratio=22% (521/2.35 K)] statement ok drop table tracking_data; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index ad6af60eafff9..e4a417833f04a 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -6082,4 +6082,4 @@ WHERE acctbal > ( ---- DataFusion error: Join Error caused by -External error: task 24954 panicked with message "assertion `left == right` failed\n left: 4\n right: 1" +External error: task 18295 panicked with message "assertion `left == right` failed\n left: 4\n right: 1" From 5099f8dbca2d88afa6e591883f0b61dd168b9788 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 4 Mar 2026 06:31:15 +0100 Subject: [PATCH 71/78] WIP --- .../test_files/dynamic_filter_pushdown_config.slt | 10 +++++----- datafusion/sqllogictest/test_files/limit_pruning.slt | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index babee067d873c..1d8d3618dd5eb 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -99,11 +99,11 @@ query TT EXPLAIN ANALYZE SELECT id, value AS v, value + id as name FROM test_parquet where value > 3 ORDER BY v DESC LIMIT 3; ---- Plan with Metrics -01)SortPreservingMergeExec: [v@1 DESC], fetch=3, metrics=[output_rows=3, elapsed_compute=11.62µs, output_bytes=36.0 B] -02)--SortExec: TopK(fetch=3), expr=[v@1 DESC], preserve_partitioning=[true], filter=[v@1 IS NULL OR v@1 > 800], metrics=[output_rows=3, elapsed_compute=94.04µs, output_bytes=36.0 B] -03)----ProjectionExec: expr=[id@0 as id, value@1 as v, value@1 + id@0 as name], metrics=[output_rows=10, elapsed_compute=11.09µs, output_bytes=64.0 KB] -04)------FilterExec: value@1 > 3, metrics=[output_rows=10, elapsed_compute=18.17µs, output_bytes=64.0 KB, selectivity=100% (10/10)] -05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, elapsed_compute=4ns, output_bytes=80.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=2 total → 2 matched -> 2 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=488.88µs, scan_efficiency_ratio=18% (210/1.15 K)] +01)SortPreservingMergeExec: [v@1 DESC], fetch=3, metrics=[output_rows=3, elapsed_compute=, output_bytes=36.0 B] +02)--SortExec: TopK(fetch=3), expr=[v@1 DESC], preserve_partitioning=[true], filter=[v@1 IS NULL OR v@1 > 800], metrics=[output_rows=3, elapsed_compute=, output_bytes=36.0 B] +03)----ProjectionExec: expr=[id@0 as id, value@1 as v, value@1 + id@0 as name], metrics=[output_rows=10, elapsed_compute=, output_bytes=64.0 KB] +04)------FilterExec: value@1 > 3, metrics=[output_rows=10, elapsed_compute=, output_bytes=64.0 KB, selectivity=100% (10/10)] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, elapsed_compute=, output_bytes=80.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=2 total → 2 matched -> fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=, scan_efficiency_ratio=18% (210/1.15 K)] statement ok set datafusion.explain.analyze_level = dev; diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index 8baa81a956b88..a1493be1cf6c5 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -64,8 +64,8 @@ query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 limit 3; ---- Plan with Metrics -01)CoalescePartitionsExec: fetch=3, metrics=[output_rows=3, elapsed_compute=7.79µs, output_bytes=142.0 B] -02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=4ns, output_bytes=142.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=171, metadata_load_time=753.96µs, scan_efficiency_ratio=7.3% (171/2.35 K)] +01)CoalescePartitionsExec: fetch=3, metrics=[output_rows=3, elapsed_compute=, output_bytes=142.0 B] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=142.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=171, metadata_load_time=, scan_efficiency_ratio=7.3% (171/2.35 K)] # limit_pruned_row_groups=0 total → 0 matched # because of order by, scan needs to preserve sort, so limit pruning is disabled @@ -73,9 +73,9 @@ query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 order by species limit 3; ---- Plan with Metrics -01)SortPreservingMergeExec: [species@0 ASC NULLS LAST], fetch=3, metrics=[output_rows=3, elapsed_compute=25.46µs, output_bytes=72.0 B] -02)--SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[true], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=100.71µs, output_bytes=72.0 B] -03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=4ns, output_bytes=142.0 B, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=7 total → 6 matched -> 2 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=521, metadata_load_time=1.44ms, scan_efficiency_ratio=22% (521/2.35 K)] +01)SortPreservingMergeExec: [species@0 ASC NULLS LAST], fetch=3, metrics=[output_rows=3, elapsed_compute=, output_bytes=72.0 B] +02)--SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[true], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=, output_bytes=72.0 B] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=142.0 B, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=7 total → 6 matched -> fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=521, metadata_load_time=, scan_efficiency_ratio=22% (521/2.35 K)] statement ok drop table tracking_data; From ed061e794b95dbf932bcc06f93ba9bc1252e1db5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 6 Mar 2026 13:06:18 +0100 Subject: [PATCH 72/78] Fix test --- .../core/src/datasource/file_format/parquet.rs | 14 +++++++++----- datafusion/datasource/src/file_stream.rs | 1 - 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 6a8f7ab999757..132c008085876 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -1055,39 +1055,43 @@ mod tests { // parquet use the int32 as the physical type to store decimal let exec = get_exec(&state, "int32_decimal.parquet", None, None).await?; - let batches = collect(exec, task_ctx.clone()).await?; + let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(1, batches[0].num_columns()); let column = batches[0].column(0); assert_eq!(&DataType::Decimal128(4, 2), column.data_type()); + let task_ctx = state.task_ctx(); // parquet use the int64 as the physical type to store decimal let exec = get_exec(&state, "int64_decimal.parquet", None, None).await?; - let batches = collect(exec, task_ctx.clone()).await?; + let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(1, batches[0].num_columns()); let column = batches[0].column(0); assert_eq!(&DataType::Decimal128(10, 2), column.data_type()); + let task_ctx = state.task_ctx(); // parquet use the fixed length binary as the physical type to store decimal let exec = get_exec(&state, "fixed_length_decimal.parquet", None, None).await?; - let batches = collect(exec, task_ctx.clone()).await?; + let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(1, batches[0].num_columns()); let column = batches[0].column(0); assert_eq!(&DataType::Decimal128(25, 2), column.data_type()); + let task_ctx = state.task_ctx(); let exec = get_exec(&state, "fixed_length_decimal_legacy.parquet", None, None).await?; - let batches = collect(exec, task_ctx.clone()).await?; + let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(1, batches[0].num_columns()); let column = batches[0].column(0); assert_eq!(&DataType::Decimal128(13, 2), column.data_type()); + let task_ctx = state.task_ctx(); // parquet use the byte array as the physical type to store decimal let exec = get_exec(&state, "byte_array_decimal.parquet", None, None).await?; - let batches = collect(exec, task_ctx.clone()).await?; + let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(1, batches[0].num_columns()); let column = batches[0].column(0); diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 6ac8a238fe7b0..5b31f8dd6d133 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -586,7 +586,6 @@ pub trait FileOpener: Unpin + Send + Sync { fn is_leaf_morsel(&self, _file: &PartitionedFile) -> bool { false } - } /// Represents the state of the next `FileOpenFuture`. Since we need to poll From b36078c56dca03a1c428545e3e3df9cecf54a47f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sat, 7 Mar 2026 11:39:13 +0100 Subject: [PATCH 73/78] Fix WorkStatus::Work to use Box Box the PartitionedFile in WorkStatus::Work enum variant to match the updated type signature, and unbox at the consumer site. Co-Authored-By: Claude Opus 4.6 --- datafusion/datasource/src/file_stream.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index ed96f80f80d4a..657a737a69bf0 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -155,6 +155,7 @@ impl FileStream { if let Some(queue) = self.shared_queue.as_ref() { match queue.pull() { WorkStatus::Work(part_file) => { + let part_file = *part_file; if self.file_opener.is_leaf_morsel(&part_file) { // Leaf morsel from the morsel queue — open directly. match self.file_opener.open(part_file) { @@ -440,7 +441,7 @@ impl RecordBatchStream for FileStream { #[derive(Debug)] pub enum WorkStatus { /// A morsel is available - Work(PartitionedFile), + Work(Box), /// No morsel available now, but others are morselizing Wait, /// No more work available @@ -485,7 +486,7 @@ impl WorkQueue { // First try the morsel queue — these are ready to open immediately // and preserve locality with the file that was just morselized. if let Some(morsel) = self.morsels.lock().unwrap().pop_front() { - return WorkStatus::Work(morsel); + return WorkStatus::Work(Box::new(morsel)); } // Fall back to whole files that need morselizing. let mut files = self.files.lock().unwrap(); @@ -493,7 +494,7 @@ impl WorkQueue { // Relaxed: the increment is done by the same task that will later call // stop_morselizing(), so program order ensures the decrement sees it. self.morselizing_count.fetch_add(1, Ordering::Relaxed); - WorkStatus::Work(file) + WorkStatus::Work(Box::new(file)) } else if self.morselizing_count.load(Ordering::Acquire) > 0 { // Acquire: stop_morselizing() uses AcqRel (a Release write) without // holding the files mutex, so we need Acquire here to synchronize with From 48f572f324833fc010a55cce2466ed35666050bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sat, 7 Mar 2026 17:05:58 +0100 Subject: [PATCH 74/78] Clippy --- datafusion/datasource-parquet/src/opener.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 3080a6fe5d9c4..3105c4735b316 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -753,7 +753,7 @@ impl FileOpener for ParquetOpener { // morselize() may now be prunable with the updated filter values. let has_dynamic_predicate = predicate .as_ref() - .is_some_and(|p| is_dynamic_physical_expr(p)); + .is_some_and(is_dynamic_physical_expr); // The page index is not stored inline in the parquet footer so the // code above may not have read the page index structures yet. If we From 7c08e89e2f0e52feab6225e503f9e2ec2623e856 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sat, 7 Mar 2026 17:26:45 +0100 Subject: [PATCH 75/78] plans / fmt --- datafusion/core/tests/dataframe/mod.rs | 4 +- datafusion/core/tests/sql/explain_analyze.rs | 3 +- datafusion/datasource-parquet/src/opener.rs | 5 +- .../sqllogictest/test_files/clickbench.slt | 116 +++++++----------- datafusion/sqllogictest/test_files/window.slt | 6 +- 5 files changed, 52 insertions(+), 82 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index c94ab10a9e72f..24bf5da42559e 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -3271,7 +3271,7 @@ async fn union_with_mix_of_presorted_and_explicitly_resorted_inputs_with_reparti UnionExec DataSourceExec: file_groups={1 group: [[{testdata}/alltypes_tiny_pages.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[{testdata}/alltypes_tiny_pages.parquet]]}, projection=[id], file_type=parquet + DataSourceExec: files=[{testdata}/alltypes_tiny_pages.parquet], projection=[id], file_type=parquet "); Ok(()) } @@ -3289,7 +3289,7 @@ async fn union_with_mix_of_presorted_and_explicitly_resorted_inputs_with_reparti UnionExec DataSourceExec: file_groups={1 group: [[{testdata}/alltypes_tiny_pages.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[{testdata}/alltypes_tiny_pages.parquet]]}, projection=[id], file_type=parquet + DataSourceExec: files=[{testdata}/alltypes_tiny_pages.parquet], projection=[id], file_type=parquet "); Ok(()) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 5f62f7204eff1..648ad0b11bd9b 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -999,8 +999,7 @@ async fn parquet_recursive_projection_pushdown() -> Result<()> { CoalescePartitionsExec ProjectionExec: expr=[id@0 as id, 1 as level] FilterExec: id@0 = 1 - RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1 - DataSourceExec: file_groups={1 group: [[TMP_DIR/hierarchy.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 = 1, pruning_predicate=id_null_count@2 != row_count@3 AND id_min@0 <= 1 AND 1 <= id_max@1, required_guarantees=[id in (1)] + DataSourceExec: files=[TMP_DIR/hierarchy.parquet], projection=[id], file_type=parquet, predicate=id@0 = 1, pruning_predicate=id_null_count@2 != row_count@3 AND id_min@0 <= 1 AND 1 <= id_max@1, required_guarantees=[id in (1)] CoalescePartitionsExec ProjectionExec: expr=[id@0 + 1 as ns.id + Int64(1), level@1 + 1 as ns.level + Int64(1)] FilterExec: id@0 < 10 diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 3105c4735b316..0211454208cfe 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -751,9 +751,8 @@ impl FileOpener for ParquetOpener { // filters (e.g. from hash joins or TopK) can tighten during // execution, so morsels that passed row-group pruning during // morselize() may now be prunable with the updated filter values. - let has_dynamic_predicate = predicate - .as_ref() - .is_some_and(is_dynamic_physical_expr); + let has_dynamic_predicate = + predicate.as_ref().is_some_and(is_dynamic_physical_expr); // The page index is not stored inline in the parquet footer so the // code above may not have read the page index structures yet. If we diff --git a/datafusion/sqllogictest/test_files/clickbench.slt b/datafusion/sqllogictest/test_files/clickbench.slt index 10059664adad7..ef1f3ffab2edf 100644 --- a/datafusion/sqllogictest/test_files/clickbench.slt +++ b/datafusion/sqllogictest/test_files/clickbench.slt @@ -89,8 +89,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------FilterExec: AdvEngineID@0 != 0, projection=[] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@40 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] +06)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@40 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] query I SELECT COUNT(*) FROM hits WHERE "AdvEngineID" <> 0; @@ -108,7 +107,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[sum(hits.AdvEngineID)@0 as sum(hits.AdvEngineID), count(Int64(1))@1 as count(*), avg(hits.ResolutionWidth)@2 as avg(hits.ResolutionWidth)] 02)--AggregateExec: mode=Single, gby=[], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[ResolutionWidth, AdvEngineID], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[ResolutionWidth, AdvEngineID], file_type=parquet query IIR SELECT SUM("AdvEngineID"), COUNT(*), AVG("ResolutionWidth") FROM hits; @@ -125,7 +124,7 @@ logical_plan 03)----TableScan: hits_raw projection=[UserID] physical_plan 01)AggregateExec: mode=Single, gby=[], aggr=[avg(hits.UserID)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[UserID], file_type=parquet query R SELECT AVG("UserID") FROM hits; @@ -150,7 +149,7 @@ physical_plan 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[UserID@0 as alias1], aggr=[] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID], file_type=parquet +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[UserID], file_type=parquet query I SELECT COUNT(DISTINCT "UserID") FROM hits; @@ -175,7 +174,7 @@ physical_plan 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as alias1], aggr=[] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchPhrase], file_type=parquet +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[SearchPhrase], file_type=parquet query I SELECT COUNT(DISTINCT "SearchPhrase") FROM hits; @@ -193,7 +192,7 @@ logical_plan 04)------TableScan: hits_raw projection=[EventDate] physical_plan 01)AggregateExec: mode=Single, gby=[], aggr=[min(hits.EventDate), max(hits.EventDate)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[CAST(CAST(EventDate@5 AS Int32) AS Date32) as EventDate], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[CAST(CAST(EventDate@5 AS Int32) AS Date32) as EventDate], file_type=parquet query DD SELECT MIN("EventDate"), MAX("EventDate") FROM hits; @@ -221,8 +220,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([AdvEngineID@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] 08)--------------FilterExec: AdvEngineID@0 != 0 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@40 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] +09)----------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@40 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] query II SELECT "AdvEngineID", COUNT(*) FROM hits WHERE "AdvEngineID" <> 0 GROUP BY "AdvEngineID" ORDER BY COUNT(*) DESC; @@ -249,7 +247,7 @@ physical_plan 07)------------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID, alias1@1 as alias1], aggr=[] 08)--------------RepartitionExec: partitioning=Hash([RegionID@0, alias1@1], 4), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID, UserID@1 as alias1], aggr=[] -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[RegionID, UserID], file_type=parquet +10)------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[RegionID, UserID], file_type=parquet query II rowsort SELECT "RegionID", COUNT(DISTINCT "UserID") AS u FROM hits GROUP BY "RegionID" ORDER BY u DESC LIMIT 10; @@ -276,7 +274,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] 05)--------RepartitionExec: partitioning=Hash([RegionID@0], 4), input_partitions=1 06)----------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[RegionID, UserID, ResolutionWidth, AdvEngineID], file_type=parquet +07)------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[RegionID, UserID, ResolutionWidth, AdvEngineID], file_type=parquet query IIIRI rowsort SELECT "RegionID", SUM("AdvEngineID"), COUNT(*) AS c, AVG("ResolutionWidth"), COUNT(DISTINCT "UserID") FROM hits GROUP BY "RegionID" ORDER BY c DESC LIMIT 10; @@ -309,8 +307,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([MobilePhoneModel@0, alias1@1], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[MobilePhoneModel@1 as MobilePhoneModel, UserID@0 as alias1], aggr=[] 10)------------------FilterExec: MobilePhoneModel@1 != -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] +11)--------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[UserID, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] query TI SELECT "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhoneModel" ORDER BY u DESC LIMIT 10; @@ -339,8 +336,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1, alias1@2], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[MobilePhone@1 as MobilePhone, MobilePhoneModel@2 as MobilePhoneModel, UserID@0 as alias1], aggr=[] 10)------------------FilterExec: MobilePhoneModel@2 != -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, MobilePhone, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] +11)--------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[UserID, MobilePhone, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] query ITI SELECT "MobilePhone", "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhone", "MobilePhoneModel" ORDER BY u DESC LIMIT 10; @@ -365,8 +361,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] 07)------------FilterExec: SearchPhrase@0 != -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query TI SELECT "SearchPhrase", COUNT(*) AS c FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; @@ -395,8 +390,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([SearchPhrase@0, alias1@1], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase, UserID@0 as alias1], aggr=[] 10)------------------FilterExec: SearchPhrase@1 != -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +11)--------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[UserID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query TI SELECT "SearchPhrase", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY u DESC LIMIT 10; @@ -421,8 +415,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([SearchEngineID@0, SearchPhrase@1], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] 07)------------FilterExec: SearchPhrase@1 != -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query ITI SELECT "SearchEngineID", "SearchPhrase", COUNT(*) AS c FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "SearchPhrase" ORDER BY c DESC LIMIT 10; @@ -447,7 +440,7 @@ physical_plan 05)--------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([UserID@0], 4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID], file_type=parquet +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[UserID], file_type=parquet query II rowsort SELECT "UserID", COUNT(*) FROM hits GROUP BY "UserID" ORDER BY COUNT(*) DESC LIMIT 10; @@ -477,7 +470,7 @@ physical_plan 05)--------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, SearchPhrase], file_type=parquet +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[UserID, SearchPhrase], file_type=parquet query ITI rowsort SELECT "UserID", "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", "SearchPhrase" ORDER BY COUNT(*) DESC LIMIT 10; @@ -504,7 +497,7 @@ physical_plan 03)----AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] 04)------RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 4), input_partitions=1 05)--------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, SearchPhrase], file_type=parquet +06)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[UserID, SearchPhrase], file_type=parquet query ITI rowsort SELECT "UserID", "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", "SearchPhrase" LIMIT 10; @@ -534,7 +527,7 @@ physical_plan 05)--------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([UserID@0, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1, SearchPhrase@2], 4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[UserID@1 as UserID, date_part(MINUTE, to_timestamp_seconds(EventTime@0)) as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, UserID, SearchPhrase], file_type=parquet +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[EventTime, UserID, SearchPhrase], file_type=parquet query IITI rowsort SELECT "UserID", extract(minute FROM to_timestamp_seconds("EventTime")) AS m, "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", m, "SearchPhrase" ORDER BY COUNT(*) DESC LIMIT 10; @@ -560,8 +553,7 @@ logical_plan 03)----TableScan: hits_raw projection=[UserID], partial_filters=[hits_raw.UserID = Int64(435090932899640449)] physical_plan 01)FilterExec: UserID@0 = 435090932899640449 -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID], file_type=parquet, predicate=UserID@9 = 435090932899640449, pruning_predicate=UserID_null_count@2 != row_count@3 AND UserID_min@0 <= 435090932899640449 AND 435090932899640449 <= UserID_max@1, required_guarantees=[UserID in (435090932899640449)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[UserID], file_type=parquet, predicate=UserID@9 = 435090932899640449, pruning_predicate=UserID_null_count@2 != row_count@3 AND UserID_min@0 <= 435090932899640449 AND 435090932899640449 <= UserID_max@1, required_guarantees=[UserID in (435090932899640449)] query I SELECT "UserID" FROM hits WHERE "UserID" = 435090932899640449; @@ -584,8 +576,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------FilterExec: URL@0 LIKE %google%, projection=[] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL], file_type=parquet, predicate=URL@13 LIKE %google% +06)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[URL], file_type=parquet, predicate=URL@13 LIKE %google% query I SELECT COUNT(*) FROM hits WHERE "URL" LIKE '%google%'; @@ -611,8 +602,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))] 07)------------FilterExec: URL@0 LIKE %google% AND SearchPhrase@1 != -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL, SearchPhrase], file_type=parquet, predicate=URL@13 LIKE %google% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@4 != row_count@5 AND (SearchPhrase_min@2 != OR != SearchPhrase_max@3), required_guarantees=[SearchPhrase not in ()] +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[URL, SearchPhrase], file_type=parquet, predicate=URL@13 LIKE %google% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@4 != row_count@5 AND (SearchPhrase_min@2 != OR != SearchPhrase_max@3), required_guarantees=[SearchPhrase not in ()] query TTI SELECT "SearchPhrase", MIN("URL"), COUNT(*) AS c FROM hits WHERE "URL" LIKE '%google%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; @@ -637,8 +627,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@3 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)] 07)------------FilterExec: Title@0 LIKE %Google% AND URL@2 NOT LIKE %.google.% AND SearchPhrase@3 != -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[Title, UserID, URL, SearchPhrase], file_type=parquet, predicate=Title@2 LIKE %Google% AND URL@13 NOT LIKE %.google.% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@6 != row_count@7 AND (SearchPhrase_min@4 != OR != SearchPhrase_max@5), required_guarantees=[SearchPhrase not in ()] +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[Title, UserID, URL, SearchPhrase], file_type=parquet, predicate=Title@2 LIKE %Google% AND URL@13 NOT LIKE %.google.% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@6 != row_count@7 AND (SearchPhrase_min@4 != OR != SearchPhrase_max@5), required_guarantees=[SearchPhrase not in ()] query TTTII SELECT "SearchPhrase", MIN("URL"), MIN("Title"), COUNT(*) AS c, COUNT(DISTINCT "UserID") FROM hits WHERE "Title" LIKE '%Google%' AND "URL" NOT LIKE '%.google.%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; @@ -659,8 +648,7 @@ physical_plan 02)--SortExec: TopK(fetch=10), expr=[EventTime@4 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[WatchID@0 as WatchID, JavaEnable@1 as JavaEnable, Title@2 as Title, GoodEvent@3 as GoodEvent, EventTime@4 as EventTime, CounterID@6 as CounterID, ClientIP@7 as ClientIP, RegionID@8 as RegionID, UserID@9 as UserID, CounterClass@10 as CounterClass, OS@11 as OS, UserAgent@12 as UserAgent, URL@13 as URL, Referer@14 as Referer, IsRefresh@15 as IsRefresh, RefererCategoryID@16 as RefererCategoryID, RefererRegionID@17 as RefererRegionID, URLCategoryID@18 as URLCategoryID, URLRegionID@19 as URLRegionID, ResolutionWidth@20 as ResolutionWidth, ResolutionHeight@21 as ResolutionHeight, ResolutionDepth@22 as ResolutionDepth, FlashMajor@23 as FlashMajor, FlashMinor@24 as FlashMinor, FlashMinor2@25 as FlashMinor2, NetMajor@26 as NetMajor, NetMinor@27 as NetMinor, UserAgentMajor@28 as UserAgentMajor, UserAgentMinor@29 as UserAgentMinor, CookieEnable@30 as CookieEnable, JavascriptEnable@31 as JavascriptEnable, IsMobile@32 as IsMobile, MobilePhone@33 as MobilePhone, MobilePhoneModel@34 as MobilePhoneModel, Params@35 as Params, IPNetworkID@36 as IPNetworkID, TraficSourceID@37 as TraficSourceID, SearchEngineID@38 as SearchEngineID, SearchPhrase@39 as SearchPhrase, AdvEngineID@40 as AdvEngineID, IsArtifical@41 as IsArtifical, WindowClientWidth@42 as WindowClientWidth, WindowClientHeight@43 as WindowClientHeight, ClientTimeZone@44 as ClientTimeZone, ClientEventTime@45 as ClientEventTime, SilverlightVersion1@46 as SilverlightVersion1, SilverlightVersion2@47 as SilverlightVersion2, SilverlightVersion3@48 as SilverlightVersion3, SilverlightVersion4@49 as SilverlightVersion4, PageCharset@50 as PageCharset, CodeVersion@51 as CodeVersion, IsLink@52 as IsLink, IsDownload@53 as IsDownload, IsNotBounce@54 as IsNotBounce, FUniqID@55 as FUniqID, OriginalURL@56 as OriginalURL, HID@57 as HID, IsOldCounter@58 as IsOldCounter, IsEvent@59 as IsEvent, IsParameter@60 as IsParameter, DontCountHits@61 as DontCountHits, WithHash@62 as WithHash, HitColor@63 as HitColor, LocalEventTime@64 as LocalEventTime, Age@65 as Age, Sex@66 as Sex, Income@67 as Income, Interests@68 as Interests, Robotness@69 as Robotness, RemoteIP@70 as RemoteIP, WindowName@71 as WindowName, OpenerName@72 as OpenerName, HistoryLength@73 as HistoryLength, BrowserLanguage@74 as BrowserLanguage, BrowserCountry@75 as BrowserCountry, SocialNetwork@76 as SocialNetwork, SocialAction@77 as SocialAction, HTTPError@78 as HTTPError, SendTiming@79 as SendTiming, DNSTiming@80 as DNSTiming, ConnectTiming@81 as ConnectTiming, ResponseStartTiming@82 as ResponseStartTiming, ResponseEndTiming@83 as ResponseEndTiming, FetchTiming@84 as FetchTiming, SocialSourceNetworkID@85 as SocialSourceNetworkID, SocialSourcePage@86 as SocialSourcePage, ParamPrice@87 as ParamPrice, ParamOrderID@88 as ParamOrderID, ParamCurrency@89 as ParamCurrency, ParamCurrencyID@90 as ParamCurrencyID, OpenstatServiceName@91 as OpenstatServiceName, OpenstatCampaignID@92 as OpenstatCampaignID, OpenstatAdID@93 as OpenstatAdID, OpenstatSourceID@94 as OpenstatSourceID, UTMSource@95 as UTMSource, UTMMedium@96 as UTMMedium, UTMCampaign@97 as UTMCampaign, UTMContent@98 as UTMContent, UTMTerm@99 as UTMTerm, FromTag@100 as FromTag, HasGCLID@101 as HasGCLID, RefererHash@102 as RefererHash, URLHash@103 as URLHash, CLID@104 as CLID, CAST(CAST(EventDate@5 AS Int32) AS Date32) as EventDate] 04)------FilterExec: URL@13 LIKE %google% -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[WatchID, JavaEnable, Title, GoodEvent, EventTime, EventDate, CounterID, ClientIP, RegionID, UserID, CounterClass, OS, UserAgent, URL, Referer, IsRefresh, RefererCategoryID, RefererRegionID, URLCategoryID, URLRegionID, ResolutionWidth, ResolutionHeight, ResolutionDepth, FlashMajor, FlashMinor, FlashMinor2, NetMajor, NetMinor, UserAgentMajor, UserAgentMinor, CookieEnable, JavascriptEnable, IsMobile, MobilePhone, MobilePhoneModel, Params, IPNetworkID, TraficSourceID, SearchEngineID, SearchPhrase, AdvEngineID, IsArtifical, WindowClientWidth, WindowClientHeight, ClientTimeZone, ClientEventTime, SilverlightVersion1, SilverlightVersion2, SilverlightVersion3, SilverlightVersion4, PageCharset, CodeVersion, IsLink, IsDownload, IsNotBounce, FUniqID, OriginalURL, HID, IsOldCounter, IsEvent, IsParameter, DontCountHits, WithHash, HitColor, LocalEventTime, Age, Sex, Income, Interests, Robotness, RemoteIP, WindowName, OpenerName, HistoryLength, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction, HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, SocialSourcePage, ParamPrice, ParamOrderID, ParamCurrency, ParamCurrencyID, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash, URLHash, CLID], file_type=parquet, predicate=URL@13 LIKE %google% AND DynamicFilter [ empty ] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[WatchID, JavaEnable, Title, GoodEvent, EventTime, EventDate, CounterID, ClientIP, RegionID, UserID, CounterClass, OS, UserAgent, URL, Referer, IsRefresh, RefererCategoryID, RefererRegionID, URLCategoryID, URLRegionID, ResolutionWidth, ResolutionHeight, ResolutionDepth, FlashMajor, FlashMinor, FlashMinor2, NetMajor, NetMinor, UserAgentMajor, UserAgentMinor, CookieEnable, JavascriptEnable, IsMobile, MobilePhone, MobilePhoneModel, Params, IPNetworkID, TraficSourceID, SearchEngineID, SearchPhrase, AdvEngineID, IsArtifical, WindowClientWidth, WindowClientHeight, ClientTimeZone, ClientEventTime, SilverlightVersion1, SilverlightVersion2, SilverlightVersion3, SilverlightVersion4, PageCharset, CodeVersion, IsLink, IsDownload, IsNotBounce, FUniqID, OriginalURL, HID, IsOldCounter, IsEvent, IsParameter, DontCountHits, WithHash, HitColor, LocalEventTime, Age, Sex, Income, Interests, Robotness, RemoteIP, WindowName, OpenerName, HistoryLength, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction, HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, SocialSourcePage, ParamPrice, ParamOrderID, ParamCurrency, ParamCurrencyID, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash, URLHash, CLID], file_type=parquet, predicate=URL@13 LIKE %google% AND DynamicFilter [ empty ] query IITIIIIIIIIITTIIIIIIIIIITIIITIIIITTIIITIIIIIIIIIITIIIIITIIIIIITIIIIIIIIIITTTTIIIIIIIITITTITTTTTTTTTTIIIID SELECT * FROM hits WHERE "URL" LIKE '%google%' ORDER BY "EventTime" LIMIT 10; @@ -683,8 +671,7 @@ physical_plan 03)----SortExec: TopK(fetch=10), expr=[EventTime@1 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[SearchPhrase@1 as SearchPhrase, EventTime@0 as EventTime] 05)--------FilterExec: SearchPhrase@1 != -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +06)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query T SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime" LIMIT 10; @@ -703,8 +690,7 @@ physical_plan 01)SortPreservingMergeExec: [SearchPhrase@0 ASC NULLS LAST], fetch=10 02)--SortExec: TopK(fetch=10), expr=[SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: SearchPhrase@0 != -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query T SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "SearchPhrase" LIMIT 10; @@ -727,8 +713,7 @@ physical_plan 03)----SortExec: TopK(fetch=10), expr=[EventTime@1 ASC NULLS LAST, SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[SearchPhrase@1 as SearchPhrase, EventTime@0 as EventTime] 05)--------FilterExec: SearchPhrase@1 != -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +06)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query T SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime", "SearchPhrase" LIMIT 10; @@ -755,8 +740,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([CounterID@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] 08)--------------FilterExec: URL@1 != -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[CounterID, URL], file_type=parquet, predicate=URL@13 != , pruning_predicate=URL_null_count@2 != row_count@3 AND (URL_min@0 != OR != URL_max@1), required_guarantees=[URL not in ()] +09)----------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[CounterID, URL], file_type=parquet, predicate=URL@13 != , pruning_predicate=URL_null_count@2 != row_count@3 AND (URL_min@0 != OR != URL_max@1), required_guarantees=[URL not in ()] query IRI SELECT "CounterID", AVG(length("URL")) AS l, COUNT(*) AS c FROM hits WHERE "URL" <> '' GROUP BY "CounterID" HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; @@ -783,8 +767,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[regexp_replace(Referer@0, ^https?://(?:www\.)?([^/]+)/.*$, \1) as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] 08)--------------FilterExec: Referer@0 != -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[Referer], file_type=parquet, predicate=Referer@14 != , pruning_predicate=Referer_null_count@2 != row_count@3 AND (Referer_min@0 != OR != Referer_max@1), required_guarantees=[Referer not in ()] +09)----------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[Referer], file_type=parquet, predicate=Referer@14 != , pruning_predicate=Referer_null_count@2 != row_count@3 AND (Referer_min@0 != OR != Referer_max@1), required_guarantees=[Referer not in ()] query TRIT SELECT REGEXP_REPLACE("Referer", '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length("Referer")) AS l, COUNT(*) AS c, MIN("Referer") FROM hits WHERE "Referer" <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; @@ -801,7 +784,7 @@ logical_plan 04)------TableScan: hits_raw projection=[ResolutionWidth] physical_plan 01)AggregateExec: mode=Single, gby=[], aggr=[sum(hits.ResolutionWidth), sum(hits.ResolutionWidth + Int64(1)), sum(hits.ResolutionWidth + Int64(2)), sum(hits.ResolutionWidth + Int64(3)), sum(hits.ResolutionWidth + Int64(4)), sum(hits.ResolutionWidth + Int64(5)), sum(hits.ResolutionWidth + Int64(6)), sum(hits.ResolutionWidth + Int64(7)), sum(hits.ResolutionWidth + Int64(8)), sum(hits.ResolutionWidth + Int64(9)), sum(hits.ResolutionWidth + Int64(10)), sum(hits.ResolutionWidth + Int64(11)), sum(hits.ResolutionWidth + Int64(12)), sum(hits.ResolutionWidth + Int64(13)), sum(hits.ResolutionWidth + Int64(14)), sum(hits.ResolutionWidth + Int64(15)), sum(hits.ResolutionWidth + Int64(16)), sum(hits.ResolutionWidth + Int64(17)), sum(hits.ResolutionWidth + Int64(18)), sum(hits.ResolutionWidth + Int64(19)), sum(hits.ResolutionWidth + Int64(20)), sum(hits.ResolutionWidth + Int64(21)), sum(hits.ResolutionWidth + Int64(22)), sum(hits.ResolutionWidth + Int64(23)), sum(hits.ResolutionWidth + Int64(24)), sum(hits.ResolutionWidth + Int64(25)), sum(hits.ResolutionWidth + Int64(26)), sum(hits.ResolutionWidth + Int64(27)), sum(hits.ResolutionWidth + Int64(28)), sum(hits.ResolutionWidth + Int64(29)), sum(hits.ResolutionWidth + Int64(30)), sum(hits.ResolutionWidth + Int64(31)), sum(hits.ResolutionWidth + Int64(32)), sum(hits.ResolutionWidth + Int64(33)), sum(hits.ResolutionWidth + Int64(34)), sum(hits.ResolutionWidth + Int64(35)), sum(hits.ResolutionWidth + Int64(36)), sum(hits.ResolutionWidth + Int64(37)), sum(hits.ResolutionWidth + Int64(38)), sum(hits.ResolutionWidth + Int64(39)), sum(hits.ResolutionWidth + Int64(40)), sum(hits.ResolutionWidth + Int64(41)), sum(hits.ResolutionWidth + Int64(42)), sum(hits.ResolutionWidth + Int64(43)), sum(hits.ResolutionWidth + Int64(44)), sum(hits.ResolutionWidth + Int64(45)), sum(hits.ResolutionWidth + Int64(46)), sum(hits.ResolutionWidth + Int64(47)), sum(hits.ResolutionWidth + Int64(48)), sum(hits.ResolutionWidth + Int64(49)), sum(hits.ResolutionWidth + Int64(50)), sum(hits.ResolutionWidth + Int64(51)), sum(hits.ResolutionWidth + Int64(52)), sum(hits.ResolutionWidth + Int64(53)), sum(hits.ResolutionWidth + Int64(54)), sum(hits.ResolutionWidth + Int64(55)), sum(hits.ResolutionWidth + Int64(56)), sum(hits.ResolutionWidth + Int64(57)), sum(hits.ResolutionWidth + Int64(58)), sum(hits.ResolutionWidth + Int64(59)), sum(hits.ResolutionWidth + Int64(60)), sum(hits.ResolutionWidth + Int64(61)), sum(hits.ResolutionWidth + Int64(62)), sum(hits.ResolutionWidth + Int64(63)), sum(hits.ResolutionWidth + Int64(64)), sum(hits.ResolutionWidth + Int64(65)), sum(hits.ResolutionWidth + Int64(66)), sum(hits.ResolutionWidth + Int64(67)), sum(hits.ResolutionWidth + Int64(68)), sum(hits.ResolutionWidth + Int64(69)), sum(hits.ResolutionWidth + Int64(70)), sum(hits.ResolutionWidth + Int64(71)), sum(hits.ResolutionWidth + Int64(72)), sum(hits.ResolutionWidth + Int64(73)), sum(hits.ResolutionWidth + Int64(74)), sum(hits.ResolutionWidth + Int64(75)), sum(hits.ResolutionWidth + Int64(76)), sum(hits.ResolutionWidth + Int64(77)), sum(hits.ResolutionWidth + Int64(78)), sum(hits.ResolutionWidth + Int64(79)), sum(hits.ResolutionWidth + Int64(80)), sum(hits.ResolutionWidth + Int64(81)), sum(hits.ResolutionWidth + Int64(82)), sum(hits.ResolutionWidth + Int64(83)), sum(hits.ResolutionWidth + Int64(84)), sum(hits.ResolutionWidth + Int64(85)), sum(hits.ResolutionWidth + Int64(86)), sum(hits.ResolutionWidth + Int64(87)), sum(hits.ResolutionWidth + Int64(88)), sum(hits.ResolutionWidth + Int64(89))] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[CAST(ResolutionWidth@20 AS Int64) as __common_expr_1], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[CAST(ResolutionWidth@20 AS Int64) as __common_expr_1], file_type=parquet query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII SELECT SUM("ResolutionWidth"), SUM("ResolutionWidth" + 1), SUM("ResolutionWidth" + 2), SUM("ResolutionWidth" + 3), SUM("ResolutionWidth" + 4), SUM("ResolutionWidth" + 5), SUM("ResolutionWidth" + 6), SUM("ResolutionWidth" + 7), SUM("ResolutionWidth" + 8), SUM("ResolutionWidth" + 9), SUM("ResolutionWidth" + 10), SUM("ResolutionWidth" + 11), SUM("ResolutionWidth" + 12), SUM("ResolutionWidth" + 13), SUM("ResolutionWidth" + 14), SUM("ResolutionWidth" + 15), SUM("ResolutionWidth" + 16), SUM("ResolutionWidth" + 17), SUM("ResolutionWidth" + 18), SUM("ResolutionWidth" + 19), SUM("ResolutionWidth" + 20), SUM("ResolutionWidth" + 21), SUM("ResolutionWidth" + 22), SUM("ResolutionWidth" + 23), SUM("ResolutionWidth" + 24), SUM("ResolutionWidth" + 25), SUM("ResolutionWidth" + 26), SUM("ResolutionWidth" + 27), SUM("ResolutionWidth" + 28), SUM("ResolutionWidth" + 29), SUM("ResolutionWidth" + 30), SUM("ResolutionWidth" + 31), SUM("ResolutionWidth" + 32), SUM("ResolutionWidth" + 33), SUM("ResolutionWidth" + 34), SUM("ResolutionWidth" + 35), SUM("ResolutionWidth" + 36), SUM("ResolutionWidth" + 37), SUM("ResolutionWidth" + 38), SUM("ResolutionWidth" + 39), SUM("ResolutionWidth" + 40), SUM("ResolutionWidth" + 41), SUM("ResolutionWidth" + 42), SUM("ResolutionWidth" + 43), SUM("ResolutionWidth" + 44), SUM("ResolutionWidth" + 45), SUM("ResolutionWidth" + 46), SUM("ResolutionWidth" + 47), SUM("ResolutionWidth" + 48), SUM("ResolutionWidth" + 49), SUM("ResolutionWidth" + 50), SUM("ResolutionWidth" + 51), SUM("ResolutionWidth" + 52), SUM("ResolutionWidth" + 53), SUM("ResolutionWidth" + 54), SUM("ResolutionWidth" + 55), SUM("ResolutionWidth" + 56), SUM("ResolutionWidth" + 57), SUM("ResolutionWidth" + 58), SUM("ResolutionWidth" + 59), SUM("ResolutionWidth" + 60), SUM("ResolutionWidth" + 61), SUM("ResolutionWidth" + 62), SUM("ResolutionWidth" + 63), SUM("ResolutionWidth" + 64), SUM("ResolutionWidth" + 65), SUM("ResolutionWidth" + 66), SUM("ResolutionWidth" + 67), SUM("ResolutionWidth" + 68), SUM("ResolutionWidth" + 69), SUM("ResolutionWidth" + 70), SUM("ResolutionWidth" + 71), SUM("ResolutionWidth" + 72), SUM("ResolutionWidth" + 73), SUM("ResolutionWidth" + 74), SUM("ResolutionWidth" + 75), SUM("ResolutionWidth" + 76), SUM("ResolutionWidth" + 77), SUM("ResolutionWidth" + 78), SUM("ResolutionWidth" + 79), SUM("ResolutionWidth" + 80), SUM("ResolutionWidth" + 81), SUM("ResolutionWidth" + 82), SUM("ResolutionWidth" + 83), SUM("ResolutionWidth" + 84), SUM("ResolutionWidth" + 85), SUM("ResolutionWidth" + 86), SUM("ResolutionWidth" + 87), SUM("ResolutionWidth" + 88), SUM("ResolutionWidth" + 89) FROM hits; @@ -828,8 +811,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([SearchEngineID@0, ClientIP@1], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@3 as SearchEngineID, ClientIP@0 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] 07)------------FilterExec: SearchPhrase@4 != , projection=[ClientIP@0, IsRefresh@1, ResolutionWidth@2, SearchEngineID@3] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[ClientIP, IsRefresh, ResolutionWidth, SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[ClientIP, IsRefresh, ResolutionWidth, SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query IIIIR SELECT "SearchEngineID", "ClientIP", COUNT(*) AS c, SUM("IsRefresh"), AVG("ResolutionWidth") FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "ClientIP" ORDER BY c DESC LIMIT 10; @@ -855,8 +837,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([WatchID@0, ClientIP@1], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] 07)------------FilterExec: SearchPhrase@4 != , projection=[WatchID@0, ClientIP@1, IsRefresh@2, ResolutionWidth@3] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query IIIIR SELECT "WatchID", "ClientIP", COUNT(*) AS c, SUM("IsRefresh"), AVG("ResolutionWidth") FROM hits WHERE "SearchPhrase" <> '' GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; @@ -879,7 +860,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] 05)--------RepartitionExec: partitioning=Hash([WatchID@0, ClientIP@1], 4), input_partitions=1 06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth], file_type=parquet +07)------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth], file_type=parquet query IIIIR rowsort SELECT "WatchID", "ClientIP", COUNT(*) AS c, SUM("IsRefresh"), AVG("ResolutionWidth") FROM hits GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; @@ -912,7 +893,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=1 06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL], file_type=parquet +07)------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[URL], file_type=parquet query TI rowsort SELECT "URL", COUNT(*) AS c FROM hits GROUP BY "URL" ORDER BY c DESC LIMIT 10; @@ -941,7 +922,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=1 06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL], file_type=parquet +07)------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[URL], file_type=parquet query ITI rowsort SELECT 1, "URL", COUNT(*) AS c FROM hits GROUP BY 1, "URL" ORDER BY c DESC LIMIT 10; @@ -971,7 +952,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[ClientIP@0 as ClientIP, hits.ClientIP - Int64(1)@1 as hits.ClientIP - Int64(1), hits.ClientIP - Int64(2)@2 as hits.ClientIP - Int64(2), hits.ClientIP - Int64(3)@3 as hits.ClientIP - Int64(3)], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([ClientIP@0, hits.ClientIP - Int64(1)@1, hits.ClientIP - Int64(2)@2, hits.ClientIP - Int64(3)@3], 4), input_partitions=1 06)----------AggregateExec: mode=Partial, gby=[ClientIP@1 as ClientIP, __common_expr_1@0 - 1 as hits.ClientIP - Int64(1), __common_expr_1@0 - 2 as hits.ClientIP - Int64(2), __common_expr_1@0 - 3 as hits.ClientIP - Int64(3)], aggr=[count(Int64(1))] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[CAST(ClientIP@7 AS Int64) as __common_expr_1, ClientIP], file_type=parquet +07)------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[CAST(ClientIP@7 AS Int64) as __common_expr_1, ClientIP], file_type=parquet query IIIII rowsort SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, COUNT(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; @@ -1001,8 +982,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] 07)------------FilterExec: CounterID@1 = 62 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND URL@2 != , projection=[URL@2] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, URL, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND URL@13 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND IsRefresh_null_count@12 != row_count@3 AND IsRefresh_min@10 <= 0 AND 0 <= IsRefresh_max@11 AND URL_null_count@15 != row_count@3 AND (URL_min@13 != OR != URL_max@14), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URL not in ()] +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[EventDate, CounterID, URL, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND URL@13 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND IsRefresh_null_count@12 != row_count@3 AND IsRefresh_min@10 <= 0 AND 0 <= IsRefresh_max@11 AND URL_null_count@15 != row_count@3 AND (URL_min@13 != OR != URL_max@14), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URL not in ()] query TI SELECT "URL", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "IsRefresh" = 0 AND "URL" <> '' GROUP BY "URL" ORDER BY PageViews DESC LIMIT 10; @@ -1028,8 +1008,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([Title@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[Title@0 as Title], aggr=[count(Int64(1))] 07)------------FilterExec: CounterID@2 = 62 AND CAST(CAST(EventDate@1 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@1 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND Title@0 != , projection=[Title@0] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[Title, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND Title@2 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND IsRefresh_null_count@12 != row_count@3 AND IsRefresh_min@10 <= 0 AND 0 <= IsRefresh_max@11 AND Title_null_count@15 != row_count@3 AND (Title_min@13 != OR != Title_max@14), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), Title not in ()] +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[Title, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND Title@2 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND IsRefresh_null_count@12 != row_count@3 AND IsRefresh_min@10 <= 0 AND 0 <= IsRefresh_max@11 AND Title_null_count@15 != row_count@3 AND (Title_min@13 != OR != Title_max@14), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), Title not in ()] query TI SELECT "Title", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "IsRefresh" = 0 AND "Title" <> '' GROUP BY "Title" ORDER BY PageViews DESC LIMIT 10; @@ -1057,8 +1036,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] 08)--------------FilterExec: CounterID@1 = 62 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@3 = 0 AND IsLink@4 != 0 AND IsDownload@5 = 0, projection=[URL@2] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, URL, IsRefresh, IsLink, IsDownload], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@15 = 0 AND IsLink@52 != 0 AND IsDownload@53 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND IsLink_null_count@12 != row_count@3 AND (IsLink_min@10 != 0 OR 0 != IsLink_max@11) AND IsDownload_null_count@15 != row_count@3 AND IsDownload_min@13 <= 0 AND 0 <= IsDownload_max@14, required_guarantees=[CounterID in (62), IsDownload in (0), IsLink not in (0), IsRefresh in (0)] +09)----------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[EventDate, CounterID, URL, IsRefresh, IsLink, IsDownload], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@15 = 0 AND IsLink@52 != 0 AND IsDownload@53 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND IsLink_null_count@12 != row_count@3 AND (IsLink_min@10 != 0 OR 0 != IsLink_max@11) AND IsDownload_null_count@15 != row_count@3 AND IsDownload_min@13 <= 0 AND 0 <= IsDownload_max@14, required_guarantees=[CounterID in (62), IsDownload in (0), IsLink not in (0), IsRefresh in (0)] query TI SELECT "URL", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "IsRefresh" = 0 AND "IsLink" <> 0 AND "IsDownload" = 0 GROUP BY "URL" ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; @@ -1086,8 +1064,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([TraficSourceID@0, SearchEngineID@1, AdvEngineID@2, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3, URL@4], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[TraficSourceID@2 as TraficSourceID, SearchEngineID@3 as SearchEngineID, AdvEngineID@4 as AdvEngineID, CASE WHEN SearchEngineID@3 = 0 AND AdvEngineID@4 = 0 THEN Referer@1 ELSE END as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@0 as URL], aggr=[count(Int64(1))] 08)--------------FilterExec: CounterID@1 = 62 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@4 = 0, projection=[URL@2, Referer@3, TraficSourceID@5, SearchEngineID@6, AdvEngineID@7] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, URL, Referer, IsRefresh, TraficSourceID, SearchEngineID, AdvEngineID], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@15 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8, required_guarantees=[CounterID in (62), IsRefresh in (0)] +09)----------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[EventDate, CounterID, URL, Referer, IsRefresh, TraficSourceID, SearchEngineID, AdvEngineID], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@15 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8, required_guarantees=[CounterID in (62), IsRefresh in (0)] query IIITTI SELECT "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END AS Src, "URL" AS Dst, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "IsRefresh" = 0 GROUP BY "TraficSourceID", "SearchEngineID", "AdvEngineID", Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; @@ -1116,8 +1093,7 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] 08)--------------ProjectionExec: expr=[URLHash@1 as URLHash, CAST(CAST(EventDate@0 AS Int32) AS Date32) as EventDate] 09)----------------FilterExec: CounterID@1 = 62 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, projection=[EventDate@0, URLHash@5] -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, IsRefresh, TraficSourceID, RefererHash, URLHash], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@15 = 0 AND (TraficSourceID@37 = -1 OR TraficSourceID@37 = 6) AND RefererHash@102 = 3594120000172545465, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND (TraficSourceID_null_count@12 != row_count@3 AND TraficSourceID_min@10 <= -1 AND -1 <= TraficSourceID_max@11 OR TraficSourceID_null_count@12 != row_count@3 AND TraficSourceID_min@10 <= 6 AND 6 <= TraficSourceID_max@11) AND RefererHash_null_count@15 != row_count@3 AND RefererHash_min@13 <= 3594120000172545465 AND 3594120000172545465 <= RefererHash_max@14, required_guarantees=[CounterID in (62), IsRefresh in (0), RefererHash in (3594120000172545465), TraficSourceID in (-1, 6)] +10)------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[EventDate, CounterID, IsRefresh, TraficSourceID, RefererHash, URLHash], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@15 = 0 AND (TraficSourceID@37 = -1 OR TraficSourceID@37 = 6) AND RefererHash@102 = 3594120000172545465, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND (TraficSourceID_null_count@12 != row_count@3 AND TraficSourceID_min@10 <= -1 AND -1 <= TraficSourceID_max@11 OR TraficSourceID_null_count@12 != row_count@3 AND TraficSourceID_min@10 <= 6 AND 6 <= TraficSourceID_max@11) AND RefererHash_null_count@15 != row_count@3 AND RefererHash_min@13 <= 3594120000172545465 AND 3594120000172545465 <= RefererHash_max@14, required_guarantees=[CounterID in (62), IsRefresh in (0), RefererHash in (3594120000172545465), TraficSourceID in (-1, 6)] query IDI SELECT "URLHash", "EventDate", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "IsRefresh" = 0 AND "TraficSourceID" IN (-1, 6) AND "RefererHash" = 3594120000172545465 GROUP BY "URLHash", "EventDate" ORDER BY PageViews DESC LIMIT 10 OFFSET 100; @@ -1145,8 +1121,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([WindowClientWidth@0, WindowClientHeight@1], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] 08)--------------FilterExec: CounterID@1 = 62 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@2 = 0 AND DontCountHits@5 = 0 AND URLHash@6 = 2868770270353813622, projection=[WindowClientWidth@3, WindowClientHeight@4] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, IsRefresh, WindowClientWidth, WindowClientHeight, DontCountHits, URLHash], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0 AND URLHash@103 = 2868770270353813622, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND DontCountHits_null_count@12 != row_count@3 AND DontCountHits_min@10 <= 0 AND 0 <= DontCountHits_max@11 AND URLHash_null_count@15 != row_count@3 AND URLHash_min@13 <= 2868770270353813622 AND 2868770270353813622 <= URLHash_max@14, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URLHash in (2868770270353813622)] +09)----------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[EventDate, CounterID, IsRefresh, WindowClientWidth, WindowClientHeight, DontCountHits, URLHash], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0 AND URLHash@103 = 2868770270353813622, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND DontCountHits_null_count@12 != row_count@3 AND DontCountHits_min@10 <= 0 AND 0 <= DontCountHits_max@11 AND URLHash_null_count@15 != row_count@3 AND URLHash_min@13 <= 2868770270353813622 AND 2868770270353813622 <= URLHash_max@14, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URLHash in (2868770270353813622)] query III SELECT "WindowClientWidth", "WindowClientHeight", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "IsRefresh" = 0 AND "DontCountHits" = 0 AND "URLHash" = 2868770270353813622 GROUP BY "WindowClientWidth", "WindowClientHeight" ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; @@ -1174,8 +1149,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[date_trunc(minute, to_timestamp_seconds(EventTime@0)) as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))] 08)--------------FilterExec: CounterID@2 = 62 AND CAST(CAST(EventDate@1 AS Int32) AS Date32) >= 2013-07-14 AND CAST(CAST(EventDate@1 AS Int32) AS Date32) <= 2013-07-15 AND IsRefresh@3 = 0 AND DontCountHits@4 = 0, projection=[EventTime@0] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-14 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-15 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-14 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-15 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND DontCountHits_null_count@12 != row_count@3 AND DontCountHits_min@10 <= 0 AND 0 <= DontCountHits_max@11, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0)] +09)----------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[EventTime, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-14 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-15 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-14 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-15 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND DontCountHits_null_count@12 != row_count@3 AND DontCountHits_min@10 <= 0 AND 0 <= DontCountHits_max@11, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0)] query PI SELECT DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) AS M, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-14' AND "EventDate" <= '2013-07-15' AND "IsRefresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) ORDER BY DATE_TRUNC('minute', M) LIMIT 10 OFFSET 1000; @@ -1192,7 +1166,7 @@ logical_plan 03)----TableScan: hits_raw projection=[MobilePhone, MobilePhoneModel, SearchPhrase] physical_plan 01)AggregateExec: mode=Single, gby=[], aggr=[count(DISTINCT hits.SearchPhrase), count(DISTINCT hits.MobilePhone), count(DISTINCT hits.MobilePhoneModel)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[MobilePhone, MobilePhoneModel, SearchPhrase], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[MobilePhone, MobilePhoneModel, SearchPhrase], file_type=parquet query III SELECT COUNT(DISTINCT "SearchPhrase"), COUNT(DISTINCT "MobilePhone"), COUNT(DISTINCT "MobilePhoneModel") FROM hits; @@ -1208,7 +1182,7 @@ logical_plan 03)----TableScan: hits_raw projection=[HitColor, BrowserLanguage, BrowserCountry] physical_plan 01)AggregateExec: mode=Single, gby=[], aggr=[count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserCountry), count(DISTINCT hits.BrowserLanguage)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[HitColor, BrowserLanguage, BrowserCountry], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[HitColor, BrowserLanguage, BrowserCountry], file_type=parquet query III SELECT COUNT(DISTINCT "HitColor"), COUNT(DISTINCT "BrowserCountry"), COUNT(DISTINCT "BrowserLanguage") FROM hits; @@ -1229,7 +1203,7 @@ physical_plan 03)----AggregateExec: mode=FinalPartitioned, gby=[BrowserCountry@0 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)] 04)------RepartitionExec: partitioning=Hash([BrowserCountry@0], 4), input_partitions=1 05)--------AggregateExec: mode=Partial, gby=[BrowserCountry@2 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[HitColor, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction], file_type=parquet +06)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet], projection=[HitColor, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction], file_type=parquet query TIIII SELECT "BrowserCountry", COUNT(DISTINCT "SocialNetwork"), COUNT(DISTINCT "HitColor"), COUNT(DISTINCT "BrowserLanguage"), COUNT(DISTINCT "SocialAction") FROM hits GROUP BY 1 ORDER BY 2 DESC LIMIT 10; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 3731a59b85549..62296c5d87f2b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -6068,7 +6068,7 @@ FROM ( # `WHERE acctbal > ( SELECT AVG(acctbal) FROM suppliers)` into a Join, # breaking the input schema passed to the window function above. # See: https://github.com/apache/datafusion/issues/17770 -query error +query I WITH suppliers AS ( SELECT * FROM (VALUES (1, 10.0), (1, 20.0)) AS t(nation, acctbal) @@ -6080,6 +6080,4 @@ WHERE acctbal > ( SELECT AVG(acctbal) FROM suppliers ); ---- -DataFusion error: Join Error -caused by -External error: task 18295 panicked with message "assertion `left == right` failed\n left: 4\n right: 1" +1 From b0515bb9e7d178b0485787ead71b95addfd977c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sat, 7 Mar 2026 18:09:36 +0100 Subject: [PATCH 76/78] Add morsel_driven field to FileScanExecConf proto The morsel_driven flag controls display format (files= vs file_groups=) and was lost during proto roundtrip, causing test failures. Co-Authored-By: Claude Opus 4.6 --- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 18 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 2 ++ .../proto/src/physical_plan/from_proto.rs | 1 + datafusion/proto/src/physical_plan/to_proto.rs | 1 + 5 files changed, 23 insertions(+) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index e64bcdb41eaf8..e29595e423624 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1075,6 +1075,7 @@ message FileScanExecConf { optional uint64 batch_size = 12; optional ProjectionExprs projection_exprs = 13; + bool morsel_driven = 14; } message ParquetScanExecNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 15b9ba88f4038..6a17f0613c3cd 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -6434,6 +6434,9 @@ impl serde::Serialize for FileScanExecConf { if self.projection_exprs.is_some() { len += 1; } + if self.morsel_driven { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.FileScanExecConf", len)?; if !self.file_groups.is_empty() { struct_ser.serialize_field("fileGroups", &self.file_groups)?; @@ -6470,6 +6473,9 @@ impl serde::Serialize for FileScanExecConf { if let Some(v) = self.projection_exprs.as_ref() { struct_ser.serialize_field("projectionExprs", v)?; } + if self.morsel_driven { + struct_ser.serialize_field("morselDriven", &self.morsel_driven)?; + } struct_ser.end() } } @@ -6497,6 +6503,8 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { "batchSize", "projection_exprs", "projectionExprs", + "morsel_driven", + "morselDriven", ]; #[allow(clippy::enum_variant_names)] @@ -6512,6 +6520,7 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { Constraints, BatchSize, ProjectionExprs, + MorselDriven, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -6544,6 +6553,7 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { "constraints" => Ok(GeneratedField::Constraints), "batchSize" | "batch_size" => Ok(GeneratedField::BatchSize), "projectionExprs" | "projection_exprs" => Ok(GeneratedField::ProjectionExprs), + "morselDriven" | "morsel_driven" => Ok(GeneratedField::MorselDriven), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -6574,6 +6584,7 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { let mut constraints__ = None; let mut batch_size__ = None; let mut projection_exprs__ = None; + let mut morsel_driven__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::FileGroups => { @@ -6647,6 +6658,12 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { } projection_exprs__ = map_.next_value()?; } + GeneratedField::MorselDriven => { + if morsel_driven__.is_some() { + return Err(serde::de::Error::duplicate_field("morselDriven")); + } + morsel_driven__ = Some(map_.next_value()?); + } } } Ok(FileScanExecConf { @@ -6661,6 +6678,7 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { constraints: constraints__, batch_size: batch_size__, projection_exprs: projection_exprs__, + morsel_driven: morsel_driven__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 48ad8dcfeabcc..db45787c5b9a4 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1630,6 +1630,8 @@ pub struct FileScanExecConf { pub batch_size: ::core::option::Option, #[prost(message, optional, tag = "13")] pub projection_exprs: ::core::option::Option, + #[prost(bool, tag = "14")] + pub morsel_driven: bool, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct ParquetScanExecNode { diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index e424be162648b..060051c50b586 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -690,6 +690,7 @@ pub fn parse_protobuf_file_scan_config( .with_limit(proto.limit.as_ref().map(|sl| sl.limit as usize)) .with_output_ordering(output_ordering) .with_batch_size(proto.batch_size.map(|s| s as usize)) + .with_morsel_driven(proto.morsel_driven) .build(); Ok(config) } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index de2f36e81e3ba..172cf67921a7b 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -700,6 +700,7 @@ pub fn serialize_file_scan_config( constraints: Some(conf.constraints.clone().into()), batch_size: conf.batch_size.map(|s| s as u64), projection_exprs, + morsel_driven: conf.morsel_driven, }) } From 8c59da966dae4e09e73be514c6ce584b62d8d3d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sat, 7 Mar 2026 19:56:38 +0100 Subject: [PATCH 77/78] Fix test --- datafusion/core/src/datasource/physical_plan/parquet.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 55dc8805a437e..ebe28ea8a1f2b 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -1341,7 +1341,6 @@ mod tests { let filename = "int96_from_spark.parquet"; let session_ctx = SessionContext::new(); let state = session_ctx.state(); - let task_ctx = state.task_ctx(); let time_units_and_expected = vec![ ( @@ -1393,7 +1392,8 @@ mod tests { .unwrap(); assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); - let mut results = parquet_exec.execute(0, task_ctx.clone())?; + let task_ctx = state.task_ctx(); + let mut results = parquet_exec.execute(0, task_ctx)?; let batch = results.next().await.unwrap()?; assert_eq!(6, batch.num_rows()); From b7fdeaa3755720f0a769286284ce2bb299627b6d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Sat, 7 Mar 2026 20:26:46 +0100 Subject: [PATCH 78/78] Fix test --- datafusion/substrait/src/physical_plan/consumer.rs | 12 ++++++++++-- .../substrait/tests/cases/roundtrip_physical_plan.rs | 1 + 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index ccaf1abec4245..19923548be0d1 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -135,8 +135,16 @@ pub async fn from_substrait_rel( file_groups[part_index].push(partitioned_file) } - base_config_builder = - base_config_builder.with_file_groups(file_groups); + let morsel_driven = _ctx + .state() + .config() + .options() + .execution + .parquet + .allow_morsel_driven; + base_config_builder = base_config_builder + .with_file_groups(file_groups) + .with_morsel_driven(morsel_driven); if let Some(MaskExpression { select, .. }) = &read.projection && let Some(projection) = &select.as_ref() diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 9773cf4aba10f..fe199040915ee 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -63,6 +63,7 @@ async fn parquet_exec() -> Result<()> { producer::to_substrait_rel(parquet_exec.as_ref(), &mut extension_info)?; let ctx = SessionContext::new(); + ctx.state_ref().write().config_mut().options_mut().execution.parquet.allow_morsel_driven = false; let parquet_exec_roundtrip = consumer::from_substrait_rel(&ctx, substrait_rel.as_ref(), &HashMap::new())