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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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 e9d23911babe07d23068a8e05b8605ed329ca775 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 27 Feb 2026 16:27:27 +0100 Subject: [PATCH 46/53] Tryout different approach --- .../custom_data_source/csv_json_opener.rs | 2 - .../custom_data_source/custom_datasource.rs | 4 +- .../examples/data_io/json_shredding.rs | 1 - .../src/datasource/physical_plan/parquet.rs | 147 --------- .../partition_statistics.rs | 3 +- .../datasource-parquet/src/file_format.rs | 50 ++- datafusion/datasource/src/file_scan_config.rs | 47 +-- datafusion/datasource/src/file_stream.rs | 285 +----------------- datafusion/datasource/src/source.rs | 70 +---- datafusion/physical-plan/src/lib.rs | 1 + .../sqllogictest/test_files/parquet.slt | 4 +- .../test_files/parquet_sorted_statistics.slt | 25 +- .../test_files/preserve_file_partitioning.slt | 10 +- .../test_files/projection_pushdown.slt | 23 +- .../push_down_filter_regression.slt | 12 +- .../repartition_subset_satisfaction.slt | 37 +-- .../sqllogictest/test_files/sort_pushdown.slt | 33 +- 17 files changed, 158 insertions(+), 596 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-examples/examples/custom_data_source/custom_datasource.rs b/datafusion-examples/examples/custom_data_source/custom_datasource.rs index 73ce6f5cf2002..f4a53d49f7cdd 100644 --- a/datafusion-examples/examples/custom_data_source/custom_datasource.rs +++ b/datafusion-examples/examples/custom_data_source/custom_datasource.rs @@ -62,9 +62,7 @@ async fn search_accounts( 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); + 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 1040b7d3df04e..77dba5a98ac6f 100644 --- a/datafusion-examples/examples/data_io/json_shredding.rs +++ b/datafusion-examples/examples/data_io/json_shredding.rs @@ -93,7 +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; - 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/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index bee11c06ca073..1d387b37d19d0 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -2463,151 +2463,4 @@ mod tests { 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 = SpawnedTask::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 = SpawnedTask::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/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index b04090f0dc813..fd278d147edbc 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -78,8 +78,7 @@ mod test { target_partition: Option, ) -> Arc { let mut session_config = SessionConfig::new() - .with_collect_statistics(true) - .set_bool("datafusion.execution.parquet.allow_morsel_driven", false); + .with_collect_statistics(true); if let Some(partition) = target_partition { session_config = session_config.with_target_partitions(partition); } diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 584a84b05352f..327687dacaada 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -48,6 +48,7 @@ use datafusion_common::{ use datafusion_common::{HashMap, Statistics}; use datafusion_common_runtime::{JoinSet, SpawnedTask}; use datafusion_datasource::display::FileGroupDisplay; +use datafusion_datasource::file_groups::FileGroup; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; use datafusion_datasource::sink::{DataSink, DataSinkExec}; @@ -55,6 +56,7 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReserv use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_plan::merge_partitions::MergePartitionsExec; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use datafusion_session::Session; @@ -532,11 +534,49 @@ impl FileFormat for ParquetFormat { source = self.set_source_encryption_factory(source, state)?; - 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)) + let use_merge_partitions = self.options.global.allow_morsel_driven; + + if use_merge_partitions { + // Create one partition per file for maximum parallelism, + // then wrap with MergePartitionsExec to reduce to target_partitions. + // This uses atomic work-stealing at the partition level. + let one_per_file: Vec = conf + .file_groups + .iter() + .flat_map(|group| { + group + .iter() + .map(|file| FileGroup::new(vec![file.clone()])) + }) + .collect(); + + let target_partitions = + state.config_options().execution.target_partitions; + + let conf = FileScanConfigBuilder::from(conf) + .with_source(Arc::new(source)) + .with_file_groups(one_per_file) + .build(); + let data_source = DataSourceExec::from_data_source(conf); + let input_partitions = data_source + .properties() + .output_partitioning() + .partition_count(); + + if input_partitions > target_partitions { + Ok(Arc::new(MergePartitionsExec::new( + data_source, + target_partitions, + ))) + } else { + Ok(data_source) + } + } else { + let conf = FileScanConfigBuilder::from(conf) + .with_source(Arc::new(source)) + .build(); + Ok(DataSourceExec::from_data_source(conf)) + } } async fn create_writer_physical_plan( diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index b89efb0696fa9..c15b25869ca43 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -24,7 +24,7 @@ use crate::{ display::FileGroupsDisplay, file::FileSource, file_compression_type::FileCompressionType, - file_stream::{FileStream, WorkQueue}, + file_stream::FileStream, source::DataSource, statistics::MinMaxStatistics, }; @@ -208,11 +208,6 @@ 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 morsel queue, set via [`DataSource::with_shared_morsel_queue`]. - shared_morsel_queue: Option>, } /// A builder for [`FileScanConfig`]'s. @@ -283,7 +278,6 @@ pub struct FileScanConfigBuilder { batch_size: Option, expr_adapter_factory: Option>, partitioned_by_file_group: bool, - morsel_driven: bool, } impl FileScanConfigBuilder { @@ -310,7 +304,6 @@ impl FileScanConfigBuilder { batch_size: None, expr_adapter_factory: None, partitioned_by_file_group: false, - morsel_driven: false, } } @@ -511,12 +504,6 @@ 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`. @@ -538,7 +525,6 @@ impl FileScanConfigBuilder { batch_size, expr_adapter_factory: expr_adapter, partitioned_by_file_group, - morsel_driven, } = self; let constraints = constraints.unwrap_or_default(); @@ -551,24 +537,6 @@ 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, 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, file_source, @@ -582,8 +550,6 @@ impl FileScanConfigBuilder { expr_adapter_factory: expr_adapter, statistics, partitioned_by_file_group, - morsel_driven, - shared_morsel_queue: None, } } } @@ -603,7 +569,6 @@ 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, } } } @@ -628,20 +593,10 @@ impl DataSource for FileScanConfig { partition, opener, source.metrics(), - 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/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 03ca526d469ae..1b8387483a908 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -24,10 +24,8 @@ use std::collections::VecDeque; use std::mem; use std::pin::Pin; -use std::sync::atomic::{AtomicUsize, Ordering}; -use std::sync::{Arc, Mutex}; +use std::sync::Arc; use std::task::{Context, Poll}; -use tokio::sync::Notify; use crate::PartitionedFile; use crate::file_scan_config::FileScanConfig; @@ -45,25 +43,10 @@ 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, @@ -80,8 +63,6 @@ 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 { @@ -91,21 +72,14 @@ 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) - } else { - let file_group = config.file_groups[partition].clone(); - (file_group.into_inner().into_iter().collect(), None) - }; + let file_group = config.file_groups[partition].clone(); + let file_iter = file_group.into_inner().into_iter().collect(); Ok(Self { file_iter, - shared_queue, - morsel_driven: config.morsel_driven, projected_schema, remain: config.limit, file_opener, @@ -113,7 +87,6 @@ impl FileStream { file_stream_metrics: FileStreamMetrics::new(metrics, partition), baseline_metrics: BaselineMetrics::new(metrics, partition), on_error: OnError::Fail, - morsel_guard: None, }) } @@ -130,16 +103,7 @@ 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 { - // In morsel-driven don't "prefetch" - return None; - } let part_file = self.file_iter.pop_front()?; Some(self.file_opener.open(part_file)) } @@ -150,123 +114,17 @@ 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"); - 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), - }; - } - } - 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 { - // 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(); - 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; - } + match self.start_next_file().transpose() { + Ok(Some(future)) => { + self.state = FileStreamState::Open { future } } + Ok(None) => return Poll::Ready(None), 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` @@ -360,11 +218,7 @@ impl FileStream { } } None => { - if self.morsel_driven { - self.state = FileStreamState::Idle; - } else { - return Poll::Ready(None); - } + return Poll::Ready(None); } }, OnError::Fail => { @@ -395,11 +249,7 @@ impl FileStream { } } None => { - if self.morsel_driven { - self.state = FileStreamState::Idle; - } else { - return Poll::Ready(None); - } + return Poll::Ready(None); } } } @@ -433,111 +283,6 @@ 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 - } - } - - /// 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 - } - - /// 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. 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) { - 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 - } - - /// 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>>>; @@ -602,16 +347,6 @@ 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 { @@ -881,7 +616,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); diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 82aadfa7eb786..c75dd228687ac 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, Mutex}; +use std::sync::Arc; use datafusion_physical_expr::projection::ProjectionExprs; use datafusion_physical_plan::execution_plan::{ @@ -36,7 +36,6 @@ 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}; @@ -127,17 +126,6 @@ pub trait DataSource: Send + Sync + Debug { context: Arc, ) -> 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; @@ -244,15 +232,6 @@ 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, } impl DisplayAs for DataSourceExec { @@ -322,49 +301,7 @@ 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. - 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 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 stream = self.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}" @@ -393,7 +330,6 @@ impl ExecutionPlan for DataSourceExec { Some(Arc::new(Self { data_source, cache, - morsel_state: Arc::new(Mutex::new(MorselState::default())), })) } @@ -488,7 +424,6 @@ impl DataSourceExec { Self { data_source, cache: Arc::new(cache), - morsel_state: Arc::new(Mutex::new(MorselState::default())), } } @@ -500,7 +435,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_state = Arc::new(Mutex::new(MorselState::default())); self } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 6467d7a2e389d..c38d4649b5809 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -79,6 +79,7 @@ pub mod filter_pushdown; pub mod joins; pub mod limit; pub mod memory; +pub mod merge_partitions; pub mod metrics; pub mod placeholder_row; pub mod projection; diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index be713b963b451..63a9e2395ced9 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -145,7 +145,9 @@ logical_plan 02)--TableScan: test_table projection=[int_col, string_col] physical_plan 01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] -02)--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], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/2.parquet]]}, projection=[int_col, string_col], output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], file_type=parquet +02)--SortExec: expr=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----MergePartitionsExec: partitions=3→2 +04)------DataSourceExec: file_groups={3 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], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/2.parquet]]}, projection=[int_col, string_col], output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], file_type=parquet # Perform queries using MIN and MAX diff --git a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt index fd3a40ca17079..1c2b2a0243d62 100644 --- a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt @@ -120,7 +120,9 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col, nulls_first_col, nulls_last_col] physical_plan 01)SortPreservingMergeExec: [int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST, nulls_first_col@2 ASC, nulls_last_col@3 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, nulls_first_col, nulls_last_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST, nulls_first_col@2 ASC, nulls_last_col@3 ASC NULLS LAST], file_type=parquet +02)--SortExec: expr=[int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST, nulls_first_col@2 ASC, nulls_last_col@3 ASC NULLS LAST], preserve_partitioning=[true] +03)----MergePartitionsExec: partitions=3→2 +04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, nulls_first_col, nulls_last_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST, nulls_first_col@2 ASC, nulls_last_col@3 ASC NULLS LAST], file_type=parquet # Another planning test, but project on a column with unsupported statistics # We should be able to ignore this and look at only the relevant statistics @@ -137,7 +139,10 @@ logical_plan physical_plan 01)ProjectionExec: expr=[string_col@0 as string_col] 02)--SortPreservingMergeExec: [int_col@1 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST, nulls_first_col@3 ASC, nulls_last_col@4 ASC NULLS LAST] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[string_col, int_col, bigint_col, nulls_first_col, nulls_last_col], output_ordering=[int_col@1 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST, nulls_first_col@3 ASC, nulls_last_col@4 ASC NULLS LAST], file_type=parquet +03)----SortExec: expr=[int_col@1 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST, nulls_first_col@3 ASC, nulls_last_col@4 ASC NULLS LAST], preserve_partitioning=[true] +04)------ProjectionExec: expr=[string_col@1 as string_col, int_col@0 as int_col, bigint_col@2 as bigint_col, nulls_first_col@3 as nulls_first_col, nulls_last_col@4 as nulls_last_col] +05)--------MergePartitionsExec: partitions=3→2 +06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, string_col, bigint_col, nulls_first_col, nulls_last_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST, nulls_first_col@3 ASC, nulls_last_col@4 ASC NULLS LAST], file_type=parquet # Clean up & recreate but sort on descending column statement ok @@ -171,7 +176,9 @@ logical_plan 02)--TableScan: test_table projection=[descending_col, bigint_col] physical_plan 01)SortPreservingMergeExec: [descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[descending_col, bigint_col], output_ordering=[descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet +02)--SortExec: expr=[descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST], preserve_partitioning=[true] +03)----MergePartitionsExec: partitions=3→2 +04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[descending_col, bigint_col], output_ordering=[descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Clean up & re-create with partition columns in sort order statement ok @@ -207,7 +214,9 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col, partition_col] physical_plan 01)SortPreservingMergeExec: [partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, partition_col], output_ordering=[partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet +02)--SortExec: expr=[partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], preserve_partitioning=[true] +03)----MergePartitionsExec: partitions=3→2 +04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, partition_col], output_ordering=[partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Clean up & re-create with overlapping column in sort order # This will test the ability to sort files with overlapping statistics @@ -240,7 +249,9 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col, overlapping_col] physical_plan 01)SortPreservingMergeExec: [overlapping_col@2 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, overlapping_col], output_ordering=[overlapping_col@2 ASC NULLS LAST], file_type=parquet +02)--SortExec: expr=[overlapping_col@2 ASC NULLS LAST], preserve_partitioning=[true] +03)----MergePartitionsExec: partitions=3→2 +04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, overlapping_col], output_ordering=[overlapping_col@2 ASC NULLS LAST], file_type=parquet # Clean up & re-create with constant column in sort order # This will require a sort because the # of required file groups (3) @@ -274,4 +285,6 @@ logical_plan 02)--TableScan: test_table projection=[constant_col] physical_plan 01)SortPreservingMergeExec: [constant_col@0 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col], output_ordering=[constant_col@0 ASC NULLS LAST], file_type=parquet +02)--SortExec: expr=[constant_col@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----MergePartitionsExec: partitions=3→2 +04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col], output_ordering=[constant_col@0 ASC NULLS LAST], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 297094fab16e7..f13c4335a72c7 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -551,8 +551,11 @@ logical_plan 03)----TableScan: high_cardinality_table projection=[value, f_dkey] physical_plan 01)ProjectionExec: expr=[f_dkey@0 as f_dkey, count(Int64(1))@1 as count(*), sum(high_cardinality_table.value)@2 as sum(high_cardinality_table.value)] -02)--AggregateExec: mode=SinglePartitioned, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)] -03)----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=D/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=E/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet +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)--------MergePartitionsExec: partitions=5→3 +06)----------DataSourceExec: file_groups={5 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=D/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=E/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet # Verify results with optimization match results without optimization query TIR rowsort @@ -596,7 +599,8 @@ 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)--------MergePartitionsExec: partitions=5→3 +06)----------DataSourceExec: file_groups={5 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 query TIR rowsort SELECT f_dkey, count(*), sum(value) diff --git a/datafusion/sqllogictest/test_files/projection_pushdown.slt b/datafusion/sqllogictest/test_files/projection_pushdown.slt index dbb77b33c21b7..8f160537a7d56 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -684,7 +684,9 @@ 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)----ProjectionExec: expr=[id@0 as id, get_field(s@1, value) as multi_struct.s[value]] +04)------MergePartitionsExec: partitions=5→4 +05)--------DataSourceExec: file_groups={5 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, s], file_type=parquet # Verify correctness query II @@ -710,7 +712,9 @@ 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)----ProjectionExec: expr=[id@0 as id, get_field(s@1, value) as multi_struct.s[value]] +04)------MergePartitionsExec: partitions=5→4 +05)--------DataSourceExec: file_groups={5 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, s], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -734,7 +738,9 @@ 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)----ProjectionExec: expr=[id@0 as id, get_field(s@1, value) + 1 as multi_struct.s[value] + Int64(1)] +04)------MergePartitionsExec: partitions=5→4 +05)--------DataSourceExec: file_groups={5 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, s], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -762,8 +768,9 @@ 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)--------ProjectionExec: expr=[get_field(s@1, value) as __datafusion_extracted_1, id@0 as id] +06)----------MergePartitionsExec: partitions=5→4 +07)------------DataSourceExec: file_groups={5 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, s], 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 @@ -788,9 +795,11 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as multi_struct.s[label], sum(__datafusion_extracted_2)@1 as sum(multi_struct.s[value])] 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 +03)----RepartitionExec: partitioning=Hash([__datafusion_extracted_1@0], 4), input_partitions=4 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)--------ProjectionExec: expr=[get_field(s@0, label) as __datafusion_extracted_1, get_field(s@0, value) as __datafusion_extracted_2] +06)----------MergePartitionsExec: partitions=5→4 +07)------------DataSourceExec: file_groups={5 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=[s], file_type=parquet # Verify correctness query TI diff --git a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt index ca4a30fa96c35..601b597204816 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt @@ -151,7 +151,8 @@ 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)------MergePartitionsExec: partitions=4→2 +05)--------DataSourceExec: file_groups={4 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=[] query I select max(id) from agg_dyn_test where id > 1; @@ -166,7 +167,8 @@ 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)------MergePartitionsExec: partitions=4→2 +05)--------DataSourceExec: file_groups={4 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 ] # Expect dynamic filter available inside data source query TT @@ -176,7 +178,8 @@ 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)------MergePartitionsExec: partitions=4→2 +05)--------DataSourceExec: file_groups={4 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=[] # Dynamic filter should not be available for grouping sets query TT @@ -188,7 +191,8 @@ 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)--------MergePartitionsExec: partitions=4→2 +06)----------DataSourceExec: file_groups={4 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=[] statement ok drop table agg_dyn_test; diff --git a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt index e2c9fa4237939..1f2842be0d2e9 100644 --- a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt +++ b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt @@ -374,16 +374,16 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([env@0, time_bin@1], 3), input_partitions=3 06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] -08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) -09)----------------SortExec: expr=[f_dkey@0 ASC NULLS LAST, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] -10)------------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3 -11)--------------------AggregateExec: mode=Partial, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) -12)----------------------ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] -13)------------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] -14)--------------------------CoalescePartitionsExec -15)----------------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] -16)------------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -17)--------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)] +09)----------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3 +10)------------------AggregateExec: mode=Partial, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)] +11)--------------------ProjectionExec: expr=[f_dkey@2 as f_dkey, env@3 as env, timestamp@0 as timestamp, value@1 as value] +12)----------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(f_dkey@2, d_dkey@1)], projection=[timestamp@0, value@1, f_dkey@2, env@3] +13)------------------------CoalescePartitionsExec +14)--------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet +15)------------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] +16)--------------------------MergePartitionsExec: partitions=4→3 +17)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log AND DynamicFilter [ empty ], pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] # Verify results without subset satisfaction query TPR rowsort @@ -473,13 +473,16 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([env@0, time_bin@1], 3), input_partitions=3 06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] -08)--------------AggregateExec: mode=SinglePartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) -09)----------------ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] -10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] -11)--------------------CoalescePartitionsExec -12)----------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] -13)------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -14)--------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)] +09)----------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3 +10)------------------AggregateExec: mode=Partial, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)] +11)--------------------ProjectionExec: expr=[f_dkey@2 as f_dkey, env@3 as env, timestamp@0 as timestamp, value@1 as value] +12)----------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(f_dkey@2, d_dkey@1)], projection=[timestamp@0, value@1, f_dkey@2, env@3] +13)------------------------CoalescePartitionsExec +14)--------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet +15)------------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] +16)--------------------------MergePartitionsExec: partitions=4→3 +17)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log AND DynamicFilter [ empty ], pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] # Verify results match with subset satisfaction query TPR rowsort diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index 99f26b66d458b..84a7df09d7fb2 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -1109,7 +1109,8 @@ logical_plan 02)--TableScan: reversed_parquet projection=[id, value] 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/sort_pushdown/reversed/a_high.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet]]}, projection=[id, value], file_type=parquet +02)--MergePartitionsExec: partitions=3→1 +03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/a_high.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet # Test 4.2: Results must be correct query II @@ -1160,7 +1161,8 @@ logical_plan 02)--TableScan: overlap_parquet projection=[id, value] 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/sort_pushdown/overlap/file_x.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/overlap/file_y.parquet]]}, projection=[id, value], file_type=parquet +02)--MergePartitionsExec: partitions=2→1 +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/overlap/file_x.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/overlap/file_y.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet # Test 5.2: Results must be correct query II @@ -1193,7 +1195,8 @@ logical_plan 02)--TableScan: reversed_with_order_parquet projection=[id, value] 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/sort_pushdown/reversed/a_high.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet]]}, projection=[id, value], file_type=parquet +02)--MergePartitionsExec: partitions=3→1 +03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/a_high.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet # Test 6.2: Results must be correct query II @@ -1253,7 +1256,10 @@ EXPLAIN SELECT * FROM correct_parquet ORDER BY id ASC; logical_plan 01)Sort: correct_parquet.id ASC NULLS LAST 02)--TableScan: correct_parquet projection=[id, value] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/a_low.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/c_high.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet +physical_plan +01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--MergePartitionsExec: partitions=3→1 +03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/a_low.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/b_mid.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/c_high.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet # Test 7.2: Results must be correct query II @@ -1279,7 +1285,8 @@ logical_plan 02)--TableScan: correct_parquet projection=[id, value] physical_plan 01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/c_high.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/a_low.parquet]]}, projection=[id, value], file_type=parquet, reverse_row_groups=true +02)--MergePartitionsExec: partitions=3→1 +03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/a_low.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/b_mid.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/c_high.parquet]]}, projection=[id, value], file_type=parquet, reverse_row_groups=true query II SELECT * FROM correct_parquet ORDER BY id DESC; @@ -1332,7 +1339,8 @@ logical_plan 02)--TableScan: desc_reversed_parquet projection=[id, value] physical_plan 01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/a_low.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/b_high.parquet]]}, projection=[id, value], file_type=parquet +02)--MergePartitionsExec: partitions=2→1 +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/a_low.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/b_high.parquet]]}, projection=[id, value], output_ordering=[id@0 DESC], file_type=parquet # Test 8.2: Results must be correct query II @@ -1387,7 +1395,8 @@ logical_plan 02)--TableScan: multi_col_parquet projection=[category, id, value] physical_plan 01)SortExec: expr=[category@0 ASC NULLS LAST, id@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col/a_first.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col/b_second.parquet]]}, projection=[category, id, value], file_type=parquet +02)--MergePartitionsExec: partitions=2→1 +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col/a_first.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col/b_second.parquet]]}, projection=[category, id, value], output_ordering=[category@0 ASC NULLS LAST, id@1 ASC NULLS LAST], file_type=parquet # Test 9.2: Results must be correct query TII @@ -1436,7 +1445,10 @@ EXPLAIN SELECT * FROM multi_col_clean_parquet ORDER BY category ASC, id ASC; logical_plan 01)Sort: multi_col_clean_parquet.category ASC NULLS LAST, multi_col_clean_parquet.id ASC NULLS LAST 02)--TableScan: multi_col_clean_parquet projection=[category, id, value] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col_clean/x_first.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col_clean/y_second.parquet]]}, projection=[category, id, value], output_ordering=[category@0 ASC NULLS LAST, id@1 ASC NULLS LAST], file_type=parquet +physical_plan +01)SortExec: expr=[category@0 ASC NULLS LAST, id@1 ASC NULLS LAST], preserve_partitioning=[false] +02)--MergePartitionsExec: partitions=2→1 +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col_clean/x_first.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col_clean/y_second.parquet]]}, projection=[category, id, value], output_ordering=[category@0 ASC NULLS LAST, id@1 ASC NULLS LAST], file_type=parquet # Test 9.3b: Results must be correct query TII @@ -1464,7 +1476,10 @@ EXPLAIN SELECT * FROM correct_with_order_parquet ORDER BY id ASC; logical_plan 01)Sort: correct_with_order_parquet.id ASC NULLS LAST 02)--TableScan: correct_with_order_parquet projection=[id, value] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/a_low.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/c_high.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet +physical_plan +01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--MergePartitionsExec: partitions=3→1 +03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/a_low.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/b_mid.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/c_high.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet # Test 10.2: Results must be correct query II From 73267e5ac998012e7efe9b0188fade58f0f62018 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 27 Feb 2026 16:30:34 +0100 Subject: [PATCH 47/53] Tryout different approach --- datafusion/sqllogictest/test_files/limit_pruning.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index 037eb3de8a93b..9a67e767dea43 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -71,8 +71,8 @@ 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)SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=411.38µs, output_bytes=72.0 B] +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=1ns, 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=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=237.46µs, scan_efficiency_ratio=22% (521/2.35 K)] statement ok drop table tracking_data; From 5901ac5d291e9e50044864e09930b44e955526ae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 27 Feb 2026 16:51:07 +0100 Subject: [PATCH 48/53] Tryout different approach --- datafusion/datasource-parquet/src/opener.rs | 447 +----------------- datafusion/datasource/src/file_stream.rs | 23 - .../physical-plan/src/merge_partitions.rs | 430 +++++++++++++++++ 3 files changed, 450 insertions(+), 450 deletions(-) create mode 100644 datafusion/physical-plan/src/merge_partitions.rs diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 7d0f215b0ff87..3561df55f5003 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -56,7 +56,6 @@ use crate::sort::reverse_row_selection; 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; @@ -125,14 +124,6 @@ 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 @@ -229,259 +220,6 @@ 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, - ) -> 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 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 - .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(); - 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; - 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")] - 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 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 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()); - 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); - - // 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| { - 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, page_pruning_predicate) = build_pruning_predicates( - adapted_predicate.as_ref(), - &physical_file_schema, - &predicate_creation_errors, - ); - - let mut row_groups = Self::build_row_group_access_filter( - &file_name, - extensions, - num_row_groups, - reader_metadata.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, - }), - )?; - - // 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, - reader_metadata.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). - // - // 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() { - // 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.clone(), - ); - 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()); - } - } - - // 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() - // 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 { - if !access_plan.should_scan(i) { - continue; - } - 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), - 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. @@ -581,11 +319,6 @@ 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")] @@ -665,20 +398,9 @@ impl FileOpener for ParquetOpener { // 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). - // 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 { + let mut reader_metadata = 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. @@ -748,11 +470,7 @@ 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. - // 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 - { + if should_enable_page_index(enable_page_index, &page_pruning_predicate) { reader_metadata = load_page_index( reader_metadata, &mut async_file_reader, @@ -823,7 +541,7 @@ 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) .map(|predicate| RowGroupStatisticsPruningContext { physical_file_schema: &physical_file_schema, parquet_schema: builder.parquet_schema(), @@ -848,25 +566,21 @@ impl FileOpener for ParquetOpener { 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()); - } + 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) @@ -887,11 +601,8 @@ 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( @@ -1255,14 +966,6 @@ 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}"); } @@ -1355,7 +1058,7 @@ mod test { use super::{ConstantColumns, constant_columns_from_stats}; use crate::{ DefaultParquetFileReaderFactory, RowGroupAccess, - opener::{ParquetMorsel, ParquetOpener}, + opener::ParquetOpener, }; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use bytes::{BufMut, BytesMut}; @@ -2357,114 +2060,4 @@ mod test { ); } - #[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 baseline_bloom_metrics = get_pruning_metric( - &baseline_opener.metrics, - "row_groups_pruned_bloom_filter", - ); - - 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", - ); - let morsel_bloom_metrics = get_pruning_metric( - &morsel_opener.metrics, - "row_groups_pruned_bloom_filter", - ); - - 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})" - ); - - 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})" - ); - } - } } diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 1b8387483a908..3e5ca1e1e8267 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -305,29 +305,6 @@ 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]))) - } - - /// 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/physical-plan/src/merge_partitions.rs b/datafusion/physical-plan/src/merge_partitions.rs new file mode 100644 index 0000000000000..d22549eee8492 --- /dev/null +++ b/datafusion/physical-plan/src/merge_partitions.rs @@ -0,0 +1,430 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! MergePartitionsExec maps N input partitions to M output partitions using +//! an atomic counter for work-stealing. Each output stream atomically claims +//! the next input partition, executes it, yields all batches, then claims +//! the next. No channels needed. + +use std::any::Any; +use std::pin::Pin; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use super::{DisplayAs, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, Statistics}; +use crate::execution_plan::{CardinalityEffect, EvaluationType}; +use crate::filter_pushdown::{FilterDescription, FilterPushdownPhase}; +use crate::sort_pushdown::SortOrderPushdownResult; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, check_if_same_properties}; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; + +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use datafusion_common::config::ConfigOptions; +use datafusion_common::Result; +use datafusion_execution::{RecordBatchStream, TaskContext}; +use datafusion_physical_expr::PhysicalExpr; + +use futures::{Stream, ready}; + +/// Maps N input partitions to M output partitions using work-stealing. +/// +/// Each output stream atomically claims the next unclaimed input partition, +/// executes it to completion (yielding all batches), then claims the next. +/// This provides natural load balancing via Tokio's task scheduler without +/// requiring channels or background tasks. +#[derive(Debug)] +pub struct MergePartitionsExec { + input: Arc, + output_partitions: usize, + next_partition: Arc, + metrics: ExecutionPlanMetricsSet, + cache: Arc, +} + +impl Clone for MergePartitionsExec { + fn clone(&self) -> Self { + Self { + input: Arc::clone(&self.input), + output_partitions: self.output_partitions, + next_partition: Arc::new(AtomicUsize::new(0)), + metrics: ExecutionPlanMetricsSet::new(), + cache: Arc::clone(&self.cache), + } + } +} + +impl MergePartitionsExec { + /// Create a new MergePartitionsExec that maps the input's partitions + /// to `output_partitions` output partitions. + pub fn new(input: Arc, output_partitions: usize) -> Self { + let cache = Self::compute_properties(&input, output_partitions); + MergePartitionsExec { + input, + output_partitions, + next_partition: Arc::new(AtomicUsize::new(0)), + metrics: ExecutionPlanMetricsSet::new(), + cache: Arc::new(cache), + } + } + + /// Input execution plan + pub fn input(&self) -> &Arc { + &self.input + } + + /// Number of output partitions + pub fn output_partitions(&self) -> usize { + self.output_partitions + } + + fn compute_properties( + input: &Arc, + output_partitions: usize, + ) -> PlanProperties { + let mut eq_properties = input.equivalence_properties().clone(); + eq_properties.clear_orderings(); + eq_properties.clear_per_partition_constants(); + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(output_partitions), + input.pipeline_behavior(), + input.boundedness(), + ) + .with_evaluation_type(EvaluationType::Lazy) + .with_scheduling_type(input.properties().scheduling_type) + } + + fn with_new_children_and_same_properties( + &self, + mut children: Vec>, + ) -> Self { + Self { + input: children.swap_remove(0), + metrics: ExecutionPlanMetricsSet::new(), + next_partition: Arc::new(AtomicUsize::new(0)), + ..Self::clone(self) + } + } +} + +impl DisplayAs for MergePartitionsExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + let input_partitions = self.input.output_partitioning().partition_count(); + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!( + f, + "MergePartitionsExec: partitions={input_partitions}→{}", + self.output_partitions + ) + } + DisplayFormatType::TreeRender => { + write!( + f, + "partitions: {input_partitions}→{}", + self.output_partitions + ) + } + } + } +} + +impl ExecutionPlan for MergePartitionsExec { + fn name(&self) -> &'static str { + "MergePartitionsExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &Arc { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + fn benefits_from_input_partitioning(&self) -> Vec { + vec![false] + } + + fn with_new_children( + self: Arc, + mut children: Vec>, + ) -> Result> { + check_if_same_properties!(self, children); + Ok(Arc::new(MergePartitionsExec::new( + children.swap_remove(0), + self.output_partitions, + ))) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + let total_input_partitions = + self.input.output_partitioning().partition_count(); + let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + + Ok(Box::pin(MergeStream { + input: Arc::clone(&self.input), + context, + next_partition: Arc::clone(&self.next_partition), + total_input_partitions, + current_stream: None, + schema: self.schema(), + baseline_metrics, + done: false, + })) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + + fn partition_statistics(&self, _partition: Option) -> Result { + // Can't predict which output partition gets which input data + Ok(Statistics::new_unknown(&self.schema())) + } + + fn cardinality_effect(&self) -> CardinalityEffect { + CardinalityEffect::Equal + } + + fn gather_filters_for_pushdown( + &self, + _phase: FilterPushdownPhase, + parent_filters: Vec>, + _config: &ConfigOptions, + ) -> Result { + FilterDescription::from_children(parent_filters, &self.children()) + } + + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + let result = self.input.try_pushdown_sort(order)?; + + let has_multiple_partitions = + self.input.output_partitioning().partition_count() > 1; + + result + .try_map(|new_input| { + Ok(Arc::new(MergePartitionsExec::new( + new_input, + self.output_partitions, + )) as Arc) + }) + .map(|r| { + if has_multiple_partitions { + r.into_inexact() + } else { + r + } + }) + } +} + +struct MergeStream { + input: Arc, + context: Arc, + next_partition: Arc, + total_input_partitions: usize, + current_stream: Option, + schema: SchemaRef, + baseline_metrics: BaselineMetrics, + done: bool, +} + +impl Stream for MergeStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let this = &mut *self; + + if this.done { + return Poll::Ready(None); + } + + loop { + // If we have a current stream, poll it + if let Some(stream) = &mut this.current_stream { + let poll = stream.as_mut().poll_next(cx); + match ready!(poll) { + Some(Ok(batch)) => { + this.baseline_metrics + .record_output(batch.num_rows()); + return Poll::Ready(Some(Ok(batch))); + } + Some(Err(e)) => { + this.done = true; + return Poll::Ready(Some(Err(e))); + } + None => { + // Current stream exhausted, drop it and try next + this.current_stream = None; + } + } + } + + // Claim next input partition atomically + let partition_idx = + this.next_partition.fetch_add(1, Ordering::Relaxed); + if partition_idx >= this.total_input_partitions { + this.done = true; + return Poll::Ready(None); + } + + // Start new input stream (synchronous, lazy — actual I/O on poll) + match this + .input + .execute(partition_idx, Arc::clone(&this.context)) + { + Ok(stream) => { + this.current_stream = Some(stream); + // Loop back to poll the new stream + } + Err(e) => { + this.done = true; + return Poll::Ready(Some(Err(e))); + } + } + } + } +} + +impl RecordBatchStream for MergeStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::test; + use crate::common; + + + #[tokio::test] + async fn test_merge_10_to_3() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + + let num_input_partitions = 10; + let num_output_partitions = 3; + let input = test::scan_partitioned(num_input_partitions); + + assert_eq!( + input.output_partitioning().partition_count(), + num_input_partitions + ); + + let merge = + MergePartitionsExec::new(input, num_output_partitions); + + assert_eq!( + merge + .properties() + .output_partitioning() + .partition_count(), + num_output_partitions + ); + + // Collect all output partitions + let mut total_rows = 0; + for partition in 0..num_output_partitions { + let stream = merge.execute(partition, Arc::clone(&task_ctx))?; + let batches = common::collect(stream).await?; + let rows: usize = + batches.iter().map(|batch| batch.num_rows()).sum(); + total_rows += rows; + } + + // 10 partitions × 100 rows each = 1000 total rows + assert_eq!(total_rows, 1000); + + Ok(()) + } + + #[tokio::test] + async fn test_merge_output_ge_input() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + + let num_input_partitions = 3; + let num_output_partitions = 5; + let input = test::scan_partitioned(num_input_partitions); + + let merge = + MergePartitionsExec::new(input, num_output_partitions); + + assert_eq!( + merge + .properties() + .output_partitioning() + .partition_count(), + num_output_partitions + ); + + let mut total_rows = 0; + for partition in 0..num_output_partitions { + let stream = merge.execute(partition, Arc::clone(&task_ctx))?; + let batches = common::collect(stream).await?; + let rows: usize = + batches.iter().map(|batch| batch.num_rows()).sum(); + total_rows += rows; + } + + // 3 partitions × 100 rows each = 300 total rows + assert_eq!(total_rows, 300); + + Ok(()) + } + + #[tokio::test] + async fn test_schema_preserved() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + + let input = test::scan_partitioned(4); + let expected_schema = input.schema(); + + let merge = MergePartitionsExec::new(input, 2); + + // Verify the operator schema matches input + assert_eq!(merge.schema(), expected_schema); + + // Verify stream schema matches + let stream = merge.execute(0, task_ctx)?; + assert_eq!(stream.schema(), expected_schema); + + Ok(()) + } +} From 10a120c189031df5b762fcd9d0151a4403402098 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 27 Feb 2026 20:23:56 +0100 Subject: [PATCH 49/53] wip --- datafusion/physical-optimizer/src/enforce_distribution.rs | 7 +++++++ datafusion/sqllogictest/test_files/limit_pruning.slt | 4 ++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index d23a699f715de..c7bb5481b7d6b 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1306,8 +1306,15 @@ pub fn ensure_distribution( // If repartitioning is not possible (a.k.a. None is returned from `ExecutionPlan::repartitioned`) // then no repartitioning will have occurred. As the default implementation returns None, it is only // specific physical plan nodes, such as certain datasources, which are repartitioned. + // Only repartition file scans when it would not reduce the + // child's partition count. When the child already has more + // partitions than target (e.g. under MergePartitionsExec), + // collapsing them would be counterproductive. + let would_reduce_partitions = + child.plan.output_partitioning().partition_count() > target_partitions; if repartition_file_scans && roundrobin_beneficial_stats + && !would_reduce_partitions && let Some(new_child) = child.plan.repartitioned(target_partitions, config)? { diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index 9a67e767dea43..f6c54f26d25fe 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -71,8 +71,8 @@ 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=411.38µs, output_bytes=72.0 B] -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=1ns, 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=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=237.46µs, scan_efficiency_ratio=22% (521/2.35 K)] +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=161.67µs, output_bytes=72.0 B] +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=1ns, 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=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=253.42µs, scan_efficiency_ratio=22% (521/2.35 K)] statement ok drop table tracking_data; From 82312b6194d196a909ae5ee0d66b1d0b5fa56b7f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 27 Feb 2026 20:27:26 +0100 Subject: [PATCH 50/53] Disable morsel-driven execution in partition_statistics tests These tests directly query per-partition statistics from DataSourceExec, which returns unknown stats when wrapped by MergePartitionsExec. Co-Authored-By: Claude Opus 4.6 --- .../core/tests/physical_optimizer/partition_statistics.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index fd278d147edbc..9601095bb0d9a 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -77,8 +77,12 @@ mod test { create_table_sql: Option<&str>, target_partition: Option, ) -> Arc { - let mut session_config = SessionConfig::new() - .with_collect_statistics(true); + let mut session_config = SessionConfig::new().with_collect_statistics(true); + session_config + .options_mut() + .execution + .parquet + .allow_morsel_driven = false; if let Some(partition) = target_partition { session_config = session_config.with_target_partitions(partition); } From db78f8300f263bb2e3eff2f63f733549629c8554 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 27 Feb 2026 20:28:12 +0100 Subject: [PATCH 51/53] wip --- .../custom_data_source/csv_json_opener.rs | 8 +-- .../src/datasource/physical_plan/parquet.rs | 1 - .../datasource-parquet/src/file_format.rs | 9 +-- datafusion/datasource-parquet/src/opener.rs | 6 +- datafusion/datasource/src/file_scan_config.rs | 17 ++---- datafusion/datasource/src/file_stream.rs | 4 +- datafusion/datasource/src/source.rs | 5 +- .../physical-plan/src/merge_partitions.rs | 56 +++++++------------ 8 files changed, 33 insertions(+), 73 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/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 1d387b37d19d0..9835574ab5827 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -2462,5 +2462,4 @@ mod tests { assert_eq!(calls.len(), 2); assert_eq!(calls, vec![Some(123), Some(456)]); } - } diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 327687dacaada..d5c48645d3090 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -48,8 +48,8 @@ use datafusion_common::{ use datafusion_common::{HashMap, Statistics}; use datafusion_common_runtime::{JoinSet, SpawnedTask}; use datafusion_datasource::display::FileGroupDisplay; -use datafusion_datasource::file_groups::FileGroup; use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_groups::FileGroup; use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; use datafusion_datasource::sink::{DataSink, DataSinkExec}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; @@ -544,14 +544,11 @@ impl FileFormat for ParquetFormat { .file_groups .iter() .flat_map(|group| { - group - .iter() - .map(|file| FileGroup::new(vec![file.clone()])) + group.iter().map(|file| FileGroup::new(vec![file.clone()])) }) .collect(); - let target_partitions = - state.config_options().execution.target_partitions; + let target_partitions = state.config_options().execution.target_partitions; let conf = FileScanConfigBuilder::from(conf) .with_source(Arc::new(source)) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 3561df55f5003..3ac4d312c92f5 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1056,10 +1056,7 @@ mod test { use std::sync::Arc; use super::{ConstantColumns, constant_columns_from_stats}; - use crate::{ - DefaultParquetFileReaderFactory, RowGroupAccess, - opener::ParquetOpener, - }; + use crate::{DefaultParquetFileReaderFactory, RowGroupAccess, opener::ParquetOpener}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use bytes::{BufMut, BytesMut}; use datafusion_common::{ @@ -2059,5 +2056,4 @@ mod test { "Reverse scan with non-contiguous row groups should correctly map RowSelection" ); } - } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index c15b25869ca43..c3e5cabce7bc2 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -20,13 +20,9 @@ use crate::file_groups::FileGroup; 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, + source::DataSource, statistics::MinMaxStatistics, }; use arrow::datatypes::FieldRef; use arrow::datatypes::{DataType, Schema, SchemaRef}; @@ -588,12 +584,7 @@ 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())?; Ok(Box::pin(cooperative(stream))) } diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 3e5ca1e1e8267..f216855471482 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -115,9 +115,7 @@ impl FileStream { self.file_stream_metrics.time_opening.start(); 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; diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index c75dd228687ac..cda7ab470d4b7 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -327,10 +327,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, - })) + Some(Arc::new(Self { data_source, cache })) } fn fetch(&self) -> Option { diff --git a/datafusion/physical-plan/src/merge_partitions.rs b/datafusion/physical-plan/src/merge_partitions.rs index d22549eee8492..820a78fb8ebb2 100644 --- a/datafusion/physical-plan/src/merge_partitions.rs +++ b/datafusion/physical-plan/src/merge_partitions.rs @@ -22,12 +22,15 @@ use std::any::Any; use std::pin::Pin; -use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; +use std::sync::atomic::{AtomicUsize, Ordering}; use std::task::{Context, Poll}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use super::{DisplayAs, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, Statistics}; +use super::{ + DisplayAs, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, + Statistics, +}; use crate::execution_plan::{CardinalityEffect, EvaluationType}; use crate::filter_pushdown::{FilterDescription, FilterPushdownPhase}; use crate::sort_pushdown::SortOrderPushdownResult; @@ -36,8 +39,8 @@ use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use datafusion_common::config::ConfigOptions; use datafusion_common::Result; +use datafusion_common::config::ConfigOptions; use datafusion_execution::{RecordBatchStream, TaskContext}; use datafusion_physical_expr::PhysicalExpr; @@ -187,8 +190,7 @@ impl ExecutionPlan for MergePartitionsExec { partition: usize, context: Arc, ) -> Result { - let total_input_partitions = - self.input.output_partitioning().partition_count(); + let total_input_partitions = self.input.output_partitioning().partition_count(); let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); Ok(Box::pin(MergeStream { @@ -236,10 +238,10 @@ impl ExecutionPlan for MergePartitionsExec { result .try_map(|new_input| { - Ok(Arc::new(MergePartitionsExec::new( - new_input, - self.output_partitions, - )) as Arc) + Ok( + Arc::new(MergePartitionsExec::new(new_input, self.output_partitions)) + as Arc, + ) }) .map(|r| { if has_multiple_partitions { @@ -281,8 +283,7 @@ impl Stream for MergeStream { let poll = stream.as_mut().poll_next(cx); match ready!(poll) { Some(Ok(batch)) => { - this.baseline_metrics - .record_output(batch.num_rows()); + this.baseline_metrics.record_output(batch.num_rows()); return Poll::Ready(Some(Ok(batch))); } Some(Err(e)) => { @@ -297,18 +298,14 @@ impl Stream for MergeStream { } // Claim next input partition atomically - let partition_idx = - this.next_partition.fetch_add(1, Ordering::Relaxed); + let partition_idx = this.next_partition.fetch_add(1, Ordering::Relaxed); if partition_idx >= this.total_input_partitions { this.done = true; return Poll::Ready(None); } // Start new input stream (synchronous, lazy — actual I/O on poll) - match this - .input - .execute(partition_idx, Arc::clone(&this.context)) - { + match this.input.execute(partition_idx, Arc::clone(&this.context)) { Ok(stream) => { this.current_stream = Some(stream); // Loop back to poll the new stream @@ -331,9 +328,8 @@ impl RecordBatchStream for MergeStream { #[cfg(test)] mod tests { use super::*; - use crate::test; use crate::common; - + use crate::test; #[tokio::test] async fn test_merge_10_to_3() -> Result<()> { @@ -348,14 +344,10 @@ mod tests { num_input_partitions ); - let merge = - MergePartitionsExec::new(input, num_output_partitions); + let merge = MergePartitionsExec::new(input, num_output_partitions); assert_eq!( - merge - .properties() - .output_partitioning() - .partition_count(), + merge.properties().output_partitioning().partition_count(), num_output_partitions ); @@ -364,8 +356,7 @@ mod tests { for partition in 0..num_output_partitions { let stream = merge.execute(partition, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; - let rows: usize = - batches.iter().map(|batch| batch.num_rows()).sum(); + let rows: usize = batches.iter().map(|batch| batch.num_rows()).sum(); total_rows += rows; } @@ -383,14 +374,10 @@ mod tests { let num_output_partitions = 5; let input = test::scan_partitioned(num_input_partitions); - let merge = - MergePartitionsExec::new(input, num_output_partitions); + let merge = MergePartitionsExec::new(input, num_output_partitions); assert_eq!( - merge - .properties() - .output_partitioning() - .partition_count(), + merge.properties().output_partitioning().partition_count(), num_output_partitions ); @@ -398,8 +385,7 @@ mod tests { for partition in 0..num_output_partitions { let stream = merge.execute(partition, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; - let rows: usize = - batches.iter().map(|batch| batch.num_rows()).sum(); + let rows: usize = batches.iter().map(|batch| batch.num_rows()).sum(); total_rows += rows; } From 513374bc1c9347cb767783401c94fe3c9ac4ebd0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 27 Feb 2026 21:14:02 +0100 Subject: [PATCH 52/53] wip --- .../datasource-parquet/src/file_format.rs | 59 ++- datafusion/datasource-parquet/src/mod.rs | 1 + datafusion/datasource-parquet/src/opener.rs | 2 +- .../physical-plan/src/merge_partitions.rs | 371 ++++++++++++++---- 4 files changed, 342 insertions(+), 91 deletions(-) diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index d5c48645d3090..e1b2ca3bcf8f0 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -27,12 +27,12 @@ use std::{fmt, vec}; use arrow::array::RecordBatch; use arrow::datatypes::{Fields, Schema, SchemaRef, TimeUnit}; -use datafusion_datasource::TableSchema; use datafusion_datasource::file_compression_type::FileCompressionType; use datafusion_datasource::file_sink_config::{FileSink, FileSinkConfig}; use datafusion_datasource::write::{ ObjectWriterBuilder, SharedBuffer, get_writer_schema, }; +use datafusion_datasource::{PartitionedFile, TableSchema}; use datafusion_datasource::file_format::{FileFormat, FileFormatFactory}; use datafusion_datasource::write::demux::DemuxedStreamReceiver; @@ -524,8 +524,10 @@ impl FileFormat for ParquetFormat { let store = state .runtime_env() .object_store(conf.object_store_url.clone())?; - let cached_parquet_read_factory = - Arc::new(CachedParquetFileReaderFactory::new(store, metadata_cache)); + let cached_parquet_read_factory = Arc::new(CachedParquetFileReaderFactory::new( + Arc::clone(&store), + metadata_cache, + )); source = source.with_parquet_file_reader_factory(cached_parquet_read_factory); if let Some(metadata_size_hint) = metadata_size_hint { @@ -537,9 +539,41 @@ impl FileFormat for ParquetFormat { let use_merge_partitions = self.options.global.allow_morsel_driven; if use_merge_partitions { - // Create one partition per file for maximum parallelism, - // then wrap with MergePartitionsExec to reduce to target_partitions. - // This uses atomic work-stealing at the partition level. + // Morsel-driven execution: create one partition per file for the + // plan tree, then use a ParquetMorselizer that lazily expands + // files into row-group-level work items for fine-grained + // work-stealing via MergePartitionsExec. + let target_partitions = state.config_options().execution.target_partitions; + + // Collect all files before conf is consumed. + let all_files: Vec = conf + .file_groups + .iter() + .flat_map(|group| group.iter().cloned()) + .collect(); + + // Get the reader factory (for metadata reads in the morselizer). + // It was set above as CachedParquetFileReaderFactory. + let reader_factory: Arc = source + .parquet_file_reader_factory() + .cloned() + .expect("reader factory was set above"); + + // Set batch size on source before creating the opener (normally + // done by FileScanConfig::open_file_reader, but we create the + // opener directly here for the morselizer). + source.batch_size = Some( + conf.batch_size + .unwrap_or(state.config_options().execution.batch_size), + ); + + // Create the opener for row-group morsel execution. + let opener = source.create_file_opener(Arc::clone(&store), &conf, 0)?; + + let output_schema = conf.projected_schema()?; + let metrics = source.metrics().clone(); + + // Build the DataSourceExec (one partition per file, for plan tree). let one_per_file: Vec = conf .file_groups .iter() @@ -548,8 +582,6 @@ impl FileFormat for ParquetFormat { }) .collect(); - let target_partitions = state.config_options().execution.target_partitions; - let conf = FileScanConfigBuilder::from(conf) .with_source(Arc::new(source)) .with_file_groups(one_per_file) @@ -561,9 +593,18 @@ impl FileFormat for ParquetFormat { .partition_count(); if input_partitions > target_partitions { - Ok(Arc::new(MergePartitionsExec::new( + let morselizer = Arc::new(crate::morselizer::ParquetMorselizer::new( + all_files, + opener, + reader_factory, + metrics, + metadata_size_hint, + output_schema, + )); + Ok(Arc::new(MergePartitionsExec::new_with_morselizer( data_source, target_partitions, + morselizer, ))) } else { Ok(data_source) diff --git a/datafusion/datasource-parquet/src/mod.rs b/datafusion/datasource-parquet/src/mod.rs index 0e137a706fad7..d1e46196103ae 100644 --- a/datafusion/datasource-parquet/src/mod.rs +++ b/datafusion/datasource-parquet/src/mod.rs @@ -24,6 +24,7 @@ pub mod access_plan; pub mod file_format; pub mod metadata; mod metrics; +pub(crate) mod morselizer; mod opener; mod page_filter; mod reader; diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 3ac4d312c92f5..5fd3f21832ea0 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -69,7 +69,7 @@ use parquet::file::metadata::{PageIndexPolicy, ParquetMetaDataReader, RowGroupMe use parquet::schema::types::SchemaDescriptor; /// Implements [`FileOpener`] for a parquet file -pub(super) struct ParquetOpener { +pub(crate) struct ParquetOpener { /// Execution partition index pub(crate) partition_index: usize, /// Projection to apply on top of the table schema (i.e. can reference partition columns). diff --git a/datafusion/physical-plan/src/merge_partitions.rs b/datafusion/physical-plan/src/merge_partitions.rs index 820a78fb8ebb2..a1dcca2780ffb 100644 --- a/datafusion/physical-plan/src/merge_partitions.rs +++ b/datafusion/physical-plan/src/merge_partitions.rs @@ -15,15 +15,17 @@ // specific language governing permissions and limitations // under the License. -//! MergePartitionsExec maps N input partitions to M output partitions using -//! an atomic counter for work-stealing. Each output stream atomically claims -//! the next input partition, executes it, yields all batches, then claims -//! the next. No channels needed. +//! MergePartitionsExec maps N input work items to M output partitions using +//! a shared work queue for work-stealing. Each output stream pops the next +//! work item, executes it (which may produce additional items), yields all +//! batches, then pops the next. Supports lazy morselization: file-level items +//! can be expanded into row-group-level items at execution time. use std::any::Any; +use std::collections::VecDeque; +use std::fmt; 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 super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; @@ -44,19 +46,151 @@ use datafusion_common::config::ConfigOptions; use datafusion_execution::{RecordBatchStream, TaskContext}; use datafusion_physical_expr::PhysicalExpr; -use futures::{Stream, ready}; +use futures::future::BoxFuture; +use futures::stream::BoxStream; +use futures::{Stream, StreamExt, ready}; -/// Maps N input partitions to M output partitions using work-stealing. +// ───────────────────────────────────────────────────────────────────────────── +// Morselizer trait and types +// ───────────────────────────────────────────────────────────────────────────── + +/// An opaque unit of work that a [`Morselizer`] can execute. +/// +/// Work items are type-erased so that [`MergePartitionsExec`] stays generic. +/// The [`Morselizer`] implementation knows how to interpret them. +pub struct WorkItem { + data: Box, +} + +impl WorkItem { + /// Create a new work item wrapping any `Send + Sync + 'static` value. + pub fn new(value: T) -> Self { + Self { + data: Box::new(value), + } + } + + /// Downcast to a concrete type reference. + pub fn downcast_ref(&self) -> Option<&T> { + self.data.downcast_ref() + } + + /// Consume and downcast to a concrete type. + pub fn downcast(self) -> Result> { + match self.data.downcast::() { + Ok(val) => Ok(*val), + Err(boxed) => Err(boxed), + } + } +} + +impl fmt::Debug for WorkItem { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("WorkItem").finish_non_exhaustive() + } +} + +/// The result of executing a [`WorkItem`] via a [`Morselizer`]. +pub struct MorselResult { + /// A stream of record batches for this morsel. + pub stream: BoxStream<'static, Result>, + /// Additional work items discovered during execution (e.g., row groups + /// discovered after reading file metadata). These are pushed into the + /// shared queue for other output streams to steal. + pub additional_items: Vec, +} + +/// Provides work items and executes them for [`MergePartitionsExec`]. +/// +/// This trait enables lazy morselization: a [`Morselizer`] can start with +/// coarse-grained items (e.g., files) and expand them into fine-grained +/// items (e.g., row groups) at execution time, after reading metadata. +pub trait Morselizer: Send + Sync + fmt::Debug { + /// Return the initial set of work items (called once per execute). + fn initial_items(&self) -> Vec; + + /// Execute a work item. Returns a future that resolves to a stream + /// of record batches plus any additional work items discovered. + /// + /// For example, executing a file item might read parquet metadata, + /// return a stream for the first row group, and return additional + /// items for the remaining row groups. + fn execute_item( + &self, + item: WorkItem, + context: Arc, + ) -> Result>>; + + /// The output schema. + fn schema(&self) -> SchemaRef; +} + +// ───────────────────────────────────────────────────────────────────────────── +// PartitionMorselizer — default impl wrapping an ExecutionPlan +// ───────────────────────────────────────────────────────────────────────────── + +/// Default [`Morselizer`] that provides one work item per input partition. /// -/// Each output stream atomically claims the next unclaimed input partition, -/// executes it to completion (yielding all batches), then claims the next. -/// This provides natural load balancing via Tokio's task scheduler without -/// requiring channels or background tasks. +/// This gives the same behavior as the original atomic-counter approach: +/// each output stream claims whole input partitions. +#[derive(Debug)] +struct PartitionMorselizer { + input: Arc, +} + +impl Morselizer for PartitionMorselizer { + fn initial_items(&self) -> Vec { + let count = self.input.output_partitioning().partition_count(); + (0..count).map(|i| WorkItem::new(i as usize)).collect() + } + + fn execute_item( + &self, + item: WorkItem, + context: Arc, + ) -> Result>> { + let partition_idx: usize = *item + .downcast_ref() + .expect("PartitionMorselizer work item should be usize"); + let stream = self.input.execute(partition_idx, context)?; + Ok(Box::pin(async move { + Ok(MorselResult { + stream: stream.boxed(), + additional_items: vec![], + }) + })) + } + + fn schema(&self) -> SchemaRef { + self.input.schema() + } +} + +// ───────────────────────────────────────────────────────────────────────────── +// MergePartitionsExec +// ───────────────────────────────────────────────────────────────────────────── + +/// Maps N input work items to M output partitions using work-stealing. +/// +/// Each output stream pops the next work item from a shared queue, executes +/// it via the [`Morselizer`], yields all batches, then pops the next item. +/// Executing an item may produce additional items (e.g., row groups discovered +/// after reading file metadata), which are pushed back into the queue for +/// other streams to steal. +/// +/// This provides natural load balancing without channels or background tasks. #[derive(Debug)] pub struct MergePartitionsExec { + /// The input plan (used for plan-tree operations: children, with_new_children, etc.) input: Arc, + /// Number of output partitions output_partitions: usize, - next_partition: Arc, + /// Provides and executes work items + morselizer: Arc, + /// Shared work queue populated during execute() + queue: Arc>>, + /// Whether the queue has been initialized + queue_initialized: Arc>, metrics: ExecutionPlanMetricsSet, cache: Arc, } @@ -66,7 +200,9 @@ impl Clone for MergePartitionsExec { Self { input: Arc::clone(&self.input), output_partitions: self.output_partitions, - next_partition: Arc::new(AtomicUsize::new(0)), + morselizer: Arc::clone(&self.morselizer), + queue: Arc::new(Mutex::new(VecDeque::new())), + queue_initialized: Arc::new(Mutex::new(false)), metrics: ExecutionPlanMetricsSet::new(), cache: Arc::clone(&self.cache), } @@ -75,13 +211,32 @@ impl Clone for MergePartitionsExec { impl MergePartitionsExec { /// Create a new MergePartitionsExec that maps the input's partitions - /// to `output_partitions` output partitions. + /// to `output_partitions` output partitions using partition-level + /// work stealing (one work item per input partition). pub fn new(input: Arc, output_partitions: usize) -> Self { + let morselizer = Arc::new(PartitionMorselizer { + input: Arc::clone(&input), + }); + Self::new_with_morselizer(input, output_partitions, morselizer) + } + + /// Create a new MergePartitionsExec with a custom [`Morselizer`]. + /// + /// The morselizer controls the granularity of work stealing. For example, + /// a parquet morselizer can expand file-level items into row-group-level + /// items for finer-grained load balancing. + pub fn new_with_morselizer( + input: Arc, + output_partitions: usize, + morselizer: Arc, + ) -> Self { let cache = Self::compute_properties(&input, output_partitions); MergePartitionsExec { input, output_partitions, - next_partition: Arc::new(AtomicUsize::new(0)), + morselizer, + queue: Arc::new(Mutex::new(VecDeque::new())), + queue_initialized: Arc::new(Mutex::new(false)), metrics: ExecutionPlanMetricsSet::new(), cache: Arc::new(cache), } @@ -97,6 +252,23 @@ impl MergePartitionsExec { self.output_partitions } + fn with_new_children_and_same_properties( + &self, + mut children: Vec>, + ) -> Self { + let morselizer = Arc::new(PartitionMorselizer { + input: Arc::clone(&children[0]), + }); + Self { + input: children.swap_remove(0), + morselizer, + queue: Arc::new(Mutex::new(VecDeque::new())), + queue_initialized: Arc::new(Mutex::new(false)), + metrics: ExecutionPlanMetricsSet::new(), + ..Self::clone(self) + } + } + fn compute_properties( input: &Arc, output_partitions: usize, @@ -113,26 +285,10 @@ impl MergePartitionsExec { .with_evaluation_type(EvaluationType::Lazy) .with_scheduling_type(input.properties().scheduling_type) } - - fn with_new_children_and_same_properties( - &self, - mut children: Vec>, - ) -> Self { - Self { - input: children.swap_remove(0), - metrics: ExecutionPlanMetricsSet::new(), - next_partition: Arc::new(AtomicUsize::new(0)), - ..Self::clone(self) - } - } } impl DisplayAs for MergePartitionsExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { let input_partitions = self.input.output_partitioning().partition_count(); match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { @@ -190,18 +346,28 @@ impl ExecutionPlan for MergePartitionsExec { partition: usize, context: Arc, ) -> Result { - let total_input_partitions = self.input.output_partitioning().partition_count(); + // Initialize the shared queue exactly once (first call to execute). + { + let mut initialized = self.queue_initialized.lock().unwrap(); + if !*initialized { + let items = self.morselizer.initial_items(); + let mut queue = self.queue.lock().unwrap(); + for item in items { + queue.push_back(item); + } + *initialized = true; + } + } + let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); Ok(Box::pin(MergeStream { - input: Arc::clone(&self.input), + morselizer: Arc::clone(&self.morselizer), context, - next_partition: Arc::clone(&self.next_partition), - total_input_partitions, - current_stream: None, + queue: Arc::clone(&self.queue), + state: MergeStreamState::Idle, schema: self.schema(), baseline_metrics, - done: false, })) } @@ -210,7 +376,6 @@ impl ExecutionPlan for MergePartitionsExec { } fn partition_statistics(&self, _partition: Option) -> Result { - // Can't predict which output partition gets which input data Ok(Statistics::new_unknown(&self.schema())) } @@ -253,15 +418,32 @@ impl ExecutionPlan for MergePartitionsExec { } } +// ───────────────────────────────────────────────────────────────────────────── +// MergeStream — the output stream +// ───────────────────────────────────────────────────────────────────────────── + +enum MergeStreamState { + /// Ready to pop the next work item from the queue. + Idle, + /// Waiting for a morsel future to complete (e.g., reading metadata). + Opening { + future: BoxFuture<'static, Result>, + }, + /// Yielding batches from a morsel's stream. + Scanning { + stream: BoxStream<'static, Result>, + }, + /// Terminal state. + Done, +} + struct MergeStream { - input: Arc, + morselizer: Arc, context: Arc, - next_partition: Arc, - total_input_partitions: usize, - current_stream: Option, + queue: Arc>>, + state: MergeStreamState, schema: SchemaRef, baseline_metrics: BaselineMetrics, - done: bool, } impl Stream for MergeStream { @@ -273,46 +455,73 @@ impl Stream for MergeStream { ) -> Poll> { let this = &mut *self; - if this.done { - return Poll::Ready(None); - } - loop { - // If we have a current stream, poll it - if let Some(stream) = &mut this.current_stream { - let poll = stream.as_mut().poll_next(cx); - match ready!(poll) { - Some(Ok(batch)) => { - this.baseline_metrics.record_output(batch.num_rows()); - return Poll::Ready(Some(Ok(batch))); - } - Some(Err(e)) => { - this.done = true; - return Poll::Ready(Some(Err(e))); - } - None => { - // Current stream exhausted, drop it and try next - this.current_stream = None; + match &mut this.state { + MergeStreamState::Idle => { + // Pop next work item from the shared queue + let item = this.queue.lock().unwrap().pop_front(); + match item { + Some(item) => { + match this + .morselizer + .execute_item(item, Arc::clone(&this.context)) + { + Ok(future) => { + this.state = MergeStreamState::Opening { future }; + } + Err(e) => { + this.state = MergeStreamState::Done; + return Poll::Ready(Some(Err(e))); + } + } + } + None => { + this.state = MergeStreamState::Done; + return Poll::Ready(None); + } } } - } - // Claim next input partition atomically - let partition_idx = this.next_partition.fetch_add(1, Ordering::Relaxed); - if partition_idx >= this.total_input_partitions { - this.done = true; - return Poll::Ready(None); - } + MergeStreamState::Opening { future } => { + match ready!(future.as_mut().poll(cx)) { + Ok(result) => { + // Push additional items to the shared queue + if !result.additional_items.is_empty() { + let mut queue = this.queue.lock().unwrap(); + for item in result.additional_items { + queue.push_back(item); + } + } + this.state = MergeStreamState::Scanning { + stream: result.stream, + }; + } + Err(e) => { + this.state = MergeStreamState::Done; + return Poll::Ready(Some(Err(e))); + } + } + } - // Start new input stream (synchronous, lazy — actual I/O on poll) - match this.input.execute(partition_idx, Arc::clone(&this.context)) { - Ok(stream) => { - this.current_stream = Some(stream); - // Loop back to poll the new stream + MergeStreamState::Scanning { stream } => { + match ready!(stream.poll_next_unpin(cx)) { + Some(Ok(batch)) => { + this.baseline_metrics.record_output(batch.num_rows()); + return Poll::Ready(Some(Ok(batch))); + } + Some(Err(e)) => { + this.state = MergeStreamState::Done; + return Poll::Ready(Some(Err(e))); + } + None => { + // Current morsel exhausted, get next + this.state = MergeStreamState::Idle; + } + } } - Err(e) => { - this.done = true; - return Poll::Ready(Some(Err(e))); + + MergeStreamState::Done => { + return Poll::Ready(None); } } } @@ -360,7 +569,7 @@ mod tests { total_rows += rows; } - // 10 partitions × 100 rows each = 1000 total rows + // 10 partitions x 100 rows each = 1000 total rows assert_eq!(total_rows, 1000); Ok(()) @@ -389,7 +598,7 @@ mod tests { total_rows += rows; } - // 3 partitions × 100 rows each = 300 total rows + // 3 partitions x 100 rows each = 300 total rows assert_eq!(total_rows, 300); Ok(()) From 1862d4450869e273b1f18389e94f233b3e1e0c68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 27 Feb 2026 21:53:12 +0100 Subject: [PATCH 53/53] Fix --- .../datasource-parquet/src/morselizer.rs | 203 ++++++++++++++++++ 1 file changed, 203 insertions(+) create mode 100644 datafusion/datasource-parquet/src/morselizer.rs diff --git a/datafusion/datasource-parquet/src/morselizer.rs b/datafusion/datasource-parquet/src/morselizer.rs new file mode 100644 index 0000000000000..3593a24974f3e --- /dev/null +++ b/datafusion/datasource-parquet/src/morselizer.rs @@ -0,0 +1,203 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! [`ParquetMorselizer`] — row-group-level morselizer for parquet files. + +use std::fmt; +use std::sync::Arc; + +use crate::{ParquetAccessPlan, ParquetFileReaderFactory}; + +use arrow::datatypes::SchemaRef; +use datafusion_common::Result; +use datafusion_datasource::PartitionedFile; +use datafusion_datasource::file_stream::FileOpener; +use datafusion_execution::TaskContext; +use datafusion_physical_plan::merge_partitions::{MorselResult, Morselizer, WorkItem}; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; + +use futures::StreamExt; +use futures::future::BoxFuture; +use parquet::arrow::async_reader::AsyncFileReader; + +/// Work item types for the parquet morselizer. +enum ParquetWorkItem { + /// A file that hasn't been morselized yet. Metadata will be read lazily + /// when this item is executed, and the file will be split into row groups. + File(PartitionedFile), + /// A single row group within a file. The `PartitionedFile` carries a + /// `ParquetAccessPlan` extension that selects exactly one row group. + RowGroup(PartitionedFile), +} + +/// A [`Morselizer`] that expands parquet files into row-group-level work items. +/// +/// When a file work item is executed: +/// 1. Metadata is read lazily (first access populates the metadata cache) +/// 2. The file is split into one work item per row group +/// 3. The first row group is executed immediately +/// 4. Remaining row groups are pushed to the shared queue for other streams +/// +/// When a row-group work item is executed, it is opened directly via the +/// opener (metadata comes from the cache). +pub(crate) struct ParquetMorselizer { + /// Files to scan + files: Vec, + /// Opener for creating streams for individual files/row groups + opener: Arc, + /// Factory for creating readers (used for metadata reads) + reader_factory: Arc, + /// Metrics for the reader factory + metrics: ExecutionPlanMetricsSet, + /// Metadata size hint + metadata_size_hint: Option, + /// Output schema + schema: SchemaRef, +} + +impl fmt::Debug for ParquetMorselizer { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("ParquetMorselizer") + .field("num_files", &self.files.len()) + .finish() + } +} + +impl ParquetMorselizer { + pub(crate) fn new( + files: Vec, + opener: Arc, + reader_factory: Arc, + metrics: ExecutionPlanMetricsSet, + metadata_size_hint: Option, + schema: SchemaRef, + ) -> Self { + Self { + files, + opener, + reader_factory, + metrics, + metadata_size_hint, + schema, + } + } +} + +impl Morselizer for ParquetMorselizer { + fn initial_items(&self) -> Vec { + self.files + .iter() + .map(|f| WorkItem::new(ParquetWorkItem::File(f.clone()))) + .collect() + } + + fn execute_item( + &self, + item: WorkItem, + _context: Arc, + ) -> Result>> { + let work_item: ParquetWorkItem = item + .downcast() + .expect("ParquetMorselizer work item should be ParquetWorkItem"); + + match work_item { + ParquetWorkItem::File(file) => self.execute_file_item(file), + ParquetWorkItem::RowGroup(file) => self.execute_row_group_item(file), + } + } + + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +impl ParquetMorselizer { + /// Execute a file-level work item: read metadata, split into row groups. + fn execute_file_item( + &self, + file: PartitionedFile, + ) -> Result>> { + // Create a reader to fetch metadata + let mut reader: Box = + self.reader_factory.create_reader( + 0, // partition index doesn't matter for metadata reads + file.clone(), + self.metadata_size_hint, + &self.metrics, + )?; + let opener = Arc::clone(&self.opener); + + Ok(Box::pin(async move { + // Read metadata (this populates the cache for subsequent reads) + let metadata = reader + .get_metadata(None) + .await + .map_err(|e| datafusion_common::DataFusionError::External(Box::new(e)))?; + let num_row_groups = metadata.num_row_groups(); + + if num_row_groups == 0 { + // No row groups — return an empty stream + let empty: Vec> = vec![]; + return Ok(MorselResult { + stream: futures::stream::iter(empty).boxed(), + additional_items: vec![], + }); + } + + // Create a work item for each row group except the first + let mut additional_items = + Vec::with_capacity(num_row_groups.saturating_sub(1)); + for rg_idx in 1..num_row_groups { + let mut access_plan = ParquetAccessPlan::new_none(num_row_groups); + access_plan.scan(rg_idx); + let mut rg_file = file.clone(); + rg_file.extensions = Some(Arc::new(access_plan)); + additional_items.push(WorkItem::new(ParquetWorkItem::RowGroup(rg_file))); + } + + // Execute the first row group + let mut first_file = file; + let mut access_plan = ParquetAccessPlan::new_none(num_row_groups); + access_plan.scan(0); + first_file.extensions = Some(Arc::new(access_plan)); + + let future = opener.open(first_file)?; + let stream = future.await?; + + Ok(MorselResult { + stream, + additional_items, + }) + })) + } + + /// Execute a row-group-level work item: open the specific row group. + fn execute_row_group_item( + &self, + file: PartitionedFile, + ) -> Result>> { + let future = self.opener.open(file)?; + + Ok(Box::pin(async move { + let stream = future.await?; + Ok(MorselResult { + stream, + additional_items: vec![], + }) + })) + } +}