diff --git a/Cargo.lock b/Cargo.lock index 8101e2fe22d9..105cec152b46 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1970,6 +1970,7 @@ dependencies = [ "flate2", "futures", "glob", + "insta", "itertools 0.14.0", "liblzma", "log", 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 35f36ea8bc0c..4804586382dc 100644 --- a/datafusion-examples/examples/custom_data_source/csv_json_opener.rs +++ b/datafusion-examples/examples/custom_data_source/csv_json_opener.rs @@ -27,7 +27,9 @@ use datafusion::{ file_format::file_compression_type::FileCompressionType, listing::PartitionedFile, object_store::ObjectStoreUrl, - physical_plan::{CsvSource, FileSource, FileStream, JsonOpener, JsonSource}, + physical_plan::{ + CsvSource, FileSource, FileStreamBuilder, JsonOpener, JsonSource, + }, }, error::Result, physical_plan::metrics::ExecutionPlanMetricsSet, @@ -81,7 +83,8 @@ async fn csv_opener() -> Result<()> { let mut result = vec![]; let mut stream = - FileStream::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new())?; + FileStreamBuilder::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new()) + .build()?; while let Some(batch) = stream.next().await.transpose()? { result.push(batch); } @@ -137,12 +140,13 @@ async fn json_opener() -> Result<()> { .with_file(PartitionedFile::new(path.to_string(), 10)) .build(); - let mut stream = FileStream::new( + let mut stream = FileStreamBuilder::new( &scan_config, 0, Arc::new(opener), &ExecutionPlanMetricsSet::new(), - )?; + ) + .build()?; let mut result = vec![]; while let Some(batch) = stream.next().await.transpose()? { result.push(batch); diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 04c8ea129d05..8e4855afa66b 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -49,5 +49,5 @@ pub use datafusion_datasource::file_scan_config::{ pub use datafusion_datasource::file_sink_config::*; pub use datafusion_datasource::file_stream::{ - FileOpenFuture, FileOpener, FileStream, OnError, + FileOpenFuture, FileOpener, FileStream, FileStreamBuilder, OnError, }; diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index ca4d097c37a4..8e5f55277e00 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -16,9 +16,10 @@ // under the License. use crate::sort::reverse_row_selection; -use datafusion_common::{Result, assert_eq_or_internal_err}; +use datafusion_common::{Result, assert_eq_or_internal_err, internal_datafusion_err}; use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; use parquet::file::metadata::{ParquetMetaData, RowGroupMetaData}; +use std::collections::VecDeque; /// A selection of rows and row groups within a ParquetFile to decode. /// @@ -396,6 +397,80 @@ impl PreparedAccessPlan { Ok(self) } + + /// Split this access plan into one plan per selected row group. + /// + /// The returned plans preserve the current row-group ordering. If + /// `row_selection` is present, it is partitioned so each returned plan + /// contains only the selection entries for its single row group. + pub(crate) fn into_single_row_group_plans( + self, + file_metadata: &ParquetMetaData, + ) -> Result> { + let Self { + row_group_indexes, + row_selection, + } = self; + + let Some(row_selection) = row_selection else { + return Ok(row_group_indexes + .into_iter() + .map(|row_group_index| Self { + row_group_indexes: vec![row_group_index], + row_selection: None, + }) + .collect()); + }; + + let mut selectors: VecDeque = + Vec::::from(row_selection).into(); + let mut plans = Vec::with_capacity(row_group_indexes.len()); + + for row_group_index in row_group_indexes { + let mut remaining_rows = + file_metadata.row_groups()[row_group_index].num_rows() as usize; + let mut row_group_selectors = Vec::new(); + + while remaining_rows > 0 { + let selector = selectors.pop_front().ok_or_else(|| { + internal_datafusion_err!( + "PreparedAccessPlan row selection ended before row group {row_group_index} was fully described" + ) + })?; + + let rows_for_group = selector.row_count.min(remaining_rows); + row_group_selectors.push(if selector.skip { + RowSelector::skip(rows_for_group) + } else { + RowSelector::select(rows_for_group) + }); + + if selector.row_count > rows_for_group { + let remaining_selector_rows = selector.row_count - rows_for_group; + selectors.push_front(if selector.skip { + RowSelector::skip(remaining_selector_rows) + } else { + RowSelector::select(remaining_selector_rows) + }); + } + + remaining_rows -= rows_for_group; + } + + plans.push(Self { + row_group_indexes: vec![row_group_index], + row_selection: Some(row_group_selectors.into()), + }); + } + + if !selectors.is_empty() { + return Err(internal_datafusion_err!( + "PreparedAccessPlan row selection had leftover selectors after splitting by row group" + )); + } + + Ok(plans) + } } #[cfg(test)] diff --git a/datafusion/datasource-parquet/src/mod.rs b/datafusion/datasource-parquet/src/mod.rs index 0e137a706fad..4d46f84aa816 100644 --- a/datafusion/datasource-parquet/src/mod.rs +++ b/datafusion/datasource-parquet/src/mod.rs @@ -37,6 +37,7 @@ mod writer; pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; pub use file_format::*; pub use metrics::ParquetFileMetrics; +pub use opener::ParquetMorselizer; pub use page_filter::PagePruningAccessPlanFilter; pub use reader::*; // Expose so downstream crates can use it pub use row_filter::build_row_filter; diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index bb330c3f4caa..9f17184b1937 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! [`ParquetOpener`] for opening Parquet files +//! [`ParquetMorselizer`] for morselizing Parquet files use crate::page_filter::PagePruningAccessPlanFilter; use crate::row_group_filter::RowGroupAccessPlanFilter; @@ -25,12 +25,17 @@ use crate::{ }; use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::{DataType, Schema}; -use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; +use datafusion_datasource::morsel::{Morsel, MorselPlan, MorselPlanner, Morselizer}; use datafusion_physical_expr::projection::{ProjectionExprs, Projector}; use datafusion_physical_expr::utils::reassign_expr_columns; use datafusion_physical_expr_adapter::replace_columns_with_literals; use parquet::errors::ParquetError; use std::collections::HashMap; +use std::collections::VecDeque; +use std::fmt::Debug; +use std::future::Future; +use std::mem; +use std::ops::Deref; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -56,7 +61,7 @@ use datafusion_pruning::{FilePruner, PruningPredicate, build_pruning_predicate}; use datafusion_common::config::EncryptionFactoryOptions; #[cfg(feature = "parquet_encryption")] use datafusion_execution::parquet_encryption::EncryptionFactory; -use futures::{Stream, StreamExt, ready}; +use futures::{FutureExt, Stream, StreamExt, ready, stream::BoxStream}; use log::debug; use parquet::DecodeResult; use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; @@ -67,9 +72,28 @@ use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::push_decoder::{ParquetPushDecoder, ParquetPushDecoderBuilder}; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::file::metadata::{PageIndexPolicy, ParquetMetaDataReader}; +use tokio::sync::oneshot; -/// Implements [`FileOpener`] for a parquet file -pub(super) struct ParquetOpener { +/// Implements [`Morselizer`] for a parquet file. +/// +/// The current implementation preserves parity with the existing opener path: +/// +/// 1. `morselize` creates a single planner for the input file +/// 2. the planner's first `plan` call returns an I/O future +/// 3. that future runs the copied parquet open/setup flow: +/// file pruning, metadata loading, optional page-index / bloom-filter work, +/// row-group pruning, decoder construction, and final stream setup +/// 4. the next `plan` call emits a single ready morsel wrapping that prepared stream +/// +/// This keeps the behavioral parity of `opener.rs` while routing execution +/// through the new `Morselizer` / `MorselPlanner` API. +#[derive(Clone)] +pub struct ParquetMorselizer { + state: Arc, +} + +/// State needed to plan Parquet morsels +pub struct ParquetMorselizerState { /// Execution partition index pub(crate) partition_index: usize, /// Projection to apply on top of the table schema (i.e. can reference partition columns). @@ -91,8 +115,8 @@ pub(super) struct ParquetOpener { pub metrics: ExecutionPlanMetricsSet, /// Factory for instantiating parquet reader pub parquet_file_reader_factory: Arc, - /// Should the filters be evaluated during the parquet scan using - /// [`DataFusionArrowPredicate`](row_filter::DatafusionArrowPredicate)? + /// Should the filters be evaluated during the parquet scan using the + /// parquet row-filter predicate machinery? pub pushdown_filters: bool, /// Should the filters be reordered to optimize the scan? pub reorder_filters: bool, @@ -108,15 +132,10 @@ pub(super) struct ParquetOpener { pub enable_row_group_stats_pruning: bool, /// Coerce INT96 timestamps to specific TimeUnit pub coerce_int96: Option, - /// Optional parquet FileDecryptionProperties - #[cfg(feature = "parquet_encryption")] - pub file_decryption_properties: Option>, /// Rewrite expressions in the context of the file schema pub(crate) expr_adapter_factory: Arc, - /// Optional factory to create file decryption properties dynamically - #[cfg(feature = "parquet_encryption")] - pub encryption_factory: - Option<(Arc, EncryptionFactoryOptions)>, + /// Encryption configuration used to resolve per-file decryption properties. + pub(crate) encryption_context: EncryptionContext, /// Maximum size of the predicate cache, in bytes. If none, uses /// the arrow-rs default. pub max_predicate_cache_size: Option, @@ -124,15 +143,110 @@ pub(super) struct ParquetOpener { pub reverse_row_groups: bool, } -impl FileOpener for ParquetOpener { - fn open(&self, partitioned_file: PartitionedFile) -> Result { +impl ParquetMorselizer { + pub(crate) fn new(state: ParquetMorselizerState) -> Self { + Self { + state: Arc::new(state), + } + } +} + +impl Deref for ParquetMorselizer { + type Target = ParquetMorselizerState; + + fn deref(&self) -> &Self::Target { + &self.state + } +} + +impl Debug for ParquetMorselizer { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("ParquetMorselizer") + .field("partition_index", &self.partition_index) + .field("batch_size", &self.batch_size) + .field("limit", &self.limit) + .field("preserve_order", &self.preserve_order) + .field("metadata_size_hint", &self.metadata_size_hint) + .field("pushdown_filters", &self.pushdown_filters) + .field("reorder_filters", &self.reorder_filters) + .field("force_filter_selections", &self.force_filter_selections) + .field("enable_page_index", &self.enable_page_index) + .field("enable_bloom_filter", &self.enable_bloom_filter) + .field( + "enable_row_group_stats_pruning", + &self.enable_row_group_stats_pruning, + ) + .field("coerce_int96", &self.coerce_int96) + .field("max_predicate_cache_size", &self.max_predicate_cache_size) + .field("reverse_row_groups", &self.reverse_row_groups) + .finish() + } +} + +/// Result of preparing a PartitionedFile using CPU before any I/O. +/// +/// This captures the state computed from `PartitionedFile`, the table schema, +/// and scan configuration so that later planner states only need to perform +/// async work such as metadata loading and stream construction. +struct PreparedParquetOpen { + state: Arc, + partitioned_file: PartitionedFile, + file_range: Option, + extensions: Option>, + file_metrics: ParquetFileMetrics, + file_pruner: Option, + metadata_size_hint: Option, + async_file_reader: Box, + logical_file_schema: SchemaRef, + output_schema: Arc, + projection: ProjectionExprs, + predicate: Option>, + #[cfg(feature = "parquet_encryption")] + file_decryption_properties: Option>, +} + +/// Result of loading parquet metadata after file-level pruning has completed. +struct MetadataLoadedParquetOpen { + prepared: PreparedParquetOpen, + reader_metadata: ArrowReaderMetadata, + options: ArrowReaderOptions, +} + +/// Result of CPU-only preparation after metadata has been loaded. +/// +/// This captures the file schema coercions and file-specific pruning predicates +/// so the next async step only has to fetch any missing page index data. +struct FiltersPreparedParquetOpen { + loaded: MetadataLoadedParquetOpen, + physical_file_schema: SchemaRef, + projection: ProjectionExprs, + predicate: Option>, + pruning_predicate: Option>, + page_pruning_predicate: Option>, +} + +/// Result of CPU-only row-group pruning using parquet metadata. +/// +/// This captures the row groups that remain after range, statistics, and +/// limit-based pruning so the next async step can optionally load and apply +/// bloom filters before the final stream is built. +struct RowGroupsPreparedParquetOpen { + prepared: FiltersPreparedParquetOpen, + row_groups: RowGroupAccessPlanFilter, +} + +impl ParquetMorselizerState { + /// Perform the CPU-only setup for opening a parquet file. + fn prepare_open_file( + self: &Arc, + partitioned_file: PartitionedFile, + ) -> Result { // ----------------------------------- // Step: prepare configurations, etc. // ----------------------------------- let file_range = partitioned_file.range.clone(); let extensions = partitioned_file.extensions.clone(); - let file_location = partitioned_file.object_meta.location.clone(); - let file_name = file_location.to_string(); + let file_name = partitioned_file.object_meta.location.to_string(); let file_metrics = ParquetFileMetrics::new(self.partition_index, &file_name, &self.metrics); @@ -140,7 +254,7 @@ impl FileOpener for ParquetOpener { .metadata_size_hint .or(self.metadata_size_hint); - let mut async_file_reader: Box = + let async_file_reader: Box = self.parquet_file_reader_factory.create_reader( self.partition_index, partitioned_file.clone(), @@ -148,8 +262,6 @@ impl FileOpener for ParquetOpener { &self.metrics, )?; - let batch_size = self.batch_size; - // Calculate the output schema from the original projection (before literal replacement) // so we get correct field names from column references let logical_file_schema = Arc::clone(self.table_schema.file_schema()); @@ -204,329 +316,444 @@ impl FileOpener for ParquetOpener { .transpose()?; } - let reorder_predicates = self.reorder_filters; - let pushdown_filters = self.pushdown_filters; - let force_filter_selections = self.force_filter_selections; - let coerce_int96 = self.coerce_int96; - let enable_bloom_filter = self.enable_bloom_filter; - let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; - let limit = self.limit; - let parquet_file_reader_factory = Arc::clone(&self.parquet_file_reader_factory); - let partition_index = self.partition_index; - let metrics = self.metrics.clone(); - let predicate_creation_errors = MetricBuilder::new(&self.metrics) .global_counter("num_predicate_creation_errors"); - let expr_adapter_factory = Arc::clone(&self.expr_adapter_factory); - - let enable_page_index = self.enable_page_index; - #[cfg(feature = "parquet_encryption")] - let encryption_context = self.get_encryption_context(); - let max_predicate_cache_size = self.max_predicate_cache_size; - - let reverse_row_groups = self.reverse_row_groups; - let preserve_order = self.preserve_order; - - Ok(Box::pin(async move { + let file_pruner = predicate + .as_ref() + .filter(|p| is_dynamic_physical_expr(p) || partitioned_file.has_statistics()) + .and_then(|p| { + FilePruner::try_new( + Arc::clone(p), + &logical_file_schema, + &partitioned_file, + predicate_creation_errors.clone(), + ) + }); + + Ok(PreparedParquetOpen { + state: Arc::clone(self), + partitioned_file, + file_range, + extensions, + file_metrics, + file_pruner, + metadata_size_hint, + async_file_reader, + logical_file_schema, + output_schema, + projection, + predicate, #[cfg(feature = "parquet_encryption")] - let file_decryption_properties = encryption_context - .get_file_decryption_properties(&file_location) - .await?; - - // --------------------------------------------- - // Step: try to prune the current file partition - // --------------------------------------------- - - // Prune this file using the file level statistics and partition values. - // Since dynamic filters may have been updated since planning it is possible that we are able - // to prune files now that we couldn't prune at planning time. - // It is assumed that there is no point in doing pruning here if the predicate is not dynamic, - // as it would have been done at planning time. - // We'll also check this after every record batch we read, - // and if at some point we are able to prove we can prune the file using just the file level statistics - // we can end the stream early. - let mut file_pruner = predicate - .as_ref() - .filter(|p| { - // Make a FilePruner only if there is either - // 1. a dynamic expr in the predicate - // 2. the file has file-level statistics. - // - // File-level statistics may prune the file without loading - // any row groups or metadata. - // - // Dynamic filters may prune the file after initial - // planning, as the dynamic filter is updated during - // execution. - // - // The case where there is a dynamic filter but no - // statistics corresponds to a dynamic filter that - // references partition columns. While rare, this is possible - // e.g. `select * from table order by partition_col limit - // 10` could hit this condition. - is_dynamic_physical_expr(p) || partitioned_file.has_statistics() - }) - .and_then(|p| { - FilePruner::try_new( - Arc::clone(p), - &logical_file_schema, - &partitioned_file, - predicate_creation_errors.clone(), - ) - }); - - if let Some(file_pruner) = &mut file_pruner - && file_pruner.should_prune()? - { - // Return an empty stream immediately to skip the work of setting up the actual stream - file_metrics.files_ranges_pruned_statistics.add_pruned(1); - return Ok(futures::stream::empty().boxed()); - } - - file_metrics.files_ranges_pruned_statistics.add_matched(1); + file_decryption_properties: None, + }) + } +} - // -------------------------------------------------------- - // Step: fetch Parquet metadata (and optionally page index) - // -------------------------------------------------------- +impl PreparedParquetOpen { + /// CPU-only file pruning performed before metadata I/O begins. + /// + /// Returns `None` if the file was completely pruned. + fn prune_file(mut self) -> Result> { + if let Some(file_pruner) = &mut self.file_pruner + && file_pruner.should_prune()? + { + self.file_metrics + .files_ranges_pruned_statistics + .add_pruned(1); + return Ok(None); + } - // Don't load the page index yet. Since it is not stored inline in - // the footer, loading the page index if it is not needed will do - // unnecessary I/O. We decide later if it is needed to evaluate the - // pruning predicates. Thus default to not requesting it from the - // underlying reader. - let mut options = - ArrowReaderOptions::new().with_page_index_policy(PageIndexPolicy::Skip); - #[cfg(feature = "parquet_encryption")] - if let Some(fd_val) = file_decryption_properties { - options = options.with_file_decryption_properties(Arc::clone(&fd_val)); - } - let mut metadata_timer = file_metrics.metadata_load_time.timer(); - - // 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 = - ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) - .await?; - - // Note about schemas: we are actually dealing with **3 different schemas** here: - // - The table schema as defined by the TableProvider. - // This is what the user sees, what they get when they `SELECT * FROM table`, etc. - // - The logical file schema: this is the table schema minus any hive partition columns and projections. - // This is what the physical file schema is coerced to. - // - The physical file schema: this is the schema that the arrow-rs - // parquet reader will actually produce. - let mut physical_file_schema = Arc::clone(reader_metadata.schema()); - - // The schema loaded from the file may not be the same as the - // desired schema (for example if we want to instruct the parquet - // reader to read strings using Utf8View instead). Update if necessary - if let Some(merged) = apply_file_schema_type_coercions( - &logical_file_schema, - &physical_file_schema, - ) { - physical_file_schema = Arc::new(merged); - options = options.with_schema(Arc::clone(&physical_file_schema)); - reader_metadata = ArrowReaderMetadata::try_new( - Arc::clone(reader_metadata.metadata()), - options.clone(), - )?; - } + self.file_metrics + .files_ranges_pruned_statistics + .add_matched(1); + Ok(Some(self)) + } - if let Some(ref coerce) = coerce_int96 - && let Some(merged) = coerce_int96_to_resolution( - reader_metadata.parquet_schema(), - &physical_file_schema, - coerce, - ) - { - physical_file_schema = Arc::new(merged); - options = options.with_schema(Arc::clone(&physical_file_schema)); - reader_metadata = ArrowReaderMetadata::try_new( - Arc::clone(reader_metadata.metadata()), - options.clone(), - )?; - } + /// Fetch parquet metadata once file-level pruning is complete. + async fn load(mut self) -> Result { + let options = + ArrowReaderOptions::new().with_page_index_policy(PageIndexPolicy::Skip); + #[cfg(feature = "parquet_encryption")] + let mut options = options; + #[cfg(feature = "parquet_encryption")] + if let Some(fd_val) = &self.file_decryption_properties { + options = options.with_file_decryption_properties(Arc::clone(fd_val)); + } + let reader_metadata = { + let mut metadata_timer = self.file_metrics.metadata_load_time.timer(); + let reader_metadata = ArrowReaderMetadata::load_async( + &mut self.async_file_reader, + options.clone(), + ) + .await?; + metadata_timer.stop(); + reader_metadata + }; + Ok(MetadataLoadedParquetOpen { + prepared: self, + reader_metadata, + options, + }) + } +} - // Adapt the projection & filter predicate to the physical file schema. - // This evaluates missing columns and inserts any necessary casts. - // After rewriting to the file schema, further simplifications may be possible. - // For example, if `'a' = col_that_is_missing` becomes `'a' = NULL` that can then be simplified to `FALSE` - // and we can avoid doing any more work on the file (bloom filters, loading the page index, etc.). - // Additionally, if any casts were inserted we can move casts from the column to the literal side: - // `CAST(col AS INT) = 5` can become `col = CAST(5 AS )`, which can be evaluated statically. - let rewriter = expr_adapter_factory.create( - Arc::clone(&logical_file_schema), - Arc::clone(&physical_file_schema), +impl MetadataLoadedParquetOpen { + /// Prepare file-specific filters and schema coercions after metadata is loaded. + fn prepare_filters(self) -> Result { + let MetadataLoadedParquetOpen { + mut prepared, + mut reader_metadata, + mut options, + } = self; + let state = Arc::clone(&prepared.state); + let coerce_int96 = state.coerce_int96; + let predicate_creation_errors = MetricBuilder::new(&state.metrics) + .global_counter("num_predicate_creation_errors"); + let expr_adapter_factory = Arc::clone(&state.expr_adapter_factory); + + // Note about schemas: we are actually dealing with **3 different schemas** here: + // - The table schema as defined by the TableProvider. + // This is what the user sees, what they get when they `SELECT * FROM table`, etc. + // - The logical file schema: this is the table schema minus any hive partition columns and projections. + // This is what the physical file schema is coerced to. + // - The physical file schema: this is the schema that the arrow-rs + // parquet reader will actually produce. + let logical_file_schema = Arc::clone(&prepared.logical_file_schema); + let mut physical_file_schema = Arc::clone(reader_metadata.schema()); + + // The schema loaded from the file may not be the same as the + // desired schema (for example if we want to instruct the parquet + // reader to read strings using Utf8View instead). Update if necessary. + if let Some(merged) = + apply_file_schema_type_coercions(&logical_file_schema, &physical_file_schema) + { + physical_file_schema = Arc::new(merged); + options = options.with_schema(Arc::clone(&physical_file_schema)); + reader_metadata = ArrowReaderMetadata::try_new( + Arc::clone(reader_metadata.metadata()), + options.clone(), )?; - let simplifier = PhysicalExprSimplifier::new(&physical_file_schema); - predicate = predicate - .map(|p| simplifier.simplify(rewriter.rewrite(p)?)) - .transpose()?; - // Adapt projections to the physical file schema as well - projection = projection - .try_map_exprs(|p| simplifier.simplify(rewriter.rewrite(p)?))?; + } - // Build predicates for this specific file - let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates( - predicate.as_ref(), + if let Some(ref coerce) = coerce_int96 + && let Some(merged) = coerce_int96_to_resolution( + reader_metadata.parquet_schema(), &physical_file_schema, - &predicate_creation_errors, - ); + coerce, + ) + { + physical_file_schema = Arc::new(merged); + options = options.with_schema(Arc::clone(&physical_file_schema)); + reader_metadata = ArrowReaderMetadata::try_new( + Arc::clone(reader_metadata.metadata()), + options.clone(), + )?; + } - // 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) { - reader_metadata = load_page_index( - reader_metadata, - &mut async_file_reader, - // Since we're manually loading the page index the option here should not matter but we pass it in for consistency - options.with_page_index_policy(PageIndexPolicy::Optional), - ) - .await?; - } + // Adapt the projection & filter predicate to the physical file schema. + // This evaluates missing columns and inserts any necessary casts. + // After rewriting to the file schema, further simplifications may be possible. + // For example, if `'a' = col_that_is_missing` becomes `'a' = NULL` that can then be simplified to `FALSE` + // and we can avoid doing any more work on the file (bloom filters, loading the page index, etc.). + // Additionally, if any casts were inserted we can move casts from the column to the literal side: + // `CAST(col AS INT) = 5` can become `col = CAST(5 AS )`, which can be evaluated statically. + let rewriter = expr_adapter_factory.create( + Arc::clone(&logical_file_schema), + Arc::clone(&physical_file_schema), + )?; + let simplifier = PhysicalExprSimplifier::new(&physical_file_schema); + prepared.predicate = prepared + .predicate + .map(|p| simplifier.simplify(rewriter.rewrite(p)?)) + .transpose()?; + // Adapt projections to the physical file schema as well + prepared.projection = prepared + .projection + .try_map_exprs(|p| simplifier.simplify(rewriter.rewrite(p)?))?; + + // Build predicates for this specific file + let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates( + prepared.predicate.as_ref(), + &physical_file_schema, + &predicate_creation_errors, + ); - metadata_timer.stop(); + let projection = prepared.projection.clone(); + let predicate = prepared.predicate.clone(); - // ------------------------------------------------------------ - // Step: prune row groups by range, predicate and bloom filter - // ------------------------------------------------------------ - - // Determine which row groups to actually read. The idea is to skip - // as many row groups as possible based on the metadata and query - let file_metadata = Arc::clone(reader_metadata.metadata()); - let pruning_pred = 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); - } + Ok(FiltersPreparedParquetOpen { + loaded: MetadataLoadedParquetOpen { + prepared, + reader_metadata, + options, + }, + physical_file_schema, + projection, + predicate, + pruning_predicate, + page_pruning_predicate, + }) + } +} - // If there is a predicate that can be evaluated against the metadata - if let Some(pruning_pred) = pruning_pred.as_ref() { - if enable_row_group_stats_pruning { - row_groups.prune_by_statistics( - &physical_file_schema, - reader_metadata.parquet_schema(), - rg_metadata, - pruning_pred, - &file_metrics, - ); - } else { - // Update metrics: statistics unavailable, so all row groups are - // matched (not pruned) - file_metrics - .row_groups_pruned_statistics - .add_matched(row_groups.remaining_row_group_count()); - } +impl FiltersPreparedParquetOpen { + /// Fetch the page index if it is needed and missing from the loaded metadata. + async fn load_page_index(mut self) -> Result { + let enable_page_index = self.loaded.prepared.state.enable_page_index; + // The page index is not stored inline in the parquet footer so the + // metadata load 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, &self.page_pruning_predicate) { + self.loaded.reader_metadata = load_page_index( + self.loaded.reader_metadata, + &mut self.loaded.prepared.async_file_reader, + // Since we're manually loading the page index the option here + // should not matter but we pass it in for consistency. + self.loaded + .options + .clone() + .with_page_index_policy(PageIndexPolicy::Optional), + ) + .await?; + } + Ok(self) + } - if enable_bloom_filter && !row_groups.is_empty() { - // Use the existing reader for bloom filter I/O; - // replace with a fresh reader for decoding below. - let bf_reader = std::mem::replace( - &mut async_file_reader, - parquet_file_reader_factory.create_reader( - partition_index, - partitioned_file.clone(), - metadata_size_hint, - &metrics, - )?, - ); - let mut bf_builder = - ParquetRecordBatchStreamBuilder::new_with_metadata( - bf_reader, - reader_metadata.clone(), - ); - row_groups - .prune_by_bloom_filters( - &physical_file_schema, - &mut bf_builder, - pruning_pred, - &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 row groups using file ranges and parquet metadata. + fn prepare_row_groups(self) -> Result { + let loaded = &self.loaded; + let inner = &loaded.prepared; + let state = &inner.state; + + // Determine which row groups to actually read. The idea is to skip + // as many row groups as possible based on the metadata and query. + let file_metadata = Arc::clone(loaded.reader_metadata.metadata()); + let rg_metadata = file_metadata.row_groups(); + let file_name = inner.partitioned_file.object_meta.location.to_string(); + let pruning_pred = self.pruning_predicate.as_ref().map(|p| p.as_ref()); + + let access_plan = + create_initial_plan(&file_name, inner.extensions.clone(), 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) = inner.file_range.as_ref() { + row_groups.prune_by_range(rg_metadata, range); + } + + // If there is a predicate that can be evaluated against the metadata. + if let Some(pruning_pred) = pruning_pred { + if state.enable_row_group_stats_pruning { + row_groups.prune_by_statistics( + &self.physical_file_schema, + loaded.reader_metadata.parquet_schema(), + rg_metadata, + pruning_pred, + &inner.file_metrics, + ); } else { - // Update metrics: no predicate, so all row groups are matched (not pruned) - let n_remaining_row_groups = row_groups.remaining_row_group_count(); - file_metrics + inner + .file_metrics .row_groups_pruned_statistics - .add_matched(n_remaining_row_groups); - file_metrics - .row_groups_pruned_bloom_filter - .add_matched(n_remaining_row_groups); + .add_matched(row_groups.remaining_row_group_count()); } - // 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); + if !state.enable_bloom_filter || row_groups.is_empty() { + inner + .file_metrics + .row_groups_pruned_bloom_filter + .add_matched(row_groups.remaining_row_group_count()); } + } else { + let n_remaining_row_groups = row_groups.remaining_row_group_count(); + inner + .file_metrics + .row_groups_pruned_statistics + .add_matched(n_remaining_row_groups); + inner + .file_metrics + .row_groups_pruned_bloom_filter + .add_matched(n_remaining_row_groups); + } - // -------------------------------------------------------- - // Step: prune pages from the kept row groups - // - let mut access_plan = row_groups.build(); - // page index pruning: if all data on individual pages can - // be ruled using page metadata, rows from other columns - // with that range can be skipped as well - // -------------------------------------------------------- - 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, - reader_metadata.parquet_schema(), - file_metadata.as_ref(), - &file_metrics, - ); - } + Ok(RowGroupsPreparedParquetOpen { + prepared: self, + row_groups, + }) + } +} + +impl RowGroupsPreparedParquetOpen { + /// Apply bloom filter pruning when it is enabled and a pruning predicate exists. + async fn prune_bloom_filters(mut self) -> Result { + let loaded = &mut self.prepared.loaded; + let inner = &mut loaded.prepared; + let state = &inner.state; + let pruning_pred = self.prepared.pruning_predicate.as_ref().map(|p| p.as_ref()); + + if let Some(pruning_pred) = pruning_pred + && state.enable_bloom_filter + && !self.row_groups.is_empty() + { + let bf_reader = mem::replace( + &mut inner.async_file_reader, + state.parquet_file_reader_factory.create_reader( + state.partition_index, + inner.partitioned_file.clone(), + inner.metadata_size_hint, + &state.metrics, + )?, + ); + let mut bf_builder = ParquetRecordBatchStreamBuilder::new_with_metadata( + bf_reader, + loaded.reader_metadata.clone(), + ); + self.row_groups + .prune_by_bloom_filters( + &self.prepared.physical_file_schema, + &mut bf_builder, + pruning_pred, + &inner.file_metrics, + ) + .await; + } - // Prepare the access plan (extract row groups and row selection) - let mut prepared_plan = access_plan.prepare(rg_metadata)?; + Ok(self) + } - // ---------------------------------------------------------- - // Step: potentially reverse the access plan for performance. - // See `ParquetSource::try_pushdown_sort` for the rationale. - // ---------------------------------------------------------- - if reverse_row_groups { - prepared_plan = prepared_plan.reverse(file_metadata.as_ref())?; - } + /// Build one or more parquet reading states once all pruning work is complete. + /// + /// In the common case, this returns one child reader state per selected row + /// group. That lets each row group become its own morsel planner, which + /// exposes more CPU work to `FileStream`. + /// + /// The current implementation keeps a conservative fallback to a single + /// combined reader when the scan has a file-scoped dynamic pruner or a + /// global `LIMIT`, as both of those semantics are currently tracked at the + /// file level rather than the row-group level. + fn build_stream_readers(self) -> Result> { + let RowGroupsPreparedParquetOpen { + prepared, + mut row_groups, + } = self; + let FiltersPreparedParquetOpen { + loaded, + physical_file_schema, + projection, + predicate, + pruning_predicate: _, + page_pruning_predicate, + } = prepared; + let MetadataLoadedParquetOpen { + prepared, + reader_metadata, + options: _, + } = loaded; + let PreparedParquetOpen { + state, + partitioned_file, + file_range: _, + extensions: _, + file_metrics, + mut file_pruner, + metadata_size_hint, + async_file_reader, + logical_file_schema: _, + output_schema, + projection: _, + predicate: _, + #[cfg(feature = "parquet_encryption")] + file_decryption_properties: _, + } = prepared; + + let batch_size = state.batch_size; + let reorder_predicates = state.reorder_filters; + let pushdown_filters = state.pushdown_filters; + let force_filter_selections = state.force_filter_selections; + let limit = state.limit; + let max_predicate_cache_size = state.max_predicate_cache_size; + let reverse_row_groups = state.reverse_row_groups; + let preserve_order = state.preserve_order; + let file_metadata = Arc::clone(reader_metadata.metadata()); + let rg_metadata = file_metadata.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); + } - if prepared_plan.row_group_indexes.is_empty() { - return Ok(futures::stream::empty().boxed()); - } + // -------------------------------------------------------- + // Step: prune pages from the kept row groups + // + // Page index pruning: if all data on individual pages can + // be ruled out using page metadata, rows from other columns + // with that range can be skipped as well. + // -------------------------------------------------------- + let mut access_plan = row_groups.build(); + if !access_plan.is_empty() + && let Some(ref p) = page_pruning_predicate + { + access_plan = p.prune_plan_with_page_index( + access_plan, + &physical_file_schema, + reader_metadata.parquet_schema(), + file_metadata.as_ref(), + &file_metrics, + ); + } + + // Prepare the access plan (extract row groups and row selection). + let mut prepared_plan = access_plan.prepare(rg_metadata)?; + // ---------------------------------------------------------- + // Step: potentially reverse the access plan for performance. + // See `ParquetSource::try_pushdown_sort` for the rationale. + // ---------------------------------------------------------- + if reverse_row_groups { + prepared_plan = prepared_plan.reverse(file_metadata.as_ref())?; + } + + if prepared_plan.row_group_indexes.is_empty() { + return Ok(vec![]); + } + + // Row-group fanout is currently only safe for plain unordered scans. + // Filter pushdown and page-index pruning tests compare full output + // batches across scan modes, and allowing individual row-group planners + // to overtake one another changes the observable row order within a + // single file. Keep those scans on the single-planner path until the + // scheduler has explicit within-file ordering support for child + // planners. + let split_by_row_group = limit.is_none() + && file_pruner.is_none() + && !preserve_order + && predicate.is_none() + && !pushdown_filters + && page_pruning_predicate.is_none(); + let prepared_plans = if split_by_row_group { + prepared_plan.into_single_row_group_plans(file_metadata.as_ref())? + } else { + vec![prepared_plan] + }; + + let mut reusable_reader = Some(async_file_reader); + let mut reading_states = Vec::with_capacity(prepared_plans.len()); + for prepared_plan in prepared_plans { // --------------------------------------------------------- // Step: construct builder for the final RecordBatch stream // --------------------------------------------------------- - let mut builder = ParquetPushDecoderBuilder::new_with_metadata(reader_metadata.clone()) .with_batch_size(batch_size); - // --------------------------------------------------------------------- + // ----------------------------------------------------------------- // Step: optionally add row filter to the builder // - // Row filter is used for late materialization in parquet decoding, see - // `row_filter` for details. - // --------------------------------------------------------------------- - - // Filter pushdown: evaluate predicates during scan + // Row filter is used for late materialization in parquet decoding, + // see `row_filter` for details. + // ----------------------------------------------------------------- if let Some(predicate) = pushdown_filters.then_some(predicate.as_ref()).flatten() { @@ -555,7 +782,6 @@ impl FileOpener for ParquetOpener { builder.with_row_selection_policy(RowSelectionPolicy::Selectors); } - // Apply the prepared plan to the builder if let Some(row_selection) = prepared_plan.row_selection { builder = builder.with_row_selection(row_selection); } @@ -569,7 +795,7 @@ impl FileOpener for ParquetOpener { builder = builder.with_max_predicate_cache_size(max_predicate_cache_size); } - // metrics from the arrow reader itself + // Metrics from the arrow reader itself. let arrow_reader_metrics = ArrowReaderMetrics::enabled(); let indices = projection.column_indices(); @@ -581,53 +807,531 @@ impl FileOpener for ParquetOpener { .with_metrics(arrow_reader_metrics.clone()) .build()?; - let files_ranges_pruned_statistics = - file_metrics.files_ranges_pruned_statistics.clone(); let predicate_cache_inner_records = file_metrics.predicate_cache_inner_records.clone(); let predicate_cache_records = file_metrics.predicate_cache_records.clone(); // Rebase column indices to match the narrowed stream schema. - // The projection expressions have indices based on physical_file_schema, - // but the stream only contains the columns selected by the ProjectionMask. + // The projection expressions have indices based on + // `physical_file_schema`, but the stream only contains the columns + // selected by the `ProjectionMask`. let stream_schema = Arc::new(physical_file_schema.project(&indices)?); let replace_schema = stream_schema != output_schema; let projection = projection + .clone() .try_map_exprs(|expr| reassign_expr_columns(expr, &stream_schema))?; let projector = projection.make_projector(&stream_schema)?; - let stream = futures::stream::unfold( - PushDecoderStreamState { - decoder, - reader: async_file_reader, - projector, - output_schema, - replace_schema, - arrow_reader_metrics, - predicate_cache_inner_records, - predicate_cache_records, - }, - |mut state| async move { - let result = state.transition().await; - result.map(|r| (r, state)) + let push_decoder_state = PushDecoderStreamState { + decoder, + reader: if let Some(reader) = reusable_reader.take() { + reader + } else { + state.parquet_file_reader_factory.create_reader( + state.partition_index, + partitioned_file.clone(), + metadata_size_hint, + &state.metrics, + )? }, - ) - .fuse(); - - // ---------------------------------------------------------------------- - // Step: wrap the stream so a dynamic filter can stop the file scan early - // ---------------------------------------------------------------------- - if let Some(file_pruner) = file_pruner { - let boxed_stream = stream.boxed(); - Ok(EarlyStoppingStream::new( - boxed_stream, + projector, + output_schema: Arc::clone(&output_schema), + replace_schema, + arrow_reader_metrics, + predicate_cache_inner_records, + predicate_cache_records, + }; + + // Keep file-scoped early-stop behavior only on the single-planner + // fallback path. A row-group split path would need a file-shared + // pruner to preserve the exact semantics across child planners. + if let Some(file_pruner) = file_pruner.take() { + reading_states.push(ReadingParquetState::with_early_stop( + push_decoder_state, file_pruner, - files_ranges_pruned_statistics, - ) - .boxed()) + file_metrics.files_ranges_pruned_statistics.clone(), + )); } else { - Ok(stream.boxed()) + reading_states.push(ReadingParquetState::new(push_decoder_state)); } - })) + } + + Ok(reading_states) + } +} + +impl ParquetMorselizerState { + /// Resolve file-specific decryption properties before metadata I/O. + #[cfg(feature = "parquet_encryption")] + async fn load_file_decryption_properties( + self: &ParquetMorselizerState, + file_location: object_store::path::Path, + ) -> Result>> { + let encryption_context = self.get_encryption_context(); + encryption_context + .get_file_decryption_properties(&file_location) + .await + } + + /// Resolve file-specific decryption properties before metadata I/O. + #[cfg(not(feature = "parquet_encryption"))] + #[expect(dead_code)] + async fn load_file_decryption_properties( + self: &ParquetMorselizerState, + _file_location: object_store::path::Path, + ) -> Result>> { + Ok(None) + } +} + +impl Morselizer for ParquetMorselizer { + fn morselize(&self, file: PartitionedFile) -> Result>> { + Ok(vec![Box::new(ParquetMorselPlanner::new( + Arc::clone(&self.state), + file, + ))]) + } +} + +/// CPU-only states for [`ParquetMorselPlanner`]. +/// +/// These are the states when the MorselPlanner has more CPU work to do +enum ReadyState { + /// Planner has not started any work yet. + Start(Box), + /// Planner is ready to resolve any file-specific encryption properties. + #[cfg(feature = "parquet_encryption")] + PrepareEncryption(Box), + /// Planner can do file-level pruning before requesting parquet metadata. + PruneFiles(Box), + /// Planner has loaded parquet metadata and can do CPU-only filter preparation. + PrepareFilters(Box), + /// Planner has prepared filters and can request any missing page index data. + Prepared(Box), + /// Planner has prepared row-group pruning and can optionally load bloom filters. + BuildStream(Box), + /// Planner has one or more per-row-group reading states ready to turn into + /// the current planner plus any child planners. + FanoutRowGroups(Vec), + /// Planner has a prepared push decoder and is trying to produce the first + /// record batch before yielding a morsel. + ReadingParquet(Box), + /// Planner has a fully prepared morsel ready to emit. + EmitMorsel(BoxStream<'static, Result>), +} + +impl ReadyState { + fn start(file: PartitionedFile) -> Self { + Self::Start(Box::new(file)) + } + + #[cfg(feature = "parquet_encryption")] + fn prepare_encryption(prepared: PreparedParquetOpen) -> Self { + Self::PrepareEncryption(Box::new(prepared)) + } + + fn prune_files(prepared: PreparedParquetOpen) -> Self { + Self::PruneFiles(Box::new(prepared)) + } + + fn prepared(prepared: MetadataLoadedParquetOpen) -> Self { + Self::PrepareFilters(prepared.into()) + } + + fn filters_prepared(prepared: FiltersPreparedParquetOpen) -> Self { + Self::Prepared(prepared.into()) + } + + fn build_stream(prepared: RowGroupsPreparedParquetOpen) -> Self { + Self::BuildStream(prepared.into()) + } + + fn fanout_row_groups(states: Vec) -> Self { + Self::FanoutRowGroups(states) + } + + fn reading_parquet(state: ReadingParquetState) -> Self { + Self::ReadingParquet(Box::new(state)) + } + + fn emit_morsel(stream: BoxStream<'static, Result>) -> Self { + Self::EmitMorsel(stream) + } +} + +/// Scheduler-visible state for [`ParquetMorselPlanner`]. +/// +/// This allows tracking outstanding IOs +enum ParquetMorselPlannerState { + /// Planner can make progress using CPU only. + Ready(Box), + /// Planner has outstanding async I/O and will become ready again when it completes. + WaitingIo(WaitingIoState), + /// Planner has emitted its morsel and has no further work. + Done, +} + +impl ParquetMorselPlannerState { + fn ready(ready_state: ReadyState) -> Self { + Self::Ready(Box::new(ready_state)) + } + + /// Return a planner state that emits an empty morsel stream. + /// + /// This is used when file-level pruning determines the file can be skipped + /// before any parquet metadata or row-group work is needed, while still + /// flowing through the normal morsel emission path in `FileStream`. + fn empty_file() -> Self { + Self::ready(ReadyState::emit_morsel(futures::stream::empty().boxed())) + } +} + +/// Result of an in-flight planner I/O phase. +struct WaitingIoState { + /// Waiting for an async step to produce the next CPU-ready planner state. + receiver: oneshot::Receiver>, +} + +impl ParquetMorselPlannerState { + fn name(&self) -> &'static str { + match self { + Self::Ready(ready_state) => match ready_state.as_ref() { + ReadyState::Start(_) => "Ready(Start)", + #[cfg(feature = "parquet_encryption")] + ReadyState::PrepareEncryption(_) => "Ready(PrepareEncryption)", + ReadyState::PruneFiles(_) => "Ready(PruneFiles)", + ReadyState::PrepareFilters(_) => "Ready(PrepareFilters)", + ReadyState::Prepared(_) => "Ready(Prepared)", + ReadyState::BuildStream(_) => "Ready(BuildStream)", + ReadyState::FanoutRowGroups(_) => "Ready(FanoutRowGroups)", + ReadyState::ReadingParquet(_) => "Ready(ReadingParquet)", + ReadyState::EmitMorsel(_) => "Ready(EmitMorsel)", + }, + Self::WaitingIo(_) => "WaitingIo", + Self::Done => "Done", + } + } +} + +/// Planner wrapper that exposes the copied opener logic through the generic +/// morsel-planning API. +struct ParquetMorselPlanner { + morselizer: Arc, + state: ParquetMorselPlannerState, +} + +impl Debug for ParquetMorselPlanner { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("ParquetMorselPlanner") + .field("morselizer", &"...") + .field("state", &self.state.name()) + .finish() + } +} + +impl ParquetMorselPlanner { + fn new(morselizer: Arc, file: PartitionedFile) -> Self { + Self { + morselizer, + state: ParquetMorselPlannerState::ready(ReadyState::start(file)), + } + } + + fn with_ready_state( + morselizer: Arc, + ready_state: ReadyState, + ) -> Self { + Self { + morselizer, + state: ParquetMorselPlannerState::ready(ready_state), + } + } + + #[cfg(feature = "parquet_encryption")] + fn needs_file_decryption_properties(&self) -> bool { + self.morselizer + .encryption_context + .needs_file_decryption_properties() + } + + /// Schedule an async step and transition the planner into `WaitingIo`. + /// + /// Sets `self.state` to ParquetMorselPlannerState::WaitingIo + fn schedule_io(&mut self, future: F) -> Option + where + F: Future> + Send + 'static, + { + let (sender, receiver) = oneshot::channel(); + let io_future = async move { + let next_state = future.await?; + // Ignore error as it means the receiver shutdown (likely due to a + // real error) and we don't want to confuse error reporting by + // reporting a closed channel. + let _ = sender.send(Ok(next_state)); + Ok(()) + }; + self.state = ParquetMorselPlannerState::WaitingIo(WaitingIoState { receiver }); + Some(MorselPlan::new().with_io_future(io_future.boxed())) + } + + /// Drive the initial push-decoder loop until it either needs more I/O or + /// can yield a stream with at least its first batch ready. + /// + /// This relies on the push decoder not doing IO after it has begun to produce + /// RecordBatches, which is only true when reading a single record batch + fn prepare_reading_parquet( + &mut self, + reading: ReadingParquetState, + ) -> Result> { + let ReadingParquetState { + mut state, + file_pruner, + files_ranges_pruned_statistics, + } = reading; + + match state.decoder.try_decode() { + Ok(DecodeResult::NeedsData(ranges)) => Ok(self.schedule_io(async move { + let data = state.reader.get_byte_ranges(ranges.clone()).await?; + state.decoder.push_ranges(ranges, data)?; + Ok(ReadyState::reading_parquet(ReadingParquetState { + state, + file_pruner, + files_ranges_pruned_statistics, + })) + })), + Ok(DecodeResult::Data(batch)) => { + state.copy_arrow_reader_metrics(); + let batch = state.project_batch(&batch)?; + let stream = ReadingParquetState { + state, + file_pruner, + files_ranges_pruned_statistics, + } + .into_stream(Some(batch)); + Ok(Some(MorselPlan::new().with_morsels(vec![Box::new( + ParquetStreamMorsel::new(stream), + )]))) + } + Ok(DecodeResult::Finished) => { + let stream = ReadingParquetState { + state, + file_pruner, + files_ranges_pruned_statistics, + } + .into_stream(None); + Ok(Some(MorselPlan::new().with_morsels(vec![Box::new( + ParquetStreamMorsel::new(stream), + )]))) + } + Err(e) => Err(DataFusionError::from(e)), + } + } +} + +impl MorselPlanner for ParquetMorselPlanner { + fn plan(&mut self) -> Result> { + // Core state machine transition + let state = mem::replace(&mut self.state, ParquetMorselPlannerState::Done); + match state { + ParquetMorselPlannerState::Ready(ready_state) => match *ready_state { + ReadyState::Start(file) => { + let prepared = self.morselizer.prepare_open_file(*file)?; + #[cfg(feature = "parquet_encryption")] + { + if self.needs_file_decryption_properties() { + self.state = ParquetMorselPlannerState::ready( + ReadyState::prepare_encryption(prepared), + ); + } else { + self.state = ParquetMorselPlannerState::ready( + ReadyState::prune_files(prepared), + ); + } + } + #[cfg(not(feature = "parquet_encryption"))] + { + self.state = ParquetMorselPlannerState::ready( + ReadyState::prune_files(prepared), + ); + } + Ok(Some(MorselPlan::new())) + } + #[cfg(feature = "parquet_encryption")] + ReadyState::PrepareEncryption(mut prepared) => { + let file_location = + prepared.partitioned_file.object_meta.location.clone(); + let state = Arc::clone(&prepared.state); + Ok(self.schedule_io(async move { + let properties = + state.load_file_decryption_properties(file_location).await?; + prepared.file_decryption_properties = properties; + Ok(ReadyState::prune_files(*prepared)) + })) + } + ReadyState::PruneFiles(prepared) => { + let Some(prepared) = prepared.prune_file()? else { + // File was totally pruned + self.state = ParquetMorselPlannerState::empty_file(); + return Ok(Some(MorselPlan::new())); + }; + Ok(self.schedule_io(async move { + let loaded = prepared.load().await?; + Ok(ReadyState::prepared(loaded)) + })) + } + ReadyState::PrepareFilters(prepared) => { + let prepared = prepared.prepare_filters()?; + self.state = ParquetMorselPlannerState::ready( + ReadyState::filters_prepared(prepared), + ); + Ok(Some(MorselPlan::new())) + } + ReadyState::Prepared(prepared) => Ok(self.schedule_io(async move { + let prepared = prepared.load_page_index().await?; + let prepared = prepared.prepare_row_groups()?; + Ok(ReadyState::build_stream(prepared)) + })), + ReadyState::BuildStream(prepared) => { + let should_prune_bloom = prepared + .prepared + .pruning_predicate + .is_some() + && prepared.prepared.loaded.prepared.state.enable_bloom_filter + && !prepared.row_groups.is_empty(); + if should_prune_bloom { + Ok(self.schedule_io(async move { + let prepared = prepared.prune_bloom_filters().await?; + let reading_states = prepared.build_stream_readers()?; + Ok(ReadyState::fanout_row_groups(reading_states)) + })) + } else { + let reading_states = prepared.build_stream_readers()?; + self.state = ParquetMorselPlannerState::ready( + ReadyState::fanout_row_groups(reading_states), + ); + Ok(Some(MorselPlan::new())) + } + } + ReadyState::FanoutRowGroups(reading_states) => { + let mut reading_states: VecDeque<_> = reading_states.into(); + let Some(first_state) = reading_states.pop_front() else { + self.state = ParquetMorselPlannerState::empty_file(); + return Ok(Some(MorselPlan::new())); + }; + + let child_planners = reading_states + .into_iter() + .map(|reading_state| { + Box::new(ParquetMorselPlanner::with_ready_state( + Arc::clone(&self.morselizer), + ReadyState::reading_parquet(reading_state), + )) as Box + }) + .collect(); + + self.state = ParquetMorselPlannerState::ready( + ReadyState::reading_parquet(first_state), + ); + Ok(Some(MorselPlan::new().with_planners(child_planners))) + } + ReadyState::ReadingParquet(reading) => { + self.prepare_reading_parquet(*reading) + } + ReadyState::EmitMorsel(stream) => Ok(Some( + MorselPlan::new() + .with_morsels(vec![Box::new(ParquetStreamMorsel::new(stream))]), + )), + }, + ParquetMorselPlannerState::WaitingIo(WaitingIoState { mut receiver }) => { + match receiver.try_recv() { + Ok(next_state) => { + self.state = ParquetMorselPlannerState::ready(next_state?); + Ok(Some(MorselPlan::new())) + } + Err(oneshot::error::TryRecvError::Empty) => { + self.state = + ParquetMorselPlannerState::WaitingIo(WaitingIoState { + receiver, + }); + Ok(None) + } + Err(oneshot::error::TryRecvError::Closed) => { + Err(DataFusionError::Execution( + "Parquet morsel planner I/O completion channel closed" + .to_string(), + )) + } + } + } + ParquetMorselPlannerState::Done => Ok(None), + } + } +} + +struct ReadingParquetState { + state: PushDecoderStreamState, + file_pruner: Option, + files_ranges_pruned_statistics: Option, +} + +impl ReadingParquetState { + fn new(state: PushDecoderStreamState) -> Self { + Self { + state, + file_pruner: None, + files_ranges_pruned_statistics: None, + } + } + + fn with_early_stop( + state: PushDecoderStreamState, + file_pruner: FilePruner, + files_ranges_pruned_statistics: PruningMetrics, + ) -> Self { + Self { + state, + file_pruner: Some(file_pruner), + files_ranges_pruned_statistics: Some(files_ranges_pruned_statistics), + } + } + + fn into_stream( + self, + first_batch: Option, + ) -> BoxStream<'static, Result> { + let stream = stream_from_push_decoder_state(self.state, first_batch).boxed(); + wrap_stream_with_early_stop( + stream, + self.file_pruner, + self.files_ranges_pruned_statistics, + ) + } +} + +struct ParquetStreamMorsel { + stream: BoxStream<'static, Result>, +} + +impl Debug for ParquetStreamMorsel { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("ParquetStreamMorsel") + .field("stream", &"...") + .finish() + } +} + +impl ParquetStreamMorsel { + fn new(stream: BoxStream<'static, Result>) -> Self { + Self { stream } + } +} + +impl Morsel for ParquetStreamMorsel { + fn into_stream(self: Box) -> BoxStream<'static, Result> { + self.stream + } + + fn split(&mut self) -> Result>> { + Ok(vec![]) } } @@ -710,6 +1414,34 @@ impl PushDecoderStreamState { } } +fn stream_from_push_decoder_state( + state: PushDecoderStreamState, + first_batch: Option, +) -> impl Stream> + Send + 'static { + let first = first_batch + .map(|batch| futures::stream::once(async move { Ok(batch) }).left_stream()) + .unwrap_or_else(|| futures::stream::empty().right_stream()); + + first.chain(futures::stream::unfold(state, |mut state| async move { + let result = state.transition().await; + result.map(|r| (r, state)) + })) +} + +fn wrap_stream_with_early_stop( + stream: BoxStream<'static, Result>, + file_pruner: Option, + files_ranges_pruned_statistics: Option, +) -> BoxStream<'static, Result> { + match (file_pruner, files_ranges_pruned_statistics) { + (Some(file_pruner), Some(files_ranges_pruned_statistics)) => { + EarlyStoppingStream::new(stream, file_pruner, files_ranges_pruned_statistics) + .boxed() + } + _ => stream, + } +} + type ConstantColumns = HashMap; /// Extract constant column values from statistics, keyed by column name in the logical file schema. @@ -851,8 +1583,8 @@ where } } -#[derive(Default)] -struct EncryptionContext { +#[derive(Default, Clone)] +pub(crate) struct EncryptionContext { #[cfg(feature = "parquet_encryption")] file_decryption_properties: Option>, #[cfg(feature = "parquet_encryption")] @@ -861,7 +1593,11 @@ struct EncryptionContext { #[cfg(feature = "parquet_encryption")] impl EncryptionContext { - fn new( + fn needs_file_decryption_properties(&self) -> bool { + self.file_decryption_properties.is_some() || self.encryption_factory.is_some() + } + + pub(crate) fn new( file_decryption_properties: Option>, encryption_factory: Option<( Arc, @@ -895,6 +1631,10 @@ impl EncryptionContext { #[cfg(not(feature = "parquet_encryption"))] #[expect(dead_code)] impl EncryptionContext { + fn needs_file_decryption_properties(&self) -> bool { + false + } + async fn get_file_decryption_properties( &self, _file_location: &object_store::path::Path, @@ -903,19 +1643,16 @@ impl EncryptionContext { } } -impl ParquetOpener { +impl ParquetMorselizerState { #[cfg(feature = "parquet_encryption")] fn get_encryption_context(&self) -> EncryptionContext { - EncryptionContext::new( - self.file_decryption_properties.clone(), - self.encryption_factory.clone(), - ) + self.encryption_context.clone() } #[cfg(not(feature = "parquet_encryption"))] #[expect(dead_code)] fn get_encryption_context(&self) -> EncryptionContext { - EncryptionContext::default() + self.encryption_context.clone() } } @@ -1030,17 +1767,25 @@ fn should_enable_page_index( #[cfg(test)] mod test { + use std::collections::VecDeque; use std::sync::Arc; - use super::{ConstantColumns, constant_columns_from_stats}; - use crate::{DefaultParquetFileReaderFactory, RowGroupAccess, opener::ParquetOpener}; + use super::{ + ConstantColumns, EncryptionContext, ParquetMorselizerState, + constant_columns_from_stats, + }; + use crate::{DefaultParquetFileReaderFactory, ParquetMorselizer, RowGroupAccess}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use arrow::record_batch::RecordBatch; use bytes::{BufMut, BytesMut}; use datafusion_common::{ ColumnStatistics, DataFusionError, ScalarValue, Statistics, record_batch, stats::Precision, }; - use datafusion_datasource::{PartitionedFile, TableSchema, file_stream::FileOpener}; + use datafusion_datasource::{ + PartitionedFile, TableSchema, + morsel::{MorselPlanner, Morselizer}, + }; use datafusion_expr::{col, lit}; use datafusion_physical_expr::{ PhysicalExpr, @@ -1057,9 +1802,9 @@ mod test { use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; - /// Builder for creating [`ParquetOpener`] instances with sensible defaults for tests. + /// Builder for creating [`ParquetMorselizer`] instances with sensible defaults for tests. /// This helps reduce code duplication and makes it clear what differs between test cases. - struct ParquetOpenerBuilder { + struct ParquetMorselizerBuilder { store: Option>, table_schema: Option, partition_index: usize, @@ -1082,7 +1827,7 @@ mod test { preserve_order: bool, } - impl ParquetOpenerBuilder { + impl ParquetMorselizerBuilder { /// Create a new builder with sensible defaults for tests. fn new() -> Self { Self { @@ -1163,17 +1908,17 @@ mod test { self } - /// Build the ParquetOpener instance. + /// Build the ParquetMorselizer instance. /// /// # Panics /// /// Panics if required fields (store, schema/table_schema) are not set. - fn build(self) -> ParquetOpener { + fn build(self) -> ParquetMorselizer { let store = self .store - .expect("ParquetOpenerBuilder: store must be set via with_store()"); + .expect("ParquetMorselizerBuilder: store must be set via with_store()"); let table_schema = self.table_schema.expect( - "ParquetOpenerBuilder: table_schema must be set via with_schema() or with_table_schema()", + "ParquetMorselizerBuilder: table_schema must be set via with_schema() or with_table_schema()", ); let file_schema = Arc::clone(table_schema.file_schema()); @@ -1187,7 +1932,7 @@ mod test { ProjectionExprs::from_indices(&all_indices, &file_schema) }; - ParquetOpener { + ParquetMorselizer::new(ParquetMorselizerState { partition_index: self.partition_index, projection, batch_size: self.batch_size, @@ -1206,15 +1951,12 @@ mod test { enable_bloom_filter: self.enable_bloom_filter, enable_row_group_stats_pruning: self.enable_row_group_stats_pruning, coerce_int96: self.coerce_int96, - #[cfg(feature = "parquet_encryption")] - file_decryption_properties: None, expr_adapter_factory: Arc::new(DefaultPhysicalExprAdapterFactory), - #[cfg(feature = "parquet_encryption")] - encryption_factory: None, + encryption_context: EncryptionContext::default(), max_predicate_cache_size: self.max_predicate_cache_size, reverse_row_groups: self.reverse_row_groups, preserve_order: self.preserve_order, - } + }) } } @@ -1303,10 +2045,7 @@ mod test { async fn count_batches_and_rows( mut stream: std::pin::Pin< - Box< - dyn Stream> - + Send, - >, + Box> + Send>, >, ) -> (usize, usize) { let mut num_batches = 0; @@ -1321,10 +2060,7 @@ mod test { /// Helper to collect all int32 values from the first column of batches async fn collect_int32_values( mut stream: std::pin::Pin< - Box< - dyn Stream> - + Send, - >, + Box> + Send>, >, ) -> Vec { use arrow::array::Array; @@ -1344,10 +2080,38 @@ mod test { values } + async fn open_via_morselizer( + morselizer: &ParquetMorselizer, + file: PartitionedFile, + ) -> std::pin::Pin> + Send>> + { + let mut planners: VecDeque> = + morselizer.morselize(file).unwrap().into(); + let mut streams = Vec::new(); + + while let Some(mut planner) = planners.pop_front() { + while let Some(mut plan) = planner.plan().unwrap() { + if let Some(io_future) = plan.take_io_future() { + io_future.await.unwrap(); + continue; + } + + streams.extend( + plan.take_morsels() + .into_iter() + .map(|morsel| morsel.into_stream()), + ); + planners.extend(plan.take_planners()); + } + } + + futures::stream::iter(streams).flatten().boxed() + } + async fn write_parquet( store: Arc, filename: &str, - batch: arrow::record_batch::RecordBatch, + batch: RecordBatch, ) -> usize { write_parquet_batches(store, filename, vec![batch], None).await } @@ -1356,7 +2120,7 @@ mod test { async fn write_parquet_batches( store: Arc, filename: &str, - batches: Vec, + batches: Vec, props: Option, ) -> usize { let mut out = BytesMut::new().writer(); @@ -1411,7 +2175,7 @@ mod test { )); let make_opener = |predicate| { - ParquetOpenerBuilder::new() + ParquetMorselizerBuilder::new() .with_store(Arc::clone(&store)) .with_schema(Arc::clone(&schema)) .with_projection_indices(&[0, 1]) @@ -1424,7 +2188,7 @@ mod test { let expr = col("a").eq(lit(1)); let predicate = logical2physical(&expr, &schema); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 1); assert_eq!(num_rows, 3); @@ -1433,7 +2197,7 @@ mod test { let expr = col("b").eq(lit(ScalarValue::Float32(Some(5.0)))); let predicate = logical2physical(&expr, &schema); let opener = make_opener(predicate); - let stream = opener.open(file).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1464,7 +2228,7 @@ mod test { vec![Arc::new(Field::new("part", DataType::Int32, false))], ); let make_opener = |predicate| { - ParquetOpenerBuilder::new() + ParquetMorselizerBuilder::new() .with_store(Arc::clone(&store)) .with_table_schema(table_schema_for_opener.clone()) .with_projection_indices(&[0]) @@ -1479,7 +2243,7 @@ mod test { // Otherwise we assume it already happened at the planning stage and won't re-do the work here let predicate = make_dynamic_expr(logical2physical(&expr, &table_schema)); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 1); assert_eq!(num_rows, 3); @@ -1490,7 +2254,7 @@ mod test { // Otherwise we assume it already happened at the planning stage and won't re-do the work here let predicate = make_dynamic_expr(logical2physical(&expr, &table_schema)); let opener = make_opener(predicate); - let stream = opener.open(file).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1533,7 +2297,7 @@ mod test { vec![Arc::new(Field::new("part", DataType::Int32, false))], ); let make_opener = |predicate| { - ParquetOpenerBuilder::new() + ParquetMorselizerBuilder::new() .with_store(Arc::clone(&store)) .with_table_schema(table_schema_for_opener.clone()) .with_projection_indices(&[0]) @@ -1546,7 +2310,7 @@ mod test { let expr = col("part").eq(lit(1)).and(col("b").eq(lit(1.0))); let predicate = logical2physical(&expr, &table_schema); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 1); assert_eq!(num_rows, 3); @@ -1555,7 +2319,7 @@ mod test { let expr = col("part").eq(lit(2)).and(col("b").eq(lit(1.0))); let predicate = logical2physical(&expr, &table_schema); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1564,7 +2328,7 @@ mod test { let expr = col("part").eq(lit(1)).and(col("b").eq(lit(7.0))); let predicate = logical2physical(&expr, &table_schema); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1573,7 +2337,7 @@ mod test { let expr = col("part").eq(lit(2)).and(col("b").eq(lit(7.0))); let predicate = logical2physical(&expr, &table_schema); let opener = make_opener(predicate); - let stream = opener.open(file).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1605,7 +2369,7 @@ mod test { vec![Arc::new(Field::new("part", DataType::Int32, false))], ); let make_opener = |predicate| { - ParquetOpenerBuilder::new() + ParquetMorselizerBuilder::new() .with_store(Arc::clone(&store)) .with_table_schema(table_schema_for_opener.clone()) .with_projection_indices(&[0]) @@ -1619,7 +2383,7 @@ mod test { let expr = col("part").eq(lit(1)).or(col("a").eq(lit(1))); let predicate = logical2physical(&expr, &table_schema); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 1); assert_eq!(num_rows, 3); @@ -1628,7 +2392,7 @@ mod test { let expr = col("part").eq(lit(1)).or(col("a").eq(lit(3))); let predicate = logical2physical(&expr, &table_schema); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 1); assert_eq!(num_rows, 3); @@ -1637,7 +2401,7 @@ mod test { let expr = col("part").eq(lit(2)).or(col("a").eq(lit(1))); let predicate = logical2physical(&expr, &table_schema); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 1); assert_eq!(num_rows, 1); @@ -1646,7 +2410,7 @@ mod test { let expr = col("part").eq(lit(2)).or(col("a").eq(lit(3))); let predicate = logical2physical(&expr, &table_schema); let opener = make_opener(predicate); - let stream = opener.open(file).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1686,7 +2450,7 @@ mod test { vec![Arc::new(Field::new("part", DataType::Int32, false))], ); let make_opener = |predicate| { - ParquetOpenerBuilder::new() + ParquetMorselizerBuilder::new() .with_store(Arc::clone(&store)) .with_table_schema(table_schema_for_opener.clone()) .with_projection_indices(&[0]) @@ -1699,7 +2463,7 @@ mod test { let expr = col("a").eq(lit(42)); let predicate = logical2physical(&expr, &table_schema); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1708,7 +2472,7 @@ mod test { // This allows dynamic filters to prune partitions/files even if they are populated late into execution. let predicate = make_dynamic_expr(logical2physical(&expr, &table_schema)); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1718,7 +2482,7 @@ mod test { let expr = col("part").eq(lit(2)); let predicate = make_dynamic_expr(logical2physical(&expr, &table_schema)); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1727,7 +2491,7 @@ mod test { let expr = col("part").eq(lit(2)).and(col("a").eq(lit(42))); let predicate = make_dynamic_expr(logical2physical(&expr, &table_schema)); let opener = make_opener(predicate); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let (num_batches, num_rows) = count_batches_and_rows(stream).await; assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); @@ -1768,7 +2532,7 @@ mod test { ); let make_opener = |reverse_scan: bool| { - ParquetOpenerBuilder::new() + ParquetMorselizerBuilder::new() .with_store(Arc::clone(&store)) .with_schema(Arc::clone(&schema)) .with_projection_indices(&[0]) @@ -1778,12 +2542,12 @@ mod test { // Test normal scan (forward) let opener = make_opener(false); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let forward_values = collect_int32_values(stream).await; // Test reverse scan let opener = make_opener(true); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let reverse_values = collect_int32_values(stream).await; // The forward scan should return data in the order written @@ -1794,6 +2558,42 @@ mod test { assert_eq!(reverse_values, vec![7, 8, 9, 4, 5, 6, 1, 2, 3]); } + #[tokio::test] + async fn test_morselizer_basic_parity() { + let store = Arc::new(InMemory::new()) as Arc; + + let batch1 = + record_batch!(("a", Int32, vec![Some(1), Some(2), Some(3)])).unwrap(); + let batch2 = record_batch!(("a", Int32, vec![Some(4), Some(5)])).unwrap(); + + let data_len = write_parquet_batches( + Arc::clone(&store), + "morselizer_basic_parity.parquet", + vec![batch1, batch2], + None, + ) + .await; + + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); + let opener = ParquetMorselizerBuilder::new() + .with_store(Arc::clone(&store)) + .with_schema(schema) + .with_projection_indices(&[0]) + .build(); + + let file = PartitionedFile::new( + "morselizer_basic_parity.parquet".to_string(), + u64::try_from(data_len).unwrap(), + ); + + let opener_values = + collect_int32_values(open_via_morselizer(&opener, file.clone()).await).await; + let morsel_values = + collect_int32_values(open_via_morselizer(&opener, file).await).await; + + assert_eq!(opener_values, morsel_values); + } + #[tokio::test] async fn test_reverse_scan_single_row_group() { let store = Arc::new(InMemory::new()) as Arc; @@ -1810,7 +2610,7 @@ mod test { ); let make_opener = |reverse_scan: bool| { - ParquetOpenerBuilder::new() + ParquetMorselizerBuilder::new() .with_store(Arc::clone(&store)) .with_schema(Arc::clone(&schema)) .with_projection_indices(&[0]) @@ -1821,11 +2621,11 @@ mod test { // With a single row group, forward and reverse should be the same // (only the row group order is reversed, not the rows within) let opener_forward = make_opener(false); - let stream_forward = opener_forward.open(file.clone()).unwrap().await.unwrap(); + let stream_forward = open_via_morselizer(&opener_forward, file.clone()).await; let (batches_forward, _) = count_batches_and_rows(stream_forward).await; let opener_reverse = make_opener(true); - let stream_reverse = opener_reverse.open(file).unwrap().await.unwrap(); + let stream_reverse = open_via_morselizer(&opener_reverse, file).await; let (batches_reverse, _) = count_batches_and_rows(stream_reverse).await; // Both should have the same number of batches since there's only one row group @@ -1886,7 +2686,7 @@ mod test { .with_extensions(Arc::new(access_plan)); let make_opener = |reverse_scan: bool| { - ParquetOpenerBuilder::new() + ParquetMorselizerBuilder::new() .with_store(Arc::clone(&store)) .with_schema(Arc::clone(&schema)) .with_projection_indices(&[0]) @@ -1896,7 +2696,7 @@ mod test { // Forward scan: RG0(3,4), RG1(5,6,7,8), RG2(9,10) let opener = make_opener(false); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let forward_values = collect_int32_values(stream).await; // Forward scan should produce: RG0(3,4), RG1(5,6,7,8), RG2(9,10) @@ -1912,7 +2712,7 @@ mod test { // - RG1 is read second, WITH RG1's selection (select all) -> 5, 6, 7, 8 // - RG0 is read third, WITH RG0's selection (skip 2, select 2) -> 3, 4 let opener = make_opener(true); - let stream = opener.open(file).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file).await; let reverse_values = collect_int32_values(stream).await; // Correct expected result: row groups reversed but each keeps its own selection @@ -1987,7 +2787,7 @@ mod test { .with_extensions(Arc::new(access_plan)); let make_opener = |reverse_scan: bool| { - ParquetOpenerBuilder::new() + ParquetMorselizerBuilder::new() .with_store(Arc::clone(&store)) .with_schema(Arc::clone(&schema)) .with_projection_indices(&[0]) @@ -1998,7 +2798,7 @@ mod test { // Forward scan: RG0(1), RG2(5), RG3(7) // Note: RG1 is completely skipped let opener = make_opener(false); - let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file.clone()).await; let forward_values = collect_int32_values(stream).await; assert_eq!( @@ -2011,7 +2811,7 @@ mod test { // WITHOUT the bug fix, this would return WRONG values // because the RowSelection would be incorrectly mapped let opener = make_opener(true); - let stream = opener.open(file).unwrap().await.unwrap(); + let stream = open_via_morselizer(&opener, file).await; let reverse_values = collect_int32_values(stream).await; assert_eq!( diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 1355eff5b9f1..d439bc62252c 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -23,19 +23,22 @@ use std::sync::Arc; use crate::DefaultParquetFileReaderFactory; use crate::ParquetFileReaderFactory; -use crate::opener::ParquetOpener; -use crate::opener::build_pruning_predicates; +use crate::ParquetMorselizer; +use crate::opener::{ + EncryptionContext, ParquetMorselizerState, build_pruning_predicates, +}; use crate::row_filter::can_expr_be_pushed_down_with_schemas; use datafusion_common::config::ConfigOptions; #[cfg(feature = "parquet_encryption")] use datafusion_common::config::EncryptionFactoryOptions; use datafusion_datasource::as_file_source; use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::morsel::Morselizer; use arrow::datatypes::TimeUnit; -use datafusion_common::DataFusionError; use datafusion_common::config::TableParquetOptions; use datafusion_common::tree_node::TreeNodeRecursion; +use datafusion_common::{DataFusionError, internal_err}; use datafusion_datasource::TableSchema; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; @@ -474,48 +477,12 @@ impl ParquetSource { } } - pub(crate) fn with_reverse_row_groups(mut self, reverse_row_groups: bool) -> Self { - self.reverse_row_groups = reverse_row_groups; - self - } - #[cfg(test)] - pub(crate) fn reverse_row_groups(&self) -> bool { - self.reverse_row_groups - } -} - -/// Parses datafusion.common.config.ParquetOptions.coerce_int96 String to a arrow_schema.datatype.TimeUnit -pub(crate) fn parse_coerce_int96_string( - str_setting: &str, -) -> datafusion_common::Result { - let str_setting_lower: &str = &str_setting.to_lowercase(); - - match str_setting_lower { - "ns" => Ok(TimeUnit::Nanosecond), - "us" => Ok(TimeUnit::Microsecond), - "ms" => Ok(TimeUnit::Millisecond), - "s" => Ok(TimeUnit::Second), - _ => Err(DataFusionError::Configuration(format!( - "Unknown or unsupported parquet coerce_int96: \ - {str_setting}. Valid values are: ns, us, ms, and s." - ))), - } -} - -/// Allows easy conversion from ParquetSource to Arc<dyn FileSource> -impl From for Arc { - fn from(source: ParquetSource) -> Self { - as_file_source(source) - } -} - -impl FileSource for ParquetSource { - fn create_file_opener( + fn create_parquet_morselizer( &self, object_store: Arc, base_config: &FileScanConfig, partition: usize, - ) -> datafusion_common::Result> { + ) -> datafusion_common::Result { let expr_adapter_factory = base_config .expr_adapter_factory .clone() @@ -526,14 +493,24 @@ impl FileSource for ParquetSource { Arc::new(DefaultParquetFileReaderFactory::new(object_store)) as _ }); + #[cfg(not(feature = "parquet_encryption"))] + let encryption_context = EncryptionContext::default(); + #[cfg(feature = "parquet_encryption")] - let file_decryption_properties = self - .table_parquet_options() - .crypto - .file_decryption - .clone() - .map(FileDecryptionProperties::from) - .map(Arc::new); + let encryption_context = { + let file_decryption_properties = self + .table_parquet_options() + .crypto + .file_decryption + .clone() + .map(FileDecryptionProperties::from) + .map(Arc::new); + + EncryptionContext::new( + file_decryption_properties, + self.get_encryption_factory_with_config(), + ) + }; let coerce_int96 = self .table_parquet_options @@ -542,12 +519,12 @@ impl FileSource for ParquetSource { .as_ref() .map(|time_unit| parse_coerce_int96_string(time_unit.as_str()).unwrap()); - let opener = Arc::new(ParquetOpener { + Ok(ParquetMorselizer::new(ParquetMorselizerState { partition_index: partition, projection: self.projection.clone(), batch_size: self .batch_size - .expect("Batch size must set before creating ParquetOpener"), + .expect("Batch size must set before creating ParquetMorselizer"), limit: base_config.limit, preserve_order: base_config.preserve_order, predicate: self.predicate.clone(), @@ -562,15 +539,69 @@ impl FileSource for ParquetSource { enable_bloom_filter: self.bloom_filter_on_read(), enable_row_group_stats_pruning: self.table_parquet_options.global.pruning, coerce_int96, - #[cfg(feature = "parquet_encryption")] - file_decryption_properties, expr_adapter_factory, - #[cfg(feature = "parquet_encryption")] - encryption_factory: self.get_encryption_factory_with_config(), + encryption_context, max_predicate_cache_size: self.max_predicate_cache_size(), reverse_row_groups: self.reverse_row_groups, - }); - Ok(opener) + })) + } + + pub(crate) fn with_reverse_row_groups(mut self, reverse_row_groups: bool) -> Self { + self.reverse_row_groups = reverse_row_groups; + self + } + #[cfg(test)] + pub(crate) fn reverse_row_groups(&self) -> bool { + self.reverse_row_groups + } +} + +/// Parses datafusion.common.config.ParquetOptions.coerce_int96 String to a arrow_schema.datatype.TimeUnit +pub(crate) fn parse_coerce_int96_string( + str_setting: &str, +) -> datafusion_common::Result { + let str_setting_lower: &str = &str_setting.to_lowercase(); + + match str_setting_lower { + "ns" => Ok(TimeUnit::Nanosecond), + "us" => Ok(TimeUnit::Microsecond), + "ms" => Ok(TimeUnit::Millisecond), + "s" => Ok(TimeUnit::Second), + _ => Err(DataFusionError::Configuration(format!( + "Unknown or unsupported parquet coerce_int96: \ + {str_setting}. Valid values are: ns, us, ms, and s." + ))), + } +} + +/// Allows easy conversion from ParquetSource to Arc<dyn FileSource> +impl From for Arc { + fn from(source: ParquetSource) -> Self { + as_file_source(source) + } +} + +impl FileSource for ParquetSource { + fn create_file_opener( + &self, + _object_store: Arc, + _base_config: &FileScanConfig, + _partition: usize, + ) -> datafusion_common::Result> { + internal_err!( + "ParquetSource::create_file_opener called but it supports Morsel API" + ) + } + + fn create_morselizer( + &self, + object_store: Arc, + base_config: &FileScanConfig, + partition: usize, + ) -> datafusion_common::Result> { + let morselizer = + self.create_parquet_morselizer(object_store, base_config, partition)?; + Ok(Box::new(morselizer)) } fn as_any(&self) -> &dyn Any { diff --git a/datafusion/datasource/Cargo.toml b/datafusion/datasource/Cargo.toml index 1315f871a68f..e92a98a98537 100644 --- a/datafusion/datasource/Cargo.toml +++ b/datafusion/datasource/Cargo.toml @@ -72,6 +72,7 @@ zstd = { workspace = true, optional = true } [dev-dependencies] criterion = { workspace = true } +insta = { workspace = true } tempfile = { workspace = true } # Note: add additional linter rules in lib.rs. diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index b5a6760cae02..f13cc863c776 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -25,6 +25,7 @@ use std::sync::Arc; use crate::file_groups::FileGroupPartitioner; use crate::file_scan_config::FileScanConfig; use crate::file_stream::FileOpener; +use crate::morsel::{FileOpenerMorselizer, Morselizer}; #[expect(deprecated)] use crate::schema_adapter::SchemaAdapterFactory; use datafusion_common::config::ConfigOptions; @@ -63,13 +64,34 @@ pub fn as_file_source(source: T) -> Arc /// /// [`DataSource`]: crate::source::DataSource pub trait FileSource: Send + Sync { - /// Creates a `dyn FileOpener` based on given parameters + /// Creates a `dyn FileOpener` based on given parameters. + /// + /// `FileSource`s that implement the Morsel API should return a "Not + /// Implemented" or "Internal" error for this API. + /// + /// TODO: deprecate fn create_file_opener( &self, object_store: Arc, base_config: &FileScanConfig, partition: usize, ) -> Result>; + + /// Creates a `dyn Morselizer` based on given parameters. + /// + /// The default implementation preserves existing behavior by adapting the + /// legacy [`FileOpener`] API into a [`Morselizer`]. File formats with a + /// native morsel-driven implementation should override this method to + /// return a [`Morselizer`] and not implement the [`FileOpener`] API. + fn create_morselizer( + &self, + object_store: Arc, + base_config: &FileScanConfig, + partition: usize, + ) -> Result> { + let opener = self.create_file_opener(object_store, base_config, partition)?; + Ok(Box::new(FileOpenerMorselizer::new(opener))) + } /// Any fn as_any(&self) -> &dyn Any; diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 82a986a688ed..c310d2bdf04a 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -21,7 +21,7 @@ use crate::file_groups::FileGroup; use crate::{ PartitionedFile, display::FileGroupsDisplay, file::FileSource, - file_compression_type::FileCompressionType, file_stream::FileStream, + file_compression_type::FileCompressionType, file_stream::FileStreamBuilder, source::DataSource, statistics::MinMaxStatistics, }; use arrow::datatypes::FieldRef; @@ -586,9 +586,15 @@ impl DataSource for FileScanConfig { let source = self.file_source.with_batch_size(batch_size); - let opener = source.create_file_opener(object_store, self, partition)?; + let morselizer = source.create_morselizer(object_store, self, partition)?; - let stream = FileStream::new(self, partition, opener, source.metrics())?; + let stream = FileStreamBuilder::new_with_morselizer( + self, + partition, + morselizer, + source.metrics(), + ) + .build()?; Ok(Box::pin(cooperative(stream))) } diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs deleted file mode 100644 index b75e66849b7a..000000000000 --- a/datafusion/datasource/src/file_stream.rs +++ /dev/null @@ -1,952 +0,0 @@ -// 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. - -//! A generic stream over file format readers that can be used by -//! any file format that read its files from start to end. -//! -//! Note: Most traits here need to be marked `Sync + Send` to be -//! compliant with the `SendableRecordBatchStream` trait. - -use std::collections::VecDeque; -use std::mem; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll}; - -use crate::PartitionedFile; -use crate::file_scan_config::FileScanConfig; -use arrow::datatypes::SchemaRef; -use datafusion_common::error::Result; -use datafusion_execution::RecordBatchStream; -use datafusion_physical_plan::metrics::{ - BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, Time, -}; - -use arrow::record_batch::RecordBatch; -use datafusion_common::instant::Instant; - -use futures::future::BoxFuture; -use futures::stream::BoxStream; -use futures::{FutureExt as _, Stream, StreamExt as _, ready}; - -/// A stream that iterates record batch by record batch, file over file. -pub struct FileStream { - /// An iterator over input files. - file_iter: VecDeque, - /// The stream schema (file schema including partition columns and after - /// projection). - projected_schema: SchemaRef, - /// The remaining number of records to parse, None if no limit - remain: Option, - /// A dynamic [`FileOpener`]. Calling `open()` returns a [`FileOpenFuture`], - /// which can be resolved to a stream of `RecordBatch`. - file_opener: Arc, - /// The stream state - state: FileStreamState, - /// File stream specific metrics - file_stream_metrics: FileStreamMetrics, - /// runtime baseline metrics - baseline_metrics: BaselineMetrics, - /// Describes the behavior of the `FileStream` if file opening or scanning fails - on_error: OnError, -} - -impl FileStream { - /// Create a new `FileStream` using the give `FileOpener` to scan underlying files - pub fn new( - config: &FileScanConfig, - partition: usize, - file_opener: Arc, - metrics: &ExecutionPlanMetricsSet, - ) -> Result { - let projected_schema = config.projected_schema()?; - - let file_group = config.file_groups[partition].clone(); - - Ok(Self { - file_iter: file_group.into_inner().into_iter().collect(), - projected_schema, - remain: config.limit, - file_opener, - state: FileStreamState::Idle, - file_stream_metrics: FileStreamMetrics::new(metrics, partition), - baseline_metrics: BaselineMetrics::new(metrics, partition), - on_error: OnError::Fail, - }) - } - - /// Specify the behavior when an error occurs opening or scanning a file - /// - /// If `OnError::Skip` the stream will skip files which encounter an error and continue - /// If `OnError:Fail` (default) the stream will fail and stop processing when an error occurs - pub fn with_on_error(mut self, on_error: OnError) -> Self { - self.on_error = on_error; - self - } - - /// Begin opening the next file in parallel while decoding the current file in 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> { - let part_file = self.file_iter.pop_front()?; - Some(self.file_opener.open(part_file)) - } - - fn poll_inner(&mut self, cx: &mut Context<'_>) -> Poll>> { - loop { - match &mut self.state { - 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), - Err(e) => { - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } - } - FileStreamState::Open { future } => match ready!(future.poll_unpin(cx)) { - Ok(reader) => { - self.file_stream_metrics.files_opened.add(1); - // include time needed to start opening in `start_next_file` - self.file_stream_metrics.time_opening.stop(); - let next = { - let scanning_total_metric = self - .file_stream_metrics - .time_scanning_total - .metrics - .clone(); - let _timer = scanning_total_metric.timer(); - self.start_next_file().transpose() - }; - self.file_stream_metrics.time_scanning_until_data.start(); - self.file_stream_metrics.time_scanning_total.start(); - - match next { - Ok(Some(next_future)) => { - self.state = FileStreamState::Scan { - reader, - next: Some(NextOpen::Pending(next_future)), - }; - } - Ok(None) => { - self.state = FileStreamState::Scan { reader, next: None }; - } - Err(e) => { - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } - } - Err(e) => { - self.file_stream_metrics.file_open_errors.add(1); - match self.on_error { - OnError::Skip => { - self.file_stream_metrics.files_processed.add(1); - self.file_stream_metrics.time_opening.stop(); - self.state = FileStreamState::Idle - } - OnError::Fail => { - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - } - } - }, - FileStreamState::Scan { reader, next } => { - // We need to poll the next `FileOpenFuture` here to drive it forward - if let Some(next_open_future) = next - && let NextOpen::Pending(f) = next_open_future - && let Poll::Ready(reader) = f.as_mut().poll(cx) - { - *next_open_future = NextOpen::Ready(reader); - } - match ready!(reader.poll_next_unpin(cx)) { - Some(Ok(batch)) => { - self.file_stream_metrics.time_scanning_until_data.stop(); - self.file_stream_metrics.time_scanning_total.stop(); - let batch = match &mut self.remain { - Some(remain) => { - if *remain > batch.num_rows() { - *remain -= batch.num_rows(); - batch - } else { - let batch = batch.slice(0, *remain); - // Count this file, the prefetched next file - // (if any), and all remaining files we will - // never open. - let done = 1 - + self.file_iter.len() - + usize::from(next.is_some()); - self.file_stream_metrics - .files_processed - .add(done); - self.state = FileStreamState::Limit; - *remain = 0; - batch - } - } - None => batch, - }; - self.file_stream_metrics.time_scanning_total.start(); - return Poll::Ready(Some(Ok(batch))); - } - Some(Err(err)) => { - self.file_stream_metrics.file_scan_errors.add(1); - self.file_stream_metrics.time_scanning_until_data.stop(); - self.file_stream_metrics.time_scanning_total.stop(); - - match self.on_error { - // If `OnError::Skip` we skip the file as soon as we hit the first error - OnError::Skip => { - self.file_stream_metrics.files_processed.add(1); - match mem::take(next) { - Some(future) => { - self.file_stream_metrics.time_opening.start(); - - match future { - NextOpen::Pending(future) => { - self.state = - FileStreamState::Open { future } - } - NextOpen::Ready(reader) => { - self.state = FileStreamState::Open { - future: Box::pin( - std::future::ready(reader), - ), - } - } - } - } - None => return Poll::Ready(None), - } - } - OnError::Fail => { - self.state = FileStreamState::Error; - return Poll::Ready(Some(Err(err))); - } - } - } - None => { - self.file_stream_metrics.files_processed.add(1); - self.file_stream_metrics.time_scanning_until_data.stop(); - self.file_stream_metrics.time_scanning_total.stop(); - - match mem::take(next) { - Some(future) => { - self.file_stream_metrics.time_opening.start(); - - match future { - NextOpen::Pending(future) => { - self.state = FileStreamState::Open { future } - } - NextOpen::Ready(reader) => { - self.state = FileStreamState::Open { - future: Box::pin(std::future::ready( - reader, - )), - } - } - } - } - None => return Poll::Ready(None), - } - } - } - } - FileStreamState::Error | FileStreamState::Limit => { - return Poll::Ready(None); - } - } - } - } -} - -impl Stream for FileStream { - type Item = Result; - - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - self.file_stream_metrics.time_processing.start(); - let result = self.poll_inner(cx); - self.file_stream_metrics.time_processing.stop(); - self.baseline_metrics.record_poll(result) - } -} - -impl RecordBatchStream for FileStream { - fn schema(&self) -> SchemaRef { - Arc::clone(&self.projected_schema) - } -} - -/// A fallible future that resolves to a stream of [`RecordBatch`] -pub type FileOpenFuture = - BoxFuture<'static, Result>>>; - -/// Describes the behavior of the `FileStream` if file opening or scanning fails -#[derive(Default)] -pub enum OnError { - /// Fail the entire stream and return the underlying error - #[default] - Fail, - /// Continue scanning, ignoring the failed file - Skip, -} - -/// Generic API for opening a file using an [`ObjectStore`] and resolving to a -/// stream of [`RecordBatch`] -/// -/// [`ObjectStore`]: object_store::ObjectStore -pub trait FileOpener: Unpin + Send + Sync { - /// Asynchronously open the specified file and return a stream - /// of [`RecordBatch`] - fn open(&self, partitioned_file: PartitionedFile) -> Result; -} - -/// Represents the state of the next `FileOpenFuture`. Since we need to poll -/// this future while scanning the current file, we need to store the result if it -/// is ready -pub enum NextOpen { - Pending(FileOpenFuture), - Ready(Result>>), -} - -pub enum FileStreamState { - /// The idle state, no file is currently being read - Idle, - /// Currently performing asynchronous IO to obtain a stream of RecordBatch - /// for a given file - Open { - /// A [`FileOpenFuture`] returned by [`FileOpener::open`] - future: FileOpenFuture, - }, - /// Scanning the [`BoxStream`] returned by the completion of a [`FileOpenFuture`] - /// returned by [`FileOpener::open`] - Scan { - /// The reader instance - reader: BoxStream<'static, Result>, - /// A [`FileOpenFuture`] for the next file to be processed. - /// This allows the next file to be opened in parallel while the - /// current file is read. - next: Option, - }, - /// Encountered an error - Error, - /// Reached the row limit - Limit, -} - -/// A timer that can be started and stopped. -pub struct StartableTime { - pub metrics: Time, - // use for record each part cost time, will eventually add into 'metrics'. - pub start: Option, -} - -impl StartableTime { - pub fn start(&mut self) { - assert!(self.start.is_none()); - self.start = Some(Instant::now()); - } - - pub fn stop(&mut self) { - if let Some(start) = self.start.take() { - self.metrics.add_elapsed(start); - } - } -} - -/// Metrics for [`FileStream`] -/// -/// Note that all of these metrics are in terms of wall clock time -/// (not cpu time) so they include time spent waiting on I/O as well -/// as other operators. -/// -/// [`FileStream`]: -pub struct FileStreamMetrics { - /// Wall clock time elapsed for file opening. - /// - /// Time between when [`FileOpener::open`] is called and when the - /// [`FileStream`] receives a stream for reading. - /// - /// If there are multiple files being scanned, the stream - /// will open the next file in the background while scanning the - /// current file. This metric will only capture time spent opening - /// while not also scanning. - /// [`FileStream`]: - pub time_opening: StartableTime, - /// Wall clock time elapsed for file scanning + first record batch of decompression + decoding - /// - /// Time between when the [`FileStream`] requests data from the - /// stream and when the first [`RecordBatch`] is produced. - /// [`FileStream`]: - pub time_scanning_until_data: StartableTime, - /// Total elapsed wall clock time for scanning + record batch decompression / decoding - /// - /// Sum of time between when the [`FileStream`] requests data from - /// the stream and when a [`RecordBatch`] is produced for all - /// record batches in the stream. Note that this metric also - /// includes the time of the parent operator's execution. - pub time_scanning_total: StartableTime, - /// Wall clock time elapsed for data decompression + decoding - /// - /// Time spent waiting for the FileStream's input. - pub time_processing: StartableTime, - /// Count of errors opening file. - /// - /// If using `OnError::Skip` this will provide a count of the number of files - /// which were skipped and will not be included in the scan results. - pub file_open_errors: Count, - /// Count of errors scanning file - /// - /// If using `OnError::Skip` this will provide a count of the number of files - /// which were skipped and will not be included in the scan results. - pub file_scan_errors: Count, - /// Count of files successfully opened or evaluated for processing. - /// At t=end (completion of a query) this is equal to `files_opened`, and both values are equal - /// to the total number of files in the query; unless the query itself fails. - /// This value will always be greater than or equal to `files_open`. - /// Note that this value does *not* mean the file was actually scanned. - /// We increment this value for any processing of a file, even if that processing is - /// discarding it because we hit a `LIMIT` (in this case `files_opened` and `files_processed` are both incremented at the same time). - pub files_opened: Count, - /// Count of files completely processed / closed (opened, pruned, or skipped due to limit). - /// At t=0 (the beginning of a query) this is 0. - /// At t=end (completion of a query) this is equal to `files_opened`, and both values are equal - /// to the total number of files in the query; unless the query itself fails. - /// This value will always be less than or equal to `files_open`. - /// We increment this value for any processing of a file, even if that processing is - /// discarding it because we hit a `LIMIT` (in this case `files_opened` and `files_processed` are both incremented at the same time). - pub files_processed: Count, -} - -impl FileStreamMetrics { - pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { - let time_opening = StartableTime { - metrics: MetricBuilder::new(metrics) - .subset_time("time_elapsed_opening", partition), - start: None, - }; - - let time_scanning_until_data = StartableTime { - metrics: MetricBuilder::new(metrics) - .subset_time("time_elapsed_scanning_until_data", partition), - start: None, - }; - - let time_scanning_total = StartableTime { - metrics: MetricBuilder::new(metrics) - .subset_time("time_elapsed_scanning_total", partition), - start: None, - }; - - let time_processing = StartableTime { - metrics: MetricBuilder::new(metrics) - .subset_time("time_elapsed_processing", partition), - start: None, - }; - - let file_open_errors = - MetricBuilder::new(metrics).counter("file_open_errors", partition); - - let file_scan_errors = - MetricBuilder::new(metrics).counter("file_scan_errors", partition); - - let files_opened = MetricBuilder::new(metrics).counter("files_opened", partition); - - let files_processed = - MetricBuilder::new(metrics).counter("files_processed", partition); - - Self { - time_opening, - time_scanning_until_data, - time_scanning_total, - time_processing, - file_open_errors, - file_scan_errors, - files_opened, - files_processed, - } - } -} - -#[cfg(test)] -mod tests { - use crate::PartitionedFile; - use crate::file_scan_config::FileScanConfigBuilder; - use crate::tests::make_partition; - use datafusion_common::error::Result; - use datafusion_execution::object_store::ObjectStoreUrl; - use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; - use futures::{FutureExt as _, StreamExt as _}; - use std::sync::Arc; - use std::sync::atomic::{AtomicUsize, Ordering}; - - use crate::file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; - use crate::test_util::MockSource; - use arrow::array::RecordBatch; - use arrow::datatypes::Schema; - - use datafusion_common::{assert_batches_eq, exec_err, internal_err}; - - /// Test `FileOpener` which will simulate errors during file opening or scanning - #[derive(Default)] - struct TestOpener { - /// Index in stream of files which should throw an error while opening - error_opening_idx: Vec, - /// Index in stream of files which should throw an error while scanning - error_scanning_idx: Vec, - /// Index of last file in stream - current_idx: AtomicUsize, - /// `RecordBatch` to return - records: Vec, - } - - impl FileOpener for TestOpener { - fn open(&self, _partitioned_file: PartitionedFile) -> Result { - let idx = self.current_idx.fetch_add(1, Ordering::SeqCst); - - if self.error_opening_idx.contains(&idx) { - Ok(futures::future::ready(internal_err!("error opening")).boxed()) - } else if self.error_scanning_idx.contains(&idx) { - let error = futures::future::ready(exec_err!("error scanning")); - let stream = futures::stream::once(error).boxed(); - Ok(futures::future::ready(Ok(stream)).boxed()) - } else { - let iterator = self.records.clone().into_iter().map(Ok); - let stream = futures::stream::iter(iterator).boxed(); - Ok(futures::future::ready(Ok(stream)).boxed()) - } - } - } - - #[derive(Default)] - struct FileStreamTest { - /// Number of files in the stream - num_files: usize, - /// Global limit of records emitted by the stream - limit: Option, - /// Error-handling behavior of the stream - on_error: OnError, - /// Mock `FileOpener` - opener: TestOpener, - } - - impl FileStreamTest { - pub fn new() -> Self { - Self::default() - } - - /// Specify the number of files in the stream - pub fn with_num_files(mut self, num_files: usize) -> Self { - self.num_files = num_files; - self - } - - /// Specify the limit - pub fn with_limit(mut self, limit: Option) -> Self { - self.limit = limit; - self - } - - /// Specify the index of files in the stream which should - /// throw an error when opening - pub fn with_open_errors(mut self, idx: Vec) -> Self { - self.opener.error_opening_idx = idx; - self - } - - /// Specify the index of files in the stream which should - /// throw an error when scanning - pub fn with_scan_errors(mut self, idx: Vec) -> Self { - self.opener.error_scanning_idx = idx; - self - } - - /// Specify the behavior of the stream when an error occurs - pub fn with_on_error(mut self, on_error: OnError) -> Self { - self.on_error = on_error; - self - } - - /// Specify the record batches that should be returned from each - /// file that is successfully scanned - pub fn with_records(mut self, records: Vec) -> Self { - self.opener.records = records; - self - } - - /// Collect the results of the `FileStream` - pub async fn result(self) -> Result> { - let file_schema = self - .opener - .records - .first() - .map(|batch| batch.schema()) - .unwrap_or_else(|| Arc::new(Schema::empty())); - - // let ctx = SessionContext::new(); - let mock_files: Vec<(String, u64)> = (0..self.num_files) - .map(|idx| (format!("mock_file{idx}"), 10_u64)) - .collect(); - - // let mock_files_ref: Vec<(&str, u64)> = mock_files - // .iter() - // .map(|(name, size)| (name.as_str(), *size)) - // .collect(); - - let file_group = mock_files - .into_iter() - .map(|(name, size)| PartitionedFile::new(name, size)) - .collect(); - - let on_error = self.on_error; - - let table_schema = crate::table_schema::TableSchema::new(file_schema, vec![]); - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - Arc::new(MockSource::new(table_schema)), - ) - .with_file_group(file_group) - .with_limit(self.limit) - .build(); - let metrics_set = ExecutionPlanMetricsSet::new(); - let file_stream = - FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set) - .unwrap() - .with_on_error(on_error); - - file_stream - .collect::>() - .await - .into_iter() - .collect::>>() - } - } - - /// helper that creates a stream of 2 files with the same pair of batches in each ([0,1,2] and [0,1]) - async fn create_and_collect(limit: Option) -> Vec { - FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_limit(limit) - .result() - .await - .expect("error executing stream") - } - - #[tokio::test] - async fn on_error_opening() -> Result<()> { - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_open_errors(vec![0]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_open_errors(vec![1]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_open_errors(vec![0, 1]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "++", - "++", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn on_error_scanning_fail() -> Result<()> { - let result = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Fail) - .with_scan_errors(vec![1]) - .result() - .await; - - assert!(result.is_err()); - - Ok(()) - } - - #[tokio::test] - async fn on_error_opening_fail() -> Result<()> { - let result = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Fail) - .with_open_errors(vec![1]) - .result() - .await; - - assert!(result.is_err()); - - Ok(()) - } - - #[tokio::test] - async fn on_error_scanning() -> Result<()> { - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_scan_errors(vec![0]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_scan_errors(vec![1]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(2) - .with_on_error(OnError::Skip) - .with_scan_errors(vec![0, 1]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "++", - "++", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn on_error_mixed() -> Result<()> { - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(3) - .with_on_error(OnError::Skip) - .with_open_errors(vec![1]) - .with_scan_errors(vec![0]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(3) - .with_on_error(OnError::Skip) - .with_open_errors(vec![0]) - .with_scan_errors(vec![1]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(3) - .with_on_error(OnError::Skip) - .with_open_errors(vec![2]) - .with_scan_errors(vec![0, 1]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "++", - "++", - ], &batches); - - let batches = FileStreamTest::new() - .with_records(vec![make_partition(3), make_partition(2)]) - .with_num_files(3) - .with_on_error(OnError::Skip) - .with_open_errors(vec![0, 2]) - .with_scan_errors(vec![1]) - .result() - .await?; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "++", - "++", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn without_limit() -> Result<()> { - let batches = create_and_collect(None).await; - - #[rustfmt::skip] - assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn with_limit_between_files() -> Result<()> { - let batches = create_and_collect(Some(5)).await; - #[rustfmt::skip] - assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "+---+", - ], &batches); - - Ok(()) - } - - #[tokio::test] - async fn with_limit_at_middle_of_batch() -> Result<()> { - let batches = create_and_collect(Some(6)).await; - #[rustfmt::skip] - assert_batches_eq!(&[ - "+---+", - "| i |", - "+---+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 0 |", - "| 1 |", - "| 0 |", - "+---+", - ], &batches); - - Ok(()) - } -} diff --git a/datafusion/datasource/src/file_stream/mod.rs b/datafusion/datasource/src/file_stream/mod.rs new file mode 100644 index 000000000000..a0d8915d8e91 --- /dev/null +++ b/datafusion/datasource/src/file_stream/mod.rs @@ -0,0 +1,2280 @@ +// 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. + +//! A generic stream over file format readers that can be used by +//! any file format that read its files from start to end. +//! +//! Note: Most traits here need to be marked `Sync + Send` to be +//! compliant with the `SendableRecordBatchStream` trait. + +pub mod shared_state; + +pub use shared_state::{ + FileStreamId, OutstandingIoPermit, SharedFileStreamMode, SharedFileStreamState, +}; + +use std::collections::VecDeque; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use crate::PartitionedFile; +use crate::file_scan_config::FileScanConfig; +use arrow::datatypes::SchemaRef; +use datafusion_common::{Result, internal_datafusion_err}; +use datafusion_execution::RecordBatchStream; +use datafusion_physical_plan::metrics::{ + BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, Time, +}; + +use arrow::record_batch::RecordBatch; +use datafusion_common::instant::Instant; + +use crate::morsel::{FileOpenerMorselizer, Morsel, MorselPlanner, Morselizer}; +use futures::future::BoxFuture; +use futures::stream::BoxStream; +use futures::{FutureExt, Stream, StreamExt as _}; + +const DEFAULT_OUTSTANDING_IOS_PER_PARTITION: usize = 2; + +/// Keep at most this many morsels buffered before pausing additional planning. +/// +/// The default is one morsel per available core. The intent is that once work +/// stealing is added, each other core can find at least one morsel to steal +/// without requiring the scan to eagerly buffer an unbounded amount of work. +/// +/// TODO make this a config option +fn max_buffered_morsels() -> usize { + std::thread::available_parallelism() + .map(usize::from) + .unwrap_or(1) +} + +/// Resolve the shared outstanding-I/O budget for one `DataSourceExec`. +/// +/// This is temporary wiring until the datasource layer constructs and passes +/// the shared state directly into each sibling `FileStream`. +fn target_datasource_outstanding_ios(num_partitions: usize) -> usize { + std::env::var("DATAFUSION_DATASOURCE_OUTSTANDING_IOS") + .ok() + .and_then(|value| value.parse::().ok()) + .filter(|value| *value > 0) + .unwrap_or_else(|| { + DEFAULT_OUTSTANDING_IOS_PER_PARTITION * std::cmp::max(num_partitions, 1) + }) +} + +/// Build a default shared state object for streams created directly from a +/// `FileScanConfig`. +fn default_shared_file_stream_state(config: &FileScanConfig) -> SharedFileStreamState { + SharedFileStreamState::new( + target_datasource_outstanding_ios(config.file_groups.len()), + if config.preserve_order { + SharedFileStreamMode::PreserveOrder + } else { + SharedFileStreamMode::Unordered + }, + ) +} + +/// A stream that iterates record batch by record batch, file over file. +/// +/// When running, a FileStream has some number of waiting planners (that are +/// waiting on IO) and some number of read_planners that are waiting on CPU. +/// +/// When the next batch is requested, the FileStream will first poll any +/// outstanding io_requests to ensure I/O is making progress in parallel with +/// batch processing. +/// +/// It then tries to prioritize processing data it has in its cache by read from +/// the active stream, if any. If that is not ready, it will use the CPU to +/// prepare more morsels or discover new IO before launching the next morsel. +/// +/// Sibling `FileStream`s created for the same `DataSourceExec` may also share a +/// [`SharedFileStreamState`]. That shared state coordinates resources such as +/// the total number of outstanding planner I/O operations across all sibling +/// streams. Each `FileStream` registers itself with the shared state during +/// construction (via `with_shared_state`) and must acquire a shared permit +/// before advancing a ready planner toward its next I/O phase. If the planner +/// actually issues an `io_future`, the permit remains attached to that waiting +/// planner until the future resolves. +/// +/// Future feature: +/// Other FileStreams may steal morsels from this stream to increase parallelism and resource utilization. +pub struct FileStream { + /// Local file/planner/morsel queues owned by this stream. + queues: MorselQueue, + /// The current reader, if any + reader: Option>>, + /// The stream schema (file schema including partition columns and after + /// projection). + projected_schema: SchemaRef, + /// The remaining number of records to parse until limit is reached, None if no limit + remain: Option, + /// A type specific [`Morselizer`] that examines the input files and produces a stream of `Morsels` + morselizer: Box, + /// File stream specific metrics + file_stream_metrics: FileStreamMetrics, + /// runtime baseline metrics + baseline_metrics: BaselineMetrics, + /// Describes the behavior of the `FileStream` if file opening or scanning fails + on_error: OnError, + /// Preserve the logical planner/morsel order defined by the + /// [`MorselPlan`] API? + /// + /// If false (the default) morsels will be produced in the order + /// that they are ready to be run. + /// + /// If true, Morsels will be produced in the logical order defined on + /// [`MorselPlan`] + /// + /// [`MorselPlan`]: crate::morsel::MorselPlan + preserve_order: bool, + /// Shared scheduling state across all sibling `FileStream`s for the same + /// `DataSourceExec`. + /// + /// This shared state enforces the cross-stream outstanding-I/O budget and + /// wakes blocked sibling streams when capacity becomes available again. + shared_file_stream_state: SharedFileStreamState, + /// This stream's identity within `shared_file_stream_state`. + /// + /// The id is assigned when the stream registers itself with the shared + /// state and is cleared once the stream unregisters after reaching a + /// terminal state. + stream_id: Option, + /// Is the stream complete? + state: StreamState, +} + +enum StreamState { + /// Stream can make progress when polled + Active, + /// Stream is done + Done, + /// Stream is done, and errord + Error, +} + +/// Queues for a file/planner/morsel +/// +/// The flow is: +/// 1. Read each file and turn it into morsels (potentially in parallel) +/// 2. Read each morsel individually and produce `RecordBatch`es for processing +#[derive(Debug)] +pub(super) struct MorselQueue { + /// Input files that have not yet been morselized. + file_iter: VecDeque, + /// Planners that are currently waiting on an outstanding I/O phase. + waiting_planners: VecDeque, + /// Planners that are CPU-ready and may be advanced by calling `plan()`. + ready_planners: VecDeque>, + /// Morsels that are ready to be scanned into `RecordBatch`es. + morsels: VecDeque>, +} + +impl MorselQueue { + /// Create an empty queue set for one file group. + pub(super) fn new(file_iter: VecDeque) -> Self { + Self { + file_iter, + waiting_planners: VecDeque::new(), + ready_planners: VecDeque::new(), + morsels: VecDeque::new(), + } + } + + /// Clear all planner and morsel work currently owned by this stream. + pub(super) fn clear(&mut self) { + self.waiting_planners.clear(); + self.ready_planners.clear(); + self.morsels.clear(); + } + + /// Return true if the stream has no remaining queued file or morsel work. + pub(super) fn is_empty(&self) -> bool { + self.file_iter.is_empty() + && self.waiting_planners.is_empty() + && self.ready_planners.is_empty() + && self.morsels.is_empty() + } + + /// Return the number of queued ready morsels. + pub(super) fn morsel_len(&self) -> usize { + self.morsels.len() + } + + /// Return true if there is at least one queued ready morsel. + pub(super) fn has_morsels(&self) -> bool { + !self.morsels.is_empty() + } + + /// Return true if there is at least one queued ready planner. + pub(super) fn has_ready_planners(&self) -> bool { + !self.ready_planners.is_empty() + } + + /// Return true if there is at least one queued waiting planner. + pub(super) fn has_waiting_planners(&self) -> bool { + !self.waiting_planners.is_empty() + } + + /// Return the total number of queued ready and waiting planners. + pub(super) fn planner_count(&self) -> usize { + self.waiting_planners.len() + self.ready_planners.len() + } + + /// Push one CPU-ready planner into the local queue. + pub(super) fn push_ready_planner(&mut self, planner: Box) { + self.ready_planners.push_back(planner); + } + + /// Extend the local queue with CPU-ready planners. + pub(super) fn extend_ready_planners( + &mut self, + planners: impl IntoIterator>, + ) { + self.ready_planners.extend(planners); + } + + /// Pop the next CPU-ready planner from the local queue. + pub(super) fn pop_ready_planner(&mut self) -> Option> { + self.ready_planners.pop_front() + } + + /// Push one waiting planner into the local queue. + fn push_waiting_planner(&mut self, planner: WaitingPlanner) { + self.waiting_planners.push_back(planner); + } + + /// Drain all waiting planners from the queue. + fn take_waiting_planners(&mut self) -> VecDeque { + std::mem::take(&mut self.waiting_planners) + } + + /// Push one ready morsel into the local queue. + pub(super) fn push_morsel(&mut self, morsel: Box) { + self.morsels.push_back(morsel); + } + + /// Extend the local queue with ready morsels. + pub(super) fn extend_morsels( + &mut self, + morsels: impl IntoIterator>, + ) { + self.morsels.extend(morsels); + } + + /// Pop the next ready morsel from the local queue. + pub(super) fn pop_morsel(&mut self) -> Option> { + self.morsels.pop_front() + } + + /// Pop the next input file from the local queue. + pub(super) fn pop_file(&mut self) -> Option { + self.file_iter.pop_front() + } +} + +/// Builder for constructing a [`FileStream`]. +pub struct FileStreamBuilder<'a> { + config: &'a FileScanConfig, + partition: usize, + morselizer: Box, + metrics: &'a ExecutionPlanMetricsSet, + on_error: OnError, + preserve_order: bool, + shared_file_stream_state: Option, +} + +impl<'a> FileStreamBuilder<'a> { + /// Create a new builder using a legacy [`FileOpener`]. + pub fn new( + config: &'a FileScanConfig, + partition: usize, + file_opener: Arc, + metrics: &'a ExecutionPlanMetricsSet, + ) -> Self { + Self::new_with_morselizer( + config, + partition, + Box::new(FileOpenerMorselizer::new(file_opener)), + metrics, + ) + } + + /// Create a new builder using a [`Morselizer`]. + pub fn new_with_morselizer( + config: &'a FileScanConfig, + partition: usize, + morselizer: Box, + metrics: &'a ExecutionPlanMetricsSet, + ) -> Self { + Self { + config, + partition, + morselizer, + on_error: OnError::Fail, + preserve_order: config.preserve_order, + metrics, + shared_file_stream_state: None, + } + } + + /// Configure the behavior when opening or scanning a file fails. + pub fn with_on_error(mut self, on_error: OnError) -> Self { + self.on_error = on_error; + self + } + + /// Configure whether this stream should preserve logical planner order. + pub fn with_preserve_order(mut self, preserve_order: bool) -> Self { + self.preserve_order = preserve_order; + self + } + + /// Use the provided shared scheduler state instead of the default one. + pub fn with_shared_state( + mut self, + shared_file_stream_state: SharedFileStreamState, + ) -> Self { + self.shared_file_stream_state = Some(shared_file_stream_state); + self + } + + /// Build the configured [`FileStream`]. + pub fn build(self) -> Result { + let shared_file_stream_state = self + .shared_file_stream_state + .unwrap_or_else(|| default_shared_file_stream_state(self.config)); + let projected_schema = self.config.projected_schema()?; + let file_group = self.config.file_groups[self.partition].clone(); + let stream_id = shared_file_stream_state.register_stream(); + + Ok(FileStream { + queues: MorselQueue::new(file_group.into_inner().into_iter().collect()), + reader: None, + projected_schema, + remain: self.config.limit, + morselizer: self.morselizer, + file_stream_metrics: FileStreamMetrics::new(self.metrics, self.partition), + baseline_metrics: BaselineMetrics::new(self.metrics, self.partition), + on_error: self.on_error, + preserve_order: self.preserve_order, + shared_file_stream_state, + stream_id: Some(stream_id), + state: StreamState::Active, + }) + } +} + +impl FileStream { + /// Return true if this stream may publish and steal ready work from the + /// shared queue. + fn can_share_ready_work(&self) -> bool { + // Only enable shared ready-work queues when there are actual sibling + // streams. Sending work through shared state in the single-stream case + // changes local scheduling behavior without enabling any useful + // stealing. + !self.preserve_order + && self.shared_file_stream_state.registered_stream_count() > 1 + } + + /// Enqueue ready planners either locally or into the shared queue. + fn push_ready_planners( + &mut self, + planners: impl IntoIterator>, + ) { + if self.can_share_ready_work() { + for planner in planners { + self.shared_file_stream_state.push_ready_planner(planner); + } + } else { + self.queues.extend_ready_planners(planners); + } + } + + /// Enqueue ready morsels either locally or into the shared queue. + fn push_ready_morsels(&mut self, morsels: impl IntoIterator>) { + if self.can_share_ready_work() { + for morsel in morsels { + self.shared_file_stream_state.push_ready_morsel(morsel); + } + } else { + self.queues.extend_morsels(morsels); + } + } + + /// Try to steal one ready morsel or planner from the shared queue and place + /// them in the local queue. + /// + /// Morsels are preferred because they are already fully prepared CPU work. + fn try_steal_ready_work(&mut self) -> bool { + if !self.can_share_ready_work() + || self.reader.is_some() + || self.queues.has_morsels() + || self.queues.has_ready_planners() + { + return false; + } + + if let Some(morsel) = self.shared_file_stream_state.pop_ready_morsel() { + self.queues.push_morsel(morsel); + return true; + } + + if let Some(planner) = self.shared_file_stream_state.pop_ready_planner() { + self.queues.push_ready_planner(planner); + return true; + } + + false + } + + /// Create a new [`FileStream`] using a legacy [`FileOpener`]. + /// + /// Prefer [`FileStreamBuilder`] for new code. + #[deprecated(since = "52.3.0", note = "use FileStreamBuilder instead")] + pub fn new( + config: &FileScanConfig, + partition: usize, + file_opener: Arc, + metrics: &ExecutionPlanMetricsSet, + ) -> Result { + FileStreamBuilder::new(config, partition, file_opener, metrics).build() + } + + /// Return this stream's registered shared-state id. + fn stream_id(&self) -> Result { + self.stream_id.ok_or_else(|| { + internal_datafusion_err!("file stream is not registered with shared state") + }) + } + + /// Unregister this stream from the shared scheduler once it reaches a + /// terminal state. + fn unregister_stream_if_needed(&mut self) { + if let Some(stream_id) = self.stream_id.take() { + self.shared_file_stream_state.unregister_stream(stream_id); + } + } + + /// Run a planner on CPU until it either needs I/O or fully completes. + /// + /// Any morsels produced along the way are appended to `self.morsels`. If + /// the planner needs more I/O, it is moved to `waiting_planners`. + fn plan_morsels( + &mut self, + mut planner: Box, + io_permit: OutstandingIoPermit, + ) -> Result<()> { + let max_buffered_morsels = max_buffered_morsels(); + let mut io_permit = Some(io_permit); + while let Some(mut plan) = planner.plan()? { + self.push_ready_morsels(plan.take_morsels()); + self.push_ready_planners(plan.take_planners()); + if let Some(io_future) = plan.take_io_future() { + self.queues.push_waiting_planner(WaitingPlanner::new( + planner, + io_future, + io_permit + .take() + .expect("planner I/O permit should be available"), + )); + break; + } + + if self.queues.morsel_len() >= max_buffered_morsels { + self.push_ready_planners(std::iter::once(planner)); + break; + } + } + Ok(()) + } + + /// Turn one file into one or more planners and immediately drive each of + /// them into the ready queue. + /// + /// The actual `plan()` calls happen in `poll_inner` once the stream has + /// acquired a shared permit to potentially issue another outstanding I/O. + fn morselize_next_file(&mut self, file: PartitionedFile) -> Result<()> { + for planner in self.morselizer.morselize(file)? { + self.push_ready_planners(std::iter::once(planner)); + } + Ok(()) + } + + /// Pull additional files into the planner pipeline until the configured + /// planner concurrency target is reached. + /// + /// This is where new file-level work enters the stream. Formats that do all + /// of their planning synchronously may immediately populate `self.morsels`, + /// while formats that need metadata I/O will populate `waiting_planners`. + fn start_next_files(&mut self) -> Result<()> { + let max_buffered_morsels = max_buffered_morsels(); + let local_planner_target = + self.shared_file_stream_state.max_outstanding_ios().max(1); + while self.queues.planner_count() < local_planner_target { + // In ordered mode, do not admit later files while there is any + // earlier file work still buffered, waiting on I/O, or actively + // being scanned. This keeps file-level planning from introducing + // later output ahead of earlier files. + if self.preserve_order + && (self.reader.is_some() + || self.queues.has_morsels() + || self.queues.has_ready_planners() + || self.queues.has_waiting_planners()) + { + break; + } + if self.queues.morsel_len() >= max_buffered_morsels { + break; + } + let Some(file) = self.queues.pop_file() else { + break; + }; + self.morselize_next_file(file)?; + } + Ok(()) + } + + /// Poll each waiting planner's outstanding I/O once. + /// + /// When a future completes successfully, the planner becomes CPU-ready + /// again and is moved back to `ready_planners`. Failed futures are handled + /// according to `OnError`. + fn check_io(&mut self, cx: &mut Context<'_>) -> Result<()> { + for mut waiting_planner in self.queues.take_waiting_planners() { + match waiting_planner.io_future.poll_unpin(cx) { + Poll::Ready(Ok(())) => { + self.file_stream_metrics.files_opened.add(1); + self.push_ready_planners(std::iter::once(waiting_planner.planner)); + } + Poll::Ready(Err(e)) => { + self.file_stream_metrics.file_open_errors.add(1); + match self.on_error { + OnError::Skip => { + self.file_stream_metrics.files_processed.add(1); + } + OnError::Fail => return Err(e), + } + } + Poll::Pending => self.queues.push_waiting_planner(waiting_planner), + } + } + Ok(()) + } + + /// Convert the next ready morsel into an active `RecordBatch` reader. + /// + /// This only happens when there is no reader currently in flight. The + /// corresponding scan timers start here because the morsel is now eligible + /// to produce batches. + fn start_next_morsel(&mut self) { + if self.reader.is_none() + && let Some(morsel) = self.queues.pop_morsel() + { + self.reader = Some(morsel.into_stream()); + self.file_stream_metrics.time_scanning_until_data.start(); + self.file_stream_metrics.time_scanning_total.start(); + } + } + + /// Drive the `FileStream` scheduler forward by one poll. + /// + /// The order is important: + /// 1. Admit more files into the planner pipeline up to the concurrency + /// target (ensures I/O are scheduled if needed) + /// 2. Poll outstanding planner I/O (ensure I/O completes in parallel) + /// 3. Spend CPU on ready planners only when there is no morsel already ready + /// to execute. + /// 4. Launch and poll the active morsel reader. + fn poll_inner(&mut self, cx: &mut Context<'_>) -> Poll>> { + loop { + match self.state { + StreamState::Active => {} + StreamState::Done => { + self.unregister_stream_if_needed(); + return Poll::Ready(None); + } + StreamState::Error => { + self.unregister_stream_if_needed(); + return Poll::Ready(None); + } + } + + if let Err(e) = self.start_next_files() { + self.queues.clear(); + self.state = StreamState::Error; + self.unregister_stream_if_needed(); + return Poll::Ready(Some(Err(e))); + } + if let Err(e) = self.check_io(cx) { + self.queues.clear(); + self.state = StreamState::Error; + self.unregister_stream_if_needed(); + return Poll::Ready(Some(Err(e))); + } + + // Opportunistically refill the local queues from shared ready work + // before spending more CPU locally. We intentionally ignore the + // return value here because this is only a best-effort steal: the + // normal scheduler flow below will observe any newly queued work. + let _ = self.try_steal_ready_work(); + + // Give ready planners CPU whenever there is buffer space, even if a + // reader is currently active. This avoids starving planner work + // behind a reader that is itself waiting on I/O. + while self.queues.morsel_len() < max_buffered_morsels() { + // In ordered mode, once an earlier planner has produced a + // morsel or is blocked on I/O, do not advance later sibling + // planners yet. This preserves the logical `MorselPlan` order: + // direct morsels first, then child planners in API order. + if self.preserve_order + && (self.reader.is_some() + || self.queues.has_morsels() + || self.queues.has_waiting_planners()) + { + break; + } + let stream_id = match self.stream_id() { + Ok(stream_id) => stream_id, + Err(e) => { + self.queues.clear(); + self.state = StreamState::Error; + self.unregister_stream_if_needed(); + return Poll::Ready(Some(Err(e))); + } + }; + let Some(io_permit) = self + .shared_file_stream_state + .try_acquire_io_permit(stream_id) + else { + self.shared_file_stream_state + .register_waker(stream_id, cx.waker()); + break; + }; + let Some(planner) = self.queues.pop_ready_planner() else { + drop(io_permit); + break; + }; + if let Err(e) = self.plan_morsels(planner, io_permit) { + self.queues.clear(); + self.state = StreamState::Error; + self.unregister_stream_if_needed(); + return Poll::Ready(Some(Err(e))); + } + + // Once a morsel is buffered and a reader is already active, + // return to the scan side of the scheduler rather than + // continuing to spend CPU on planning in this poll. + if self.reader.is_some() && self.queues.has_morsels() { + break; + } + } + + // Newly planned work may have just discovered fresh I/O. Poll it + // once now so the future can register the current waker before we + // return `Pending`; otherwise the stream can stall waiting on an + // I/O future that has never been polled. + if let Err(e) = self.check_io(cx) { + self.queues.clear(); + self.state = StreamState::Error; + self.unregister_stream_if_needed(); + return Poll::Ready(Some(Err(e))); + } + + // After polling I/O, see if a sibling published newly ready work. + // The boolean result is ignored because this is only an + // opportunistic prefetch into the local queues; the subsequent + // checks for local planners/morsels will handle any stolen work. + let _ = self.try_steal_ready_work(); + + // The second I/O poll may have completed planner work discovered + // during this same call to `poll_inner`. Loop back so newly ready + // planners get CPU time before we consider returning `Pending`. + if self.queues.has_ready_planners() + && self.queues.morsel_len() < max_buffered_morsels() + // In ordered mode, only loop back for more planner CPU when + // there is no earlier reader, buffered morsel, or waiting I/O + // that should be drained first. Otherwise, drop to + // `start_next_morsel()` so output is produced in order. + && (!self.preserve_order + || (self.reader.is_none() + && !self.queues.has_morsels() + && !self.queues.has_waiting_planners())) + { + continue; + } + + self.start_next_morsel(); + + if let Some(reader) = self.reader.as_mut() { + match reader.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + self.file_stream_metrics.time_scanning_until_data.stop(); + self.file_stream_metrics.time_scanning_total.stop(); + let batch = match &mut self.remain { + Some(remain) => { + if batch.num_rows() > *remain { + let batch = batch.slice(0, *remain); + *remain = 0; + self.state = StreamState::Done; + batch + } else { + *remain -= batch.num_rows(); + batch + } + } + None => batch, + }; + self.file_stream_metrics.time_scanning_total.start(); + return Poll::Ready(Some(Ok(batch))); + } + Poll::Ready(Some(Err(e))) => { + self.reader = None; + self.file_stream_metrics.file_scan_errors.add(1); + self.file_stream_metrics.time_scanning_until_data.stop(); + self.file_stream_metrics.time_scanning_total.stop(); + + match self.on_error { + OnError::Fail => { + self.queues.clear(); + self.state = StreamState::Error; + self.unregister_stream_if_needed(); + return Poll::Ready(Some(Err(e))); + } + OnError::Skip => { + self.file_stream_metrics.files_processed.add(1); + continue; + } + } + } + Poll::Ready(None) => { + self.reader = None; + self.file_stream_metrics.files_processed.add(1); + self.file_stream_metrics.time_scanning_until_data.stop(); + self.file_stream_metrics.time_scanning_total.stop(); + continue; + } + Poll::Pending => {} + } + } + + if self.reader.is_none() && self.queues.is_empty() { + self.state = StreamState::Done; + self.unregister_stream_if_needed(); + return Poll::Ready(None); + } + + // try and find more work if possible, but if not,wait on the waker + if !self.try_steal_ready_work() + && self.can_share_ready_work() + && self.reader.is_none() + && !self.queues.has_morsels() + && !self.queues.has_ready_planners() + { + let stream_id = match self.stream_id() { + Ok(stream_id) => stream_id, + Err(e) => { + self.queues.clear(); + self.state = StreamState::Error; + self.unregister_stream_if_needed(); + return Poll::Ready(Some(Err(e))); + } + }; + self.shared_file_stream_state + .register_waker(stream_id, cx.waker()); + // If the active reader just returned `Pending`, yield back to the + // executor instead of looping immediately. Otherwise a reader that + // needs more I/O can hot-loop inside `poll_inner` as long as there + // is buffered work behind it, repeatedly polling the same pending + // reader without giving the executor a chance to wake it. + } else if self.reader.is_none() + && (self.queues.has_morsels() || self.queues.has_ready_planners()) + { + continue; + } + + return Poll::Pending; + } + } +} + +/// A planner that has already discovered its next I/O phase. +struct WaitingPlanner { + planner: Box, + io_future: BoxFuture<'static, Result<()>>, + _io_permit: OutstandingIoPermit, +} + +impl WaitingPlanner { + fn new( + planner: Box, + io_future: BoxFuture<'static, Result<()>>, + io_permit: OutstandingIoPermit, + ) -> Self { + Self { + planner, + io_future, + _io_permit: io_permit, + } + } +} + +impl std::fmt::Debug for WaitingPlanner { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("WaitingPlanner").finish_non_exhaustive() + } +} + +impl Drop for FileStream { + fn drop(&mut self) { + // Release any outstanding permits before unregistering this stream + // from the shared scheduler. + self.queues.take_waiting_planners(); + self.unregister_stream_if_needed(); + } +} + +impl Stream for FileStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + self.file_stream_metrics.time_processing.start(); + let result = self.poll_inner(cx); + self.file_stream_metrics.time_processing.stop(); + self.baseline_metrics.record_poll(result) + } +} + +impl RecordBatchStream for FileStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.projected_schema) + } +} + +/// A fallible future that resolves to a stream of [`RecordBatch`] +/// +/// This is typically an `async` function that opens the file, and returns a +/// stream that reads the file and produces `RecordBatch`es. +pub type FileOpenFuture = + BoxFuture<'static, Result>>>; + +/// Describes the behavior of the `FileStream` if file opening or scanning fails +#[derive(Default)] +pub enum OnError { + /// Fail the entire stream and return the underlying error + #[default] + Fail, + /// Continue scanning, ignoring the failed file + Skip, +} + +/// Generic API for opening a file using an [`ObjectStore`] and resolving to a +/// stream of [`RecordBatch`] +/// +/// [`ObjectStore`]: object_store::ObjectStore +pub trait FileOpener: Unpin + Send + Sync { + /// Asynchronously open the specified file and return a stream + /// of [`RecordBatch`] + /// + /// TODO: describe prefetching behavior here, and expectations around IO + fn open(&self, partitioned_file: PartitionedFile) -> Result; +} + +/// A timer that can be started and stopped. +pub struct StartableTime { + pub metrics: Time, + // use for record each part cost time, will eventually add into 'metrics'. + pub start: Option, +} + +impl StartableTime { + pub fn start(&mut self) { + assert!(self.start.is_none()); + self.start = Some(Instant::now()); + } + + pub fn stop(&mut self) { + if let Some(start) = self.start.take() { + self.metrics.add_elapsed(start); + } + } +} + +/// Metrics for [`FileStream`] +/// +/// Note that all of these metrics are in terms of wall clock time +/// (not cpu time) so they include time spent waiting on I/O as well +/// as other operators. +/// +/// [`FileStream`]: +pub struct FileStreamMetrics { + /// Wall clock time elapsed for file opening. + /// + /// Time between when [`FileOpener::open`] is called and when the + /// [`FileStream`] receives a stream for reading. + /// + /// If there are multiple files being scanned, the stream + /// will open the next file in the background while scanning the + /// current file. This metric will only capture time spent opening + /// while not also scanning. + /// [`FileStream`]: + pub time_opening: StartableTime, + /// Wall clock time elapsed for file scanning + first record batch of decompression + decoding + /// + /// Time between when the [`FileStream`] requests data from the + /// stream and when the first [`RecordBatch`] is produced. + /// [`FileStream`]: + pub time_scanning_until_data: StartableTime, + /// Total elapsed wall clock time for scanning + record batch decompression / decoding + /// + /// Sum of time between when the [`FileStream`] requests data from + /// the stream and when a [`RecordBatch`] is produced for all + /// record batches in the stream. Note that this metric also + /// includes the time of the parent operator's execution. + pub time_scanning_total: StartableTime, + /// Wall clock time elapsed for data decompression + decoding + /// + /// Time spent waiting for the FileStream's input. + pub time_processing: StartableTime, + /// Count of errors opening file. + /// + /// If using `OnError::Skip` this will provide a count of the number of files + /// which were skipped and will not be included in the scan results. + pub file_open_errors: Count, + /// Count of errors scanning file + /// + /// If using `OnError::Skip` this will provide a count of the number of files + /// which were skipped and will not be included in the scan results. + pub file_scan_errors: Count, + /// Count of files successfully opened or evaluated for processing. + /// At t=end (completion of a query) this is equal to `files_opened`, and both values are equal + /// to the total number of files in the query; unless the query itself fails. + /// This value will always be greater than or equal to `files_open`. + /// Note that this value does *not* mean the file was actually scanned. + /// We increment this value for any processing of a file, even if that processing is + /// discarding it because we hit a `LIMIT` (in this case `files_opened` and `files_processed` are both incremented at the same time). + pub files_opened: Count, + /// Count of files completely processed / closed (opened, pruned, or skipped due to limit). + /// At t=0 (the beginning of a query) this is 0. + /// At t=end (completion of a query) this is equal to `files_opened`, and both values are equal + /// to the total number of files in the query; unless the query itself fails. + /// This value will always be less than or equal to `files_open`. + /// We increment this value for any processing of a file, even if that processing is + /// discarding it because we hit a `LIMIT` (in this case `files_opened` and `files_processed` are both incremented at the same time). + pub files_processed: Count, +} + +impl FileStreamMetrics { + pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { + let time_opening = StartableTime { + metrics: MetricBuilder::new(metrics) + .subset_time("time_elapsed_opening", partition), + start: None, + }; + + let time_scanning_until_data = StartableTime { + metrics: MetricBuilder::new(metrics) + .subset_time("time_elapsed_scanning_until_data", partition), + start: None, + }; + + let time_scanning_total = StartableTime { + metrics: MetricBuilder::new(metrics) + .subset_time("time_elapsed_scanning_total", partition), + start: None, + }; + + let time_processing = StartableTime { + metrics: MetricBuilder::new(metrics) + .subset_time("time_elapsed_processing", partition), + start: None, + }; + + let file_open_errors = + MetricBuilder::new(metrics).counter("file_open_errors", partition); + + let file_scan_errors = + MetricBuilder::new(metrics).counter("file_scan_errors", partition); + + let files_opened = MetricBuilder::new(metrics).counter("files_opened", partition); + + let files_processed = + MetricBuilder::new(metrics).counter("files_processed", partition); + + Self { + time_opening, + time_scanning_until_data, + time_scanning_total, + time_processing, + file_open_errors, + file_scan_errors, + files_opened, + files_processed, + } + } +} + +#[cfg(test)] +mod tests { + use crate::file_groups::FileGroup; + use crate::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; + use crate::morsel::test_utils::{ + IoFutureId, MockMorselSpec, MockMorselizer, MockPlanner, MorselId, + MorselObserver, PlannerId, ReturnPlanBuilder, + }; + use crate::tests::make_partition; + use crate::{PartitionedFile, TableSchema}; + use arrow::datatypes::Int32Type; + use datafusion_common::error::Result; + use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; + use std::sync::Arc; + use std::sync::atomic::{AtomicUsize, Ordering}; + + use crate::file_stream::{ + FileOpenFuture, FileOpener, FileStream, FileStreamBuilder, OnError, + SharedFileStreamMode, SharedFileStreamState, + }; + use crate::test_util::MockSource; + use arrow::array::{Array, AsArray, RecordBatch}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::{assert_batches_eq, exec_err, internal_err}; + use futures::{FutureExt, StreamExt}; + + /// Test `FileOpener` which will simulate errors during file opening or scanning + #[derive(Default)] + struct TestOpener { + /// Index in stream of files which should throw an error while opening + error_opening_idx: Vec, + /// Index in stream of files which should throw an error while scanning + error_scanning_idx: Vec, + /// Index of last file in stream + current_idx: AtomicUsize, + /// `RecordBatch` to return + records: Vec, + } + + impl FileOpener for TestOpener { + fn open(&self, _partitioned_file: PartitionedFile) -> Result { + let idx = self.current_idx.fetch_add(1, Ordering::SeqCst); + + if self.error_opening_idx.contains(&idx) { + Ok(futures::future::ready(internal_err!("error opening")).boxed()) + } else if self.error_scanning_idx.contains(&idx) { + let error = futures::future::ready(exec_err!("error scanning")); + let stream = futures::stream::once(error).boxed(); + Ok(futures::future::ready(Ok(stream)).boxed()) + } else { + let iterator = self.records.clone().into_iter().map(Ok); + let stream = futures::stream::iter(iterator).boxed(); + Ok(futures::future::ready(Ok(stream)).boxed()) + } + } + } + + #[derive(Default)] + struct FileStreamTest { + /// Number of files in the stream + num_files: usize, + /// Global limit of records emitted by the stream + limit: Option, + /// Error-handling behavior of the stream + on_error: OnError, + /// Mock `FileOpener` + opener: TestOpener, + } + + impl FileStreamTest { + pub fn new() -> Self { + Self::default() + } + + /// Specify the number of files in the stream + pub fn with_num_files(mut self, num_files: usize) -> Self { + self.num_files = num_files; + self + } + + /// Specify the limit + pub fn with_limit(mut self, limit: Option) -> Self { + self.limit = limit; + self + } + + /// Specify the index of files in the stream which should + /// throw an error when opening + pub fn with_open_errors(mut self, idx: Vec) -> Self { + self.opener.error_opening_idx = idx; + self + } + + /// Specify the index of files in the stream which should + /// throw an error when scanning + pub fn with_scan_errors(mut self, idx: Vec) -> Self { + self.opener.error_scanning_idx = idx; + self + } + + /// Specify the behavior of the stream when an error occurs + pub fn with_on_error(mut self, on_error: OnError) -> Self { + self.on_error = on_error; + self + } + + /// Specify the record batches that should be returned from each + /// file that is successfully scanned + pub fn with_records(mut self, records: Vec) -> Self { + self.opener.records = records; + self + } + + /// Collect the results of the `FileStream` + pub async fn result(self) -> Result> { + let file_schema = self + .opener + .records + .first() + .map(|batch| batch.schema()) + .unwrap_or_else(|| Arc::new(Schema::empty())); + + // let ctx = SessionContext::new(); + let mock_files: Vec<(String, u64)> = (0..self.num_files) + .map(|idx| (format!("mock_file{idx}"), 10_u64)) + .collect(); + + // let mock_files_ref: Vec<(&str, u64)> = mock_files + // .iter() + // .map(|(name, size)| (name.as_str(), *size)) + // .collect(); + + let file_group = mock_files + .into_iter() + .map(|(name, size)| PartitionedFile::new(name, size)) + .collect(); + + let on_error = self.on_error; + + let table_schema = TableSchema::new(file_schema, vec![]); + let config = FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + Arc::new(MockSource::new(table_schema)), + ) + .with_file_group(file_group) + .with_limit(self.limit) + .build(); + let metrics_set = ExecutionPlanMetricsSet::new(); + let file_stream = + FileStreamBuilder::new(&config, 0, Arc::new(self.opener), &metrics_set) + .with_on_error(on_error) + .build() + .unwrap(); + + file_stream + .collect::>() + .await + .into_iter() + .collect::>>() + } + } + + /// helper that creates a stream of 2 files with the same pair of batches in each ([0,1,2] and [0,1]) + async fn create_and_collect(limit: Option) -> Vec { + FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_limit(limit) + .result() + .await + .expect("error executing stream") + } + + /// Helper for morsel-driven `FileStream` tests that bundles the mock + /// `Morselizer` setup with the corresponding `FileScanConfig`. + #[derive(Clone)] + struct MorselTest { + morselizer: MockMorselizer, + file_names: Vec, + preserve_order: bool, + event_summaries: bool, + } + + impl MorselTest { + /// Create an empty morsel-driven test harness. + fn new() -> Self { + Self { + morselizer: MockMorselizer::new(), + file_names: vec![], + preserve_order: false, + event_summaries: false, + } + } + + /// Add one file and its root mock planner to the test input. + fn with_file(mut self, path: impl Into, planner: MockPlanner) -> Self { + let path = path.into(); + self.morselizer = self.morselizer.with_file(path.clone(), planner); + self.file_names.push(path); + self + } + + /// Run this test harness with ordered output semantics enabled. + fn with_preserve_order(mut self, preserve_order: bool) -> Self { + self.preserve_order = preserve_order; + self + } + + /// Snapshot only the higher-level scheduler events. + /// + /// The full event trace is still useful for detailed tests, but for + /// more complex tests those lower level events obscure the important + /// events. + fn with_event_summaries(mut self) -> Self { + self.event_summaries = true; + self + } + + /// Build the `FileScanConfig` corresponding to the configured mock + /// file set. + fn test_config(&self) -> FileScanConfig { + let file_group = self + .file_names + .iter() + .map(|name| PartitionedFile::new(name, 10)) + .collect(); + let table_schema = TableSchema::new( + Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, false)])), + vec![], + ); + FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + Arc::new(MockSource::new(table_schema)), + ) + .with_file_group(file_group) + .with_preserve_order(self.preserve_order) + .build() + } + + async fn run(self) -> Result { + // handle to shared observer + let observer = self.morselizer.observer().clone(); + // Clear any prior observer events before running the test, so the + // snapshot only includes events from this run. + observer.clear(); + + let config = self.test_config(); + let metrics_set = ExecutionPlanMetricsSet::new(); + let mut stream = FileStreamBuilder::new_with_morselizer( + &config, + 0, + Box::new(self.morselizer), + &metrics_set, + ) + .build()?; + + let mut stream_contents = Vec::new(); + while let Some(result) = stream.next().await { + match result { + Ok(batch) => { + // Each batch should have a single int32 column with the + // mocked batch id, which keeps snapshot output compact. + let col = batch.column(0).as_primitive::(); + assert_eq!(col.len(), 1); + assert!(col.is_valid(0)); + let batch_id = col.value(0); + stream_contents.push(format!("Batch: {batch_id}")); + } + Err(e) => { + stream_contents.push(format!("Error: {e}")); + } + } + } + stream_contents.push("Done".to_string()); + let output = stream_contents.join("\n"); + + // Snapshot both the produced output and the scheduler trace + // together. This makes scheduler changes much easier to review than + // maintaining long hand-written event assertions separately. + let mut parts = vec!["----- Output Stream -----".to_string(), output]; + parts.push("----- File Stream Events -----".to_string()); + let events = if self.event_summaries { + observer.format_summary_events() + } else { + observer.format_events() + }; + parts.push(events); + Ok(parts.join("\n")) + } + } + + /// Helper for multi-stream morsel tests that share one + /// [`SharedFileStreamState`]. + #[derive(Clone)] + struct MultiStreamMorselTest { + /// Shared mock morselizer used by all sibling streams in the test. + morselizer: MockMorselizer, + /// Per-partition file assignments used to build one sibling + /// `FileStream` per partition. + partitions: Vec>, + /// The sequence of sibling streams to poll while exercising the + /// stealing scenario under test. + reads: Vec, + } + + /// Identifies one sibling stream in a [`MultiStreamMorselTest`]. + #[derive(Debug, Clone, Copy)] + struct TestStreamId(usize); + + impl MultiStreamMorselTest { + /// Create a sibling-stream test harness with `num_partitions` + /// independent `FileStream`s. + fn new(num_partitions: usize) -> Self { + Self { + morselizer: MockMorselizer::new(), + partitions: vec![vec![]; num_partitions], + reads: vec![], + } + } + + /// Add one file and its root planner to a specific sibling stream. + /// + /// This lets tests control which stream owns the original file-local + /// work before stealing redistributes ready morsels or planners. + fn with_file_in_partition( + mut self, + partition: usize, + path: impl Into, + planner: MockPlanner, + ) -> Self { + let path = path.into(); + self.morselizer = self.morselizer.with_file(path.clone(), planner); + self.partitions[partition].push(path); + self + } + + /// Configure the order in which sibling streams are polled while the + /// test scenario is executing. + fn with_reads(mut self, reads: Vec) -> Self { + self.reads = reads; + self + } + + /// Build a multi-partition `FileScanConfig` matching the configured + /// sibling test layout. + fn test_config(&self) -> FileScanConfig { + let table_schema = TableSchema::new( + Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, false)])), + vec![], + ); + let mut builder = FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + Arc::new(MockSource::new(table_schema)), + ); + for file_group in &self.partitions { + let file_group = file_group + .iter() + .map(|name| PartitionedFile::new(name, 10)) + .collect::>(); + builder = builder.with_file_group(FileGroup::new(file_group)); + } + builder.build() + } + + /// Build one `FileStream` per configured partition, all sharing the + /// same `SharedFileStreamState`. + /// + /// This is the core helper for stealing tests: separate streams have + /// distinct local queues, but share the same outstanding-I/O budget + /// and shared ready-work queues. + fn build_streams(&self) -> Result<(MorselObserver, Vec)> { + let observer = self.morselizer.observer().clone(); + observer.clear(); + + let config = self.test_config(); + let shared_state = + SharedFileStreamState::new(2, SharedFileStreamMode::Unordered); + let metrics_set = ExecutionPlanMetricsSet::new(); + let streams = (0..self.partitions.len()) + .map(|partition| { + FileStreamBuilder::new_with_morselizer( + &config, + partition, + Box::new(self.morselizer.clone()), + &metrics_set, + ) + .with_shared_state(shared_state.clone()) + .build() + }) + .collect::>>()?; + + Ok((observer, streams)) + } + + /// Run the configured poll sequence and format the per-stream outputs + /// plus shared scheduler events into one snapshot string. + async fn run(self) -> Result { + let reads = self.reads.clone(); + let (observer, mut streams) = self.build_streams()?; + let mut outputs = vec![vec![]; streams.len()]; + + for stream_id in reads { + let batch_id = next_batch_id(&mut streams[stream_id.0]).await?; + assert!( + batch_id.is_some(), + "expected stream {stream_id:?} to produce a batch" + ); + outputs[stream_id.0].push(batch_id.unwrap()); + } + + for stream in &mut streams { + assert_eq!(next_batch_id(stream).await?, None); + } + + let mut parts = vec![]; + for (idx, output) in outputs.iter().enumerate() { + parts.push(format!("----- Stream {idx} Output -----")); + parts.push( + output + .iter() + .map(|batch_id| format!("Batch: {batch_id}")) + .chain(std::iter::once("Done".to_string())) + .collect::>() + .join("\n"), + ); + } + parts.push("----- File Stream Events -----".to_string()); + parts.push(observer.format_summary_events()); + Ok(parts.join("\n")) + } + } + + /// Read the next single-row batch from a test stream and return its batch + /// id. + async fn next_batch_id(stream: &mut FileStream) -> Result> { + let batch = stream.next().await.transpose()?; + Ok(batch.map(|batch| { + let col = batch.column(0).as_primitive::(); + assert_eq!(col.len(), 1); + assert!(col.is_valid(0)); + col.value(0) + })) + } + + /// Verifies the simplest morsel-driven flow: one planner produces one + /// morsel immediately, and the morsel is then scanned to completion. + #[tokio::test] + async fn morsel_framework_single_morsel_no_io() -> Result<()> { + let test = MorselTest::new().with_file( + "file1.parquet", + MockPlanner::builder() + .with_id(PlannerId(0)) + .return_morsel(MorselId(10), 42) + .return_none() + .build(), + ); + + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Output Stream ----- + Batch: 42 + Done + ----- File Stream Events ----- + morselize_file: file1.parquet + planner_created: PlannerId(0) + planner_called: PlannerId(0) + morsel_produced: PlannerId(0), MorselId(10) + planner_called: PlannerId(0) + morsel_stream_started: MorselId(10) + morsel_stream_batch_produced: MorselId(10), BatchId(42) + morsel_stream_finished: MorselId(10) + "); + + Ok(()) + } + + /// Verifies that a planner can block on one I/O phase, resume, and only + /// then produce its morsel. + #[tokio::test] + async fn morsel_framework_single_morsel_io() -> Result<()> { + let test = MorselTest::new().with_file( + "file1.parquet", + MockPlanner::builder() + .with_id(PlannerId(0)) + .return_plan(ReturnPlanBuilder::new().with_io(IoFutureId(100), 1)) + .return_morsel(MorselId(10), 42) + .return_none() + .build(), + ); + + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Output Stream ----- + Batch: 42 + Done + ----- File Stream Events ----- + morselize_file: file1.parquet + planner_created: PlannerId(0) + planner_called: PlannerId(0) + io_future_created: PlannerId(0), IoFutureId(100) + io_future_polled: PlannerId(0), IoFutureId(100) + io_future_polled: PlannerId(0), IoFutureId(100) + io_future_resolved: PlannerId(0), IoFutureId(100) + planner_called: PlannerId(0) + morsel_produced: PlannerId(0), MorselId(10) + planner_called: PlannerId(0) + morsel_stream_started: MorselId(10) + morsel_stream_batch_produced: MorselId(10), BatchId(42) + morsel_stream_finished: MorselId(10) + "); + + Ok(()) + } + + /// Verifies that a planner can require multiple CPU-only `plan()` calls + /// before it discovers any morsels or I/O, matching the staged behavior of + /// the Parquet morsel planner. + #[tokio::test] + async fn morsel_framework_two_cpu_steps_before_morsel() -> Result<()> { + let test = MorselTest::new().with_file( + "file1.parquet", + MockPlanner::builder() + .with_id(PlannerId(0)) + .return_plan(ReturnPlanBuilder::new()) + .return_plan(ReturnPlanBuilder::new()) + .return_morsel(MorselId(10), 42) + .return_none() + .build(), + ); + + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Output Stream ----- + Batch: 42 + Done + ----- File Stream Events ----- + morselize_file: file1.parquet + planner_created: PlannerId(0) + planner_called: PlannerId(0) + planner_called: PlannerId(0) + planner_called: PlannerId(0) + morsel_produced: PlannerId(0), MorselId(10) + planner_called: PlannerId(0) + morsel_stream_started: MorselId(10) + morsel_stream_batch_produced: MorselId(10), BatchId(42) + morsel_stream_finished: MorselId(10) + "); + + Ok(()) + } + + /// Verifies direct morsels returned from a planner are consumed before + /// batches produced by any returned child planners. + #[tokio::test] + async fn morsel_framework_morsels_before_child_planner() -> Result<()> { + let child_planner = MockPlanner::builder() + .with_id(PlannerId(1)) + .return_morsel(MorselId(11), 43) + .return_none() + .build(); + + // planner 0 returns batch 42 + let parent_planner = MockPlanner::builder() + .with_id(PlannerId(0)) + .return_plan( + ReturnPlanBuilder::new() + .with_morsel(MockMorselSpec::single_batch(MorselId(10), 42)) + .with_planner(child_planner), + ) + .return_none() + .build(); + + let test = MorselTest::new() + .with_file("file1.parquet", parent_planner) + .with_event_summaries(); + + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Output Stream ----- + Batch: 42 + Batch: 43 + Done + ----- File Stream Events ----- + morselize_file: file1.parquet + planner_created: PlannerId(0) + morsel_produced: PlannerId(0), MorselId(10) + planner_produced_child: PlannerId(0) -> PlannerId(1) + morsel_produced: PlannerId(1), MorselId(11) + morsel_stream_batch_produced: MorselId(10), BatchId(42) + morsel_stream_batch_produced: MorselId(11), BatchId(43) + "); + + Ok(()) + } + + /// Verifies the non-ordered behavior for child planners: if the first child + /// planner blocks on I/O and the second can make progress immediately, the + /// second planner's batches are emitted first. + #[tokio::test] + async fn morsel_framework_child_planner_reorder() -> Result<()> { + let planner_1 = MockPlanner::builder() + .with_id(PlannerId(1)) + // Note IO required 2 polls + .return_plan(ReturnPlanBuilder::new().with_io(IoFutureId(100), 2)) + .return_morsel(MorselId(11), 41) + .return_none() + .build(); + let planner_2 = MockPlanner::builder() + .with_id(PlannerId(2)) + // IO only requires 1 poll, so it will resolve before planner 1's IO + .return_plan(ReturnPlanBuilder::new().with_io(IoFutureId(101), 1)) // IO returns after 1 poll + .return_morsel(MorselId(12), 42) + .return_none() + .build(); + + let parent_planner = MockPlanner::builder() + .with_id(PlannerId(0)) + .return_plan( + ReturnPlanBuilder::new() + .with_planner(planner_1) + .with_planner(planner_2), + ) + .return_none() + .build(); + + let test = MorselTest::new().with_file("file1.parquet", parent_planner); + + // Expect both futures to be polled, but second planner's (42) batch to be + // produced first + insta::assert_snapshot!(test.clone().run().await.unwrap(), @r" + ----- Output Stream ----- + Batch: 42 + Batch: 41 + Done + ----- File Stream Events ----- + morselize_file: file1.parquet + planner_created: PlannerId(0) + planner_called: PlannerId(0) + planner_produced_child: PlannerId(0) -> PlannerId(1) + planner_produced_child: PlannerId(0) -> PlannerId(2) + planner_called: PlannerId(0) + planner_called: PlannerId(1) + io_future_created: PlannerId(1), IoFutureId(100) + planner_called: PlannerId(2) + io_future_created: PlannerId(2), IoFutureId(101) + io_future_polled: PlannerId(1), IoFutureId(100) + io_future_polled: PlannerId(2), IoFutureId(101) + io_future_polled: PlannerId(1), IoFutureId(100) + io_future_polled: PlannerId(2), IoFutureId(101) + io_future_resolved: PlannerId(2), IoFutureId(101) + planner_called: PlannerId(2) + morsel_produced: PlannerId(2), MorselId(12) + planner_called: PlannerId(2) + io_future_polled: PlannerId(1), IoFutureId(100) + io_future_resolved: PlannerId(1), IoFutureId(100) + planner_called: PlannerId(1) + morsel_produced: PlannerId(1), MorselId(11) + planner_called: PlannerId(1) + morsel_stream_started: MorselId(12) + morsel_stream_batch_produced: MorselId(12), BatchId(42) + morsel_stream_finished: MorselId(12) + morsel_stream_started: MorselId(11) + morsel_stream_batch_produced: MorselId(11), BatchId(41) + morsel_stream_finished: MorselId(11) + "); + + // Run same test using `with_preserve_order(true)`, but expect the first + // planner's batch (41) to be produced before the second's (42), even + // though the second planner's I/O resolves first. + let test = test.with_preserve_order(true); + + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Output Stream ----- + Batch: 41 + Batch: 42 + Done + ----- File Stream Events ----- + morselize_file: file1.parquet + planner_created: PlannerId(0) + planner_called: PlannerId(0) + planner_produced_child: PlannerId(0) -> PlannerId(1) + planner_produced_child: PlannerId(0) -> PlannerId(2) + planner_called: PlannerId(0) + planner_called: PlannerId(1) + io_future_created: PlannerId(1), IoFutureId(100) + io_future_polled: PlannerId(1), IoFutureId(100) + io_future_polled: PlannerId(1), IoFutureId(100) + io_future_polled: PlannerId(1), IoFutureId(100) + io_future_resolved: PlannerId(1), IoFutureId(100) + planner_called: PlannerId(2) + io_future_created: PlannerId(2), IoFutureId(101) + io_future_polled: PlannerId(2), IoFutureId(101) + io_future_polled: PlannerId(2), IoFutureId(101) + io_future_resolved: PlannerId(2), IoFutureId(101) + planner_called: PlannerId(1) + morsel_produced: PlannerId(1), MorselId(11) + planner_called: PlannerId(1) + morsel_stream_started: MorselId(11) + morsel_stream_batch_produced: MorselId(11), BatchId(41) + morsel_stream_finished: MorselId(11) + planner_called: PlannerId(2) + morsel_produced: PlannerId(2), MorselId(12) + planner_called: PlannerId(2) + morsel_stream_started: MorselId(12) + morsel_stream_batch_produced: MorselId(12), BatchId(42) + morsel_stream_finished: MorselId(12) + "); + + Ok(()) + } + + /// Verifies that child planners still respect the global outstanding-I/O + /// cap. Even if a parent planner returns three ready children, only two of + /// them should be allowed to create waiting I/O futures at once. + #[tokio::test] + async fn morsel_framework_child_planner_io_respects_global_cap() -> Result<()> { + let planner_1 = MockPlanner::builder() + .with_id(PlannerId(1)) + .return_plan(ReturnPlanBuilder::new().with_io(IoFutureId(100), 1)) + .return_morsel(MorselId(11), 41) + .return_none() + .build(); + let planner_2 = MockPlanner::builder() + .with_id(PlannerId(2)) + .return_plan(ReturnPlanBuilder::new().with_io(IoFutureId(101), 3)) + .return_morsel(MorselId(12), 42) + .return_none() + .build(); + let planner_3 = MockPlanner::builder() + .with_id(PlannerId(3)) + .return_plan(ReturnPlanBuilder::new().with_io(IoFutureId(102), 1)) + .return_morsel(MorselId(13), 43) + .return_none() + .build(); + + let parent_planner = MockPlanner::builder() + .with_id(PlannerId(0)) + .return_plan( + ReturnPlanBuilder::new() + .with_planner(planner_1) + .with_planner(planner_2) + .with_planner(planner_3), + ) + .return_none() + .build(); + + let test = MorselTest::new() + .with_file("file1.parquet", parent_planner) + .with_event_summaries(); + + // Note that the future for planner 1 must resolve before planner 2 begins + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Output Stream ----- + Batch: 41 + Batch: 42 + Batch: 43 + Done + ----- File Stream Events ----- + morselize_file: file1.parquet + planner_created: PlannerId(0) + planner_produced_child: PlannerId(0) -> PlannerId(1) + planner_produced_child: PlannerId(0) -> PlannerId(2) + planner_produced_child: PlannerId(0) -> PlannerId(3) + io_future_created: PlannerId(1), IoFutureId(100) + io_future_created: PlannerId(2), IoFutureId(101) + io_future_resolved: PlannerId(1), IoFutureId(100) + io_future_created: PlannerId(3), IoFutureId(102) + io_future_resolved: PlannerId(2), IoFutureId(101) + io_future_resolved: PlannerId(3), IoFutureId(102) + morsel_produced: PlannerId(1), MorselId(11) + morsel_produced: PlannerId(2), MorselId(12) + morsel_produced: PlannerId(3), MorselId(13) + morsel_stream_batch_produced: MorselId(11), BatchId(41) + morsel_stream_batch_produced: MorselId(12), BatchId(42) + morsel_stream_batch_produced: MorselId(13), BatchId(43) + "); + + Ok(()) + } + + /// Verifies that `FileStream` overlaps planner I/O across multiple files + /// rather than waiting for the first file to finish before starting the + /// second. + #[tokio::test] + async fn morsel_framework_two_files_overlapping_io() -> Result<()> { + let test = MorselTest::new() + .with_file( + "file1.parquet", + MockPlanner::builder() + .with_id(PlannerId(0)) + .return_plan(ReturnPlanBuilder::new().with_io(IoFutureId(100), 1)) + .return_morsel(MorselId(10), 42) + .return_none() + .build(), + ) + .with_file( + "file2.parquet", + MockPlanner::builder() + .with_id(PlannerId(1)) + .return_plan(ReturnPlanBuilder::new().with_io(IoFutureId(101), 1)) + .return_morsel(MorselId(11), 43) + .return_none() + .build(), + ); + + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Output Stream ----- + Batch: 42 + Batch: 43 + Done + ----- File Stream Events ----- + morselize_file: file1.parquet + planner_created: PlannerId(0) + morselize_file: file2.parquet + planner_created: PlannerId(1) + planner_called: PlannerId(0) + io_future_created: PlannerId(0), IoFutureId(100) + planner_called: PlannerId(1) + io_future_created: PlannerId(1), IoFutureId(101) + io_future_polled: PlannerId(0), IoFutureId(100) + io_future_polled: PlannerId(1), IoFutureId(101) + io_future_polled: PlannerId(0), IoFutureId(100) + io_future_resolved: PlannerId(0), IoFutureId(100) + io_future_polled: PlannerId(1), IoFutureId(101) + io_future_resolved: PlannerId(1), IoFutureId(101) + planner_called: PlannerId(0) + morsel_produced: PlannerId(0), MorselId(10) + planner_called: PlannerId(0) + planner_called: PlannerId(1) + morsel_produced: PlannerId(1), MorselId(11) + planner_called: PlannerId(1) + morsel_stream_started: MorselId(10) + morsel_stream_batch_produced: MorselId(10), BatchId(42) + morsel_stream_finished: MorselId(10) + morsel_stream_started: MorselId(11) + morsel_stream_batch_produced: MorselId(11), BatchId(43) + morsel_stream_finished: MorselId(11) + "); + + Ok(()) + } + + /// Verifies that an idle sibling stream can steal ready morsels even when + /// it has no local files of its own. + #[tokio::test] + async fn morsel_framework_sibling_stream_steals_when_only_one_has_files() -> Result<()> + { + let test = MultiStreamMorselTest::new(2) + .with_file_in_partition( + 0, + "file1.parquet", + MockPlanner::builder() + .with_id(PlannerId(0)) + .return_plan( + ReturnPlanBuilder::new() + .return_morsel(MorselId(10), 41) + .return_morsel(MorselId(11), 42), + ) + .return_none() + .build(), + ) + // Poll sibling 0 first so it discovers the file and publishes + // ready morsels. Poll sibling 1 next: because it has no local + // files, any batch it returns must have been stolen from sibling 0. + .with_reads(vec![TestStreamId(0), TestStreamId(1)]); + + insta::assert_snapshot!( + test.run().await.unwrap(), + @r" + ----- Stream 0 Output ----- + Batch: 41 + Done + ----- Stream 1 Output ----- + Batch: 42 + Done + ----- File Stream Events ----- + morselize_file: file1.parquet + planner_created: PlannerId(0) + morsel_produced: PlannerId(0), MorselId(10) + morsel_produced: PlannerId(0), MorselId(11) + morsel_stream_batch_produced: MorselId(10), BatchId(41) + morsel_stream_batch_produced: MorselId(11), BatchId(42) + " + ); + + Ok(()) + } + + /// Verifies that a sibling stream waiting on its own file's I/O can steal + /// ready work from a faster sibling and continue making progress. + #[tokio::test] + async fn morsel_framework_sibling_stream_steals_while_own_file_waits_on_io() + -> Result<()> { + let test = MultiStreamMorselTest::new(2) + .with_file_in_partition( + 0, + "fast.parquet", + MockPlanner::builder() + .with_id(PlannerId(0)) + .return_plan( + ReturnPlanBuilder::new() + .return_morsel(MorselId(10), 41) + .return_morsel(MorselId(11), 42), + ) + .return_none() + .build(), + ) + .with_file_in_partition( + 1, + "slow.parquet", + MockPlanner::builder() + .with_id(PlannerId(1)) + .return_plan(ReturnPlanBuilder::new().with_io(IoFutureId(100), 2)) + .return_morsel(MorselId(12), 51) + .return_none() + .build(), + ) + // Poll sibling 0 first so it publishes one ready morsel from the + // fast file. Poll sibling 1 next while its own file is still + // blocked on I/O: the batch it returns at that point must have + // been stolen from sibling 0. Poll sibling 0 again last so it can + // finish once sibling 1's local I/O has resolved. + .with_reads(vec![TestStreamId(0), TestStreamId(1), TestStreamId(0)]); + + insta::assert_snapshot!( + test.run().await.unwrap(), + @r" + ----- Stream 0 Output ----- + Batch: 41 + Batch: 51 + Done + ----- Stream 1 Output ----- + Batch: 42 + Done + ----- File Stream Events ----- + morselize_file: fast.parquet + planner_created: PlannerId(0) + morsel_produced: PlannerId(0), MorselId(10) + morsel_produced: PlannerId(0), MorselId(11) + morsel_stream_batch_produced: MorselId(10), BatchId(41) + morselize_file: slow.parquet + planner_created: PlannerId(1) + morsel_stream_batch_produced: MorselId(11), BatchId(42) + io_future_created: PlannerId(1), IoFutureId(100) + io_future_resolved: PlannerId(1), IoFutureId(100) + morsel_produced: PlannerId(1), MorselId(12) + morsel_stream_batch_produced: MorselId(12), BatchId(51) + " + ); + + Ok(()) + } + + #[tokio::test] + async fn on_error_opening() -> Result<()> { + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_open_errors(vec![0]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_open_errors(vec![1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_open_errors(vec![0, 1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "++", + "++", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn on_error_scanning_fail() -> Result<()> { + let result = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Fail) + .with_scan_errors(vec![1]) + .result() + .await; + + assert!(result.is_err()); + + Ok(()) + } + + #[tokio::test] + async fn on_error_opening_fail() -> Result<()> { + let result = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Fail) + .with_open_errors(vec![1]) + .result() + .await; + + assert!(result.is_err()); + + Ok(()) + } + + #[tokio::test] + async fn on_error_scanning() -> Result<()> { + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_scan_errors(vec![0]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_scan_errors(vec![1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(2) + .with_on_error(OnError::Skip) + .with_scan_errors(vec![0, 1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "++", + "++", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn on_error_mixed() -> Result<()> { + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(3) + .with_on_error(OnError::Skip) + .with_open_errors(vec![1]) + .with_scan_errors(vec![0]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(3) + .with_on_error(OnError::Skip) + .with_open_errors(vec![0]) + .with_scan_errors(vec![1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(3) + .with_on_error(OnError::Skip) + .with_open_errors(vec![2]) + .with_scan_errors(vec![0, 1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "++", + "++", + ], &batches); + + let batches = FileStreamTest::new() + .with_records(vec![make_partition(3), make_partition(2)]) + .with_num_files(3) + .with_on_error(OnError::Skip) + .with_open_errors(vec![0, 2]) + .with_scan_errors(vec![1]) + .result() + .await?; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "++", + "++", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn without_limit() -> Result<()> { + let batches = create_and_collect(None).await; + + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn with_limit_between_files() -> Result<()> { + let batches = create_and_collect(Some(5)).await; + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "+---+", + ], &batches); + + Ok(()) + } + + #[tokio::test] + async fn with_limit_at_middle_of_batch() -> Result<()> { + let batches = create_and_collect(Some(6)).await; + #[rustfmt::skip] + assert_batches_eq!(&[ + "+---+", + "| i |", + "+---+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 0 |", + "| 1 |", + "| 0 |", + "+---+", + ], &batches); + + Ok(()) + } +} diff --git a/datafusion/datasource/src/file_stream/shared_state.rs b/datafusion/datasource/src/file_stream/shared_state.rs new file mode 100644 index 000000000000..61e1d27123c2 --- /dev/null +++ b/datafusion/datasource/src/file_stream/shared_state.rs @@ -0,0 +1,600 @@ +// 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. + +//! Shared state for sibling [`crate::file_stream::FileStream`] instances. +//! +//! A single `DataSourceExec` may create multiple sibling `FileStream`s, one per +//! output partition. These streams need a place to coordinate shared scan +//! resources such as: +//! +//! - the total number of outstanding planner I/O operations +//! - ready morsels and planners that may be stolen by idle siblings +//! +//! [`SharedFileStreamState`] is that shared home. +//! +//! # Outstanding I/O Scheduling Modes +//! +//! The shared state currently controls planner I/O in one of two modes: +//! +//! - [`SharedFileStreamMode::Unordered`]: Every registered stream has a chance +//! to hold one outstanding request before another stream is allowed to start +//! a second. After that first request per stream, remaining capacity may be +//! used in any order. +//! +//! - [`SharedFileStreamMode::PreserveOrder`]: Outstanding I/Os are split fairly +//! across all active streams, even if some streams do not currently want to +//! issue I/O. This prevents a subset of streams from consuming the full budget +//! and is intended for scans that require stable cross-stream ordering. +//! +//! # Ready Work Stealing +//! +//! In unordered mode, sibling streams may also exchange CPU-ready work through +//! this shared state: +//! +//! - ready morsels may be published into a shared morsel queue +//! - ready planners may be published into a shared planner queue +//! - idle siblings will try to steal a ready morsel first, then a ready +//! planner +//! +//! In preserve-order mode, streams keep their ready morsels and planners on +//! their local per-stream queues so later siblings cannot overtake earlier +//! output. +//! +//! Streams can call [`SharedFileStreamState::unregister_stream`] once they know +//! they will never need another I/O permit. Unregistered streams are removed +//! from future fairness calculations so their share of the budget can be +//! redistributed. + +use super::MorselQueue; +use crate::morsel::{Morsel, MorselPlanner}; +use std::collections::{BTreeMap, VecDeque}; +use std::sync::{Arc, Mutex}; +use std::task::Waker; + +/// Shared scheduling mode for sibling `FileStream`s. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum SharedFileStreamMode { + /// Allow streams to run in any order after each active, I/O-hungry stream + /// has had a chance to start one outstanding I/O. + Unordered, + /// Split outstanding I/O budget fairly across all active streams. + PreserveOrder, +} + +/// Stable identifier for one sibling `FileStream` registered with a shared +/// state object. +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct FileStreamId(usize); + +/// Shared state for all sibling `FileStream`s that belong to one `DataSourceExec`. +/// +/// # Intended Usage +/// +/// Create one `SharedFileStreamState` for the whole `DataSourceExec` and clone +/// it into each sibling `FileStream`. +/// +/// # IO Behavior +/// +/// Each stream should register itself with [`Self::register_stream`] and then: +/// +/// 1. call [`Self::try_acquire_io_permit`] before moving a planner into a +/// waiting-I/O state +/// 2. keep the returned [`OutstandingIoPermit`] alive for as long as that I/O +/// operation is outstanding +/// 3. call [`Self::unregister_stream`] once the stream knows it will never need +/// another I/O permit +/// +/// # Ready Work Behavior +/// +/// In unordered mode, streams may also publish ready morsels and planners into +/// the shared queues via [`Self::push_ready_morsel`] and +/// [`Self::push_ready_planner`]. Idle siblings can then steal that ready work +/// with [`Self::pop_ready_morsel`] and [`Self::pop_ready_planner`]. +/// +/// In preserve-order mode, streams should keep ready morsels and planners on +/// their local per-stream queues rather than publishing them into the shared +/// queues. +/// +/// If no permit is available, the caller should typically: +/// +/// 1. keep the planner in a CPU-ready state locally +/// 2. register the current task's waker with [`Self::register_waker`] +/// 3. return `Poll::Pending` +/// +/// The shared state will wake waiting tasks whenever shared capacity may have +/// become available again, or when new shared ready work is published. +#[derive(Clone, Debug)] +pub struct SharedFileStreamState { + inner: Arc>, +} + +#[derive(Debug)] +struct SharedFileStreamStateInner { + /// Shared outstanding-I/O accounting and fairness state. + io_state: IoState, + /// Shared ready-work queues used for unordered morsel stealing. + shared_ready_work: MorselQueue, + /// Tasks waiting to be woken when shared capacity or ready work appears. + waiters: VecDeque, +} + +/// Shared outstanding-I/O accounting and fairness state for sibling streams. +#[derive(Debug)] +struct IoState { + /// Shared scheduling policy for sibling streams. + mode: SharedFileStreamMode, + /// Total number of planner I/O phases currently in flight. + outstanding_ios: usize, + /// Global cap on outstanding planner I/O phases across sibling streams. + max_outstanding_ios: usize, + /// Monotonic counter used to assign stable `FileStreamId`s. + next_stream_id: usize, + /// Per-stream scheduling state for each registered sibling stream. + streams: BTreeMap, +} + +#[derive(Debug, Default)] +struct StreamIOState { + outstanding_ios: usize, + waker: Option, +} + +impl SharedFileStreamState { + /// Create a new shared state object with the provided global I/O limit and + /// scheduling mode. + /// + /// The limit applies across all sibling `FileStream`s that share this + /// state, not per individual stream. + pub fn new(max_outstanding_ios: usize, mode: SharedFileStreamMode) -> Self { + Self { + inner: Arc::new(Mutex::new(SharedFileStreamStateInner { + io_state: IoState { + mode, + outstanding_ios: 0, + max_outstanding_ios, + next_stream_id: 0, + streams: BTreeMap::new(), + }, + shared_ready_work: MorselQueue::new(VecDeque::new()), + waiters: VecDeque::new(), + })), + } + } + + /// Register a sibling stream with this shared state and return its stable id. + pub fn register_stream(&self) -> FileStreamId { + let mut inner = self + .inner + .lock() + .expect("shared file stream state poisoned"); + + inner.io_state.register_stream() + } + + /// Returns the configured shared scheduling mode. + pub fn mode(&self) -> SharedFileStreamMode { + self.inner + .lock() + .expect("shared file stream state poisoned") + .io_state + .mode + } + + /// Returns the number of currently registered sibling streams. + pub fn registered_stream_count(&self) -> usize { + self.inner + .lock() + .expect("shared file stream state poisoned") + .io_state + .streams + .len() + } + + /// Returns the maximum number of outstanding planner I/O operations + /// allowed across all sibling streams. + pub fn max_outstanding_ios(&self) -> usize { + self.inner + .lock() + .expect("shared file stream state poisoned") + .io_state + .max_outstanding_ios + } + + /// Returns the number of currently outstanding planner I/O operations + /// across all sibling streams. + pub fn outstanding_ios(&self) -> usize { + self.inner + .lock() + .expect("shared file stream state poisoned") + .io_state + .outstanding_ios + } + + /// Unregister a stream that will never request another I/O permit. + /// + /// This removes the stream from future fairness calculations, allowing its + /// share of the I/O budget to be redistributed to sibling streams. + /// + /// The stream must not have any outstanding permits when it unregisters. + pub fn unregister_stream(&self, stream_id: FileStreamId) { + let waiters = { + let mut inner = self + .inner + .lock() + .expect("shared file stream state poisoned"); + if let Some(stream) = inner.io_state.streams.remove(&stream_id) { + assert_eq!( + stream.outstanding_ios, 0, + "stream must not unregister while it still holds I/O permits" + ); + } + Self::take_waiters_locked(&mut inner) + }; + + Self::wake_waiters(waiters); + } + + /// Register a task waker to be notified when shared capacity may have + /// become available. + /// + /// Callers should typically register a waker after failing to acquire an + /// I/O permit and before returning `Poll::Pending`. + pub fn register_waker(&self, stream_id: FileStreamId, waker: &Waker) { + let mut inner = self + .inner + .lock() + .expect("shared file stream state poisoned"); + + inner.waiters.push_back(waker.clone()); + if let Some(stream) = inner.io_state.streams.get_mut(&stream_id) { + stream.waker = Some(waker.clone()); + } + } + + /// Try to reserve one shared outstanding-I/O slot for `stream_id`. + /// + /// Returns `Some(permit)` if the stream is currently eligible to issue a + /// new I/O under the configured fairness mode, or `None` otherwise. + pub fn try_acquire_io_permit( + &self, + stream_id: FileStreamId, + ) -> Option { + let mut inner = self + .inner + .lock() + .expect("shared file stream state poisoned"); + + if !inner.io_state.can_issue_io(stream_id) { + return None; + } + + inner.io_state.acquire_io(stream_id); + drop(inner); + + Some(OutstandingIoPermit { + state: Some(self.clone()), + stream_id, + }) + } + + /// Publish one ready morsel into the shared queue. + pub fn push_ready_morsel(&self, morsel: Box) { + let waiters = { + let mut inner = self + .inner + .lock() + .expect("shared file stream state poisoned"); + inner.shared_ready_work.push_morsel(morsel); + Self::take_waiters_locked(&mut inner) + }; + Self::wake_waiters(waiters); + } + + /// Publish one ready planner into the shared queue. + pub fn push_ready_planner(&self, planner: Box) { + let waiters = { + let mut inner = self + .inner + .lock() + .expect("shared file stream state poisoned"); + inner.shared_ready_work.push_ready_planner(planner); + Self::take_waiters_locked(&mut inner) + }; + Self::wake_waiters(waiters); + } + + /// Try to steal one ready morsel from the shared queue. + pub fn pop_ready_morsel(&self) -> Option> { + self.inner + .lock() + .expect("shared file stream state poisoned") + .shared_ready_work + .pop_morsel() + } + + /// Try to steal one ready planner from the shared queue. + pub fn pop_ready_planner(&self) -> Option> { + self.inner + .lock() + .expect("shared file stream state poisoned") + .shared_ready_work + .pop_ready_planner() + } + + fn release_io_permit(&self, stream_id: FileStreamId) { + let waiters = { + let mut inner = self + .inner + .lock() + .expect("shared file stream state poisoned"); + inner.io_state.release_io(stream_id); + Self::take_waiters_locked(&mut inner) + }; + + Self::wake_waiters(waiters); + } + + fn take_waiters_locked(inner: &mut SharedFileStreamStateInner) -> Vec { + let mut waiters = inner.waiters.drain(..).collect::>(); + for stream in inner.io_state.streams.values_mut() { + if let Some(waker) = stream.waker.take() { + waiters.push(waker); + } + } + waiters + } + + fn wake_waiters(waiters: Vec) { + for waiter in waiters { + waiter.wake(); + } + } +} + +impl IoState { + fn register_stream(&mut self) -> FileStreamId { + let id = FileStreamId(self.next_stream_id); + self.next_stream_id += 1; + self.streams.insert(id, StreamIOState::default()); + id + } + + fn can_issue_io(&self, stream_id: FileStreamId) -> bool { + if self.outstanding_ios >= self.max_outstanding_ios { + return false; + } + + if !self.streams.contains_key(&stream_id) { + return false; + } + + match self.mode { + SharedFileStreamMode::Unordered => self.can_issue_unordered(stream_id), + SharedFileStreamMode::PreserveOrder => { + self.can_issue_preserve_order(stream_id) + } + } + } + + fn acquire_io(&mut self, stream_id: FileStreamId) { + self.outstanding_ios += 1; + self.streams + .get_mut(&stream_id) + .expect("unregistered file stream") + .outstanding_ios += 1; + } + + fn release_io(&mut self, stream_id: FileStreamId) { + self.outstanding_ios = self + .outstanding_ios + .checked_sub(1) + .expect("outstanding I/O count underflow"); + self.streams + .get_mut(&stream_id) + .expect("unregistered file stream") + .outstanding_ios = self + .streams + .get(&stream_id) + .expect("unregistered file stream") + .outstanding_ios + .checked_sub(1) + .expect("per-stream outstanding I/O count underflow"); + } + + fn can_issue_unordered(&self, stream_id: FileStreamId) -> bool { + let stream = self + .streams + .get(&stream_id) + .expect("unregistered file stream"); + + if stream.outstanding_ios == 0 { + return true; + } + + // Once a stream already has one outstanding I/O, it may only start a + // second if every other registered sibling stream has also reached at + // least one outstanding request. + self.streams.values().all(|state| state.outstanding_ios > 0) + } + + fn can_issue_preserve_order(&self, stream_id: FileStreamId) -> bool { + let share = self.fair_share_for(stream_id); + let stream = self + .streams + .get(&stream_id) + .expect("unregistered file stream"); + stream.outstanding_ios < share + } + + fn fair_share_for(&self, stream_id: FileStreamId) -> usize { + let active_streams = self.streams.keys().collect::>(); + + if active_streams.is_empty() { + return 0; + } + + let active_count = active_streams.len(); + let base_share = self.max_outstanding_ios / active_count; + let remainder = self.max_outstanding_ios % active_count; + + let position = active_streams + .iter() + .position(|id| **id == stream_id) + .expect("stream should be active"); + + base_share + usize::from(position < remainder) + } +} + +/// RAII guard representing one shared outstanding-I/O slot. +/// +/// Hold this permit for exactly as long as the corresponding planner I/O +/// future remains outstanding. Dropping the permit releases the slot back to +/// the shared state. +#[derive(Debug)] +pub struct OutstandingIoPermit { + state: Option, + stream_id: FileStreamId, +} + +impl Drop for OutstandingIoPermit { + fn drop(&mut self) { + if let Some(state) = self.state.take() { + state.release_io_permit(self.stream_id); + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Arc; + use std::sync::atomic::{AtomicUsize, Ordering}; + use std::task::{Wake, Waker}; + + #[test] + /// In unordered mode, each stream that currently wants I/O must get a + /// chance to issue its first request before another stream is allowed to + /// issue a second. Once every interested stream has one outstanding I/O, + /// any remaining permits may be consumed in any order. + fn unordered_every_stream_gets_one_before_extras() { + let state = SharedFileStreamState::new(3, SharedFileStreamMode::Unordered); + let stream1 = state.register_stream(); + let stream2 = state.register_stream(); + + let permit1 = state.try_acquire_io_permit(stream1).unwrap(); + assert!(state.try_acquire_io_permit(stream1).is_none()); + + let permit2 = state.try_acquire_io_permit(stream2).unwrap(); + let permit3 = state.try_acquire_io_permit(stream1).unwrap(); + assert_eq!(state.outstanding_ios(), 3); + assert!(state.try_acquire_io_permit(stream2).is_none()); + + drop(permit2); + drop(permit3); + drop(permit1); + } + + #[test] + /// In unordered mode, unregistering a stream removes it from the + /// "everyone gets one first" rule, so its reserved share may be + /// immediately reused by remaining active streams. + fn unordered_closed_stream_releases_capacity_to_others() { + let state = SharedFileStreamState::new(2, SharedFileStreamMode::Unordered); + let stream1 = state.register_stream(); + let stream2 = state.register_stream(); + + let _permit1 = state.try_acquire_io_permit(stream1).unwrap(); + state.unregister_stream(stream2); + + assert!(state.try_acquire_io_permit(stream1).is_some()); + } + + #[test] + /// In preserve-order mode, the outstanding-I/O budget is split fairly + /// across all active streams. With three streams and three permits total, + /// each stream is capped at one concurrent I/O. + fn preserve_order_splits_evenly() { + let state = SharedFileStreamState::new(3, SharedFileStreamMode::PreserveOrder); + let stream1 = state.register_stream(); + let stream2 = state.register_stream(); + let stream3 = state.register_stream(); + + let _permit1 = state.try_acquire_io_permit(stream1).unwrap(); + let _permit2 = state.try_acquire_io_permit(stream2).unwrap(); + let _permit3 = state.try_acquire_io_permit(stream3).unwrap(); + + assert!(state.try_acquire_io_permit(stream1).is_none()); + assert!(state.try_acquire_io_permit(stream2).is_none()); + assert!(state.try_acquire_io_permit(stream3).is_none()); + } + + #[test] + /// In preserve-order mode, once a stream has released its final permit and + /// unregisters itself, its fair share is redistributed to the remaining + /// active streams. + fn preserve_order_redistributes_closed_stream_share() { + let state = SharedFileStreamState::new(3, SharedFileStreamMode::PreserveOrder); + let stream1 = state.register_stream(); + let stream2 = state.register_stream(); + let stream3 = state.register_stream(); + + let _permit1 = state.try_acquire_io_permit(stream1).unwrap(); + let _permit2 = state.try_acquire_io_permit(stream2).unwrap(); + let permit3 = state.try_acquire_io_permit(stream3).unwrap(); + + drop(permit3); + state.unregister_stream(stream3); + + // Shares rebalance from [1,1,1] to [2,1] in registration order. + assert!(state.try_acquire_io_permit(stream1).is_some()); + assert!(state.try_acquire_io_permit(stream2).is_none()); + } + + #[test] + /// Releasing an outstanding permit should wake blocked sibling streams so + /// they can retry permit acquisition on a future poll. + fn releasing_permit_wakes_waiters() { + let state = SharedFileStreamState::new(1, SharedFileStreamMode::Unordered); + let stream = state.register_stream(); + let permit = state.try_acquire_io_permit(stream).unwrap(); + + let wake_counter = Arc::new(WakeCounter::default()); + let waker = Waker::from(Arc::clone(&wake_counter)); + state.register_waker(stream, &waker); + + drop(permit); + + assert_eq!(wake_counter.wake_count.load(Ordering::SeqCst), 2); + assert_eq!(state.outstanding_ios(), 0); + } + + #[derive(Default)] + struct WakeCounter { + wake_count: AtomicUsize, + } + + impl Wake for WakeCounter { + fn wake(self: Arc) { + self.wake_count.fetch_add(1, Ordering::SeqCst); + } + + fn wake_by_ref(self: &Arc) { + self.wake_count.fetch_add(1, Ordering::SeqCst); + } + } +} diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index bcc4627050d4..6bb172f86e38 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -38,6 +38,7 @@ pub mod file_scan_config; pub mod file_sink_config; pub mod file_stream; pub mod memory; +pub mod morsel; pub mod projection; pub mod schema_adapter; pub mod sink; @@ -126,7 +127,8 @@ pub struct PartitionedFile { /// [`wrap_partition_value_in_dict`]: crate::file_scan_config::wrap_partition_value_in_dict /// [`table_partition_cols`]: https://github.com/apache/datafusion/blob/main/datafusion/core/src/datasource/file_format/options.rs#L87 pub partition_values: Vec, - /// An optional file range for a more fine-grained parallel execution + /// An optional file range for this file. This is used to statically + /// schedule non-overlapping sections of a file to be read in parallel. pub range: Option, /// Optional statistics that describe the data in this file if known. /// diff --git a/datafusion/datasource/src/morsel/adapters.rs b/datafusion/datasource/src/morsel/adapters.rs new file mode 100644 index 000000000000..460ba314f326 --- /dev/null +++ b/datafusion/datasource/src/morsel/adapters.rs @@ -0,0 +1,147 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::PartitionedFile; +use crate::file_stream::FileOpener; +use crate::morsel::{Morsel, MorselPlan, MorselPlanner, Morselizer}; +use arrow::array::RecordBatch; +use datafusion_common::Result; +use futures::FutureExt; +use futures::stream::BoxStream; +use std::fmt::Debug; +use std::sync::{Arc, Mutex}; + +/// An adapter for `FileOpener` that allows it to be used as a `Morselizer` for +/// backwards compatibility. +/// +/// This is useful for file formats that do not support morselization, where we +/// can treat the entire file as a single morsel. +pub struct FileOpenerMorselizer { + file_opener: Arc, +} + +impl Debug for FileOpenerMorselizer { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("FileOpenerMorselizer") + .field("file_opener", &"...") + .finish() + } +} + +impl FileOpenerMorselizer { + pub fn new(file_opener: Arc) -> Self { + Self { file_opener } + } +} + +impl Morselizer for FileOpenerMorselizer { + fn morselize(&self, file: PartitionedFile) -> Result>> { + let opener = Arc::clone(&self.file_opener); + let planner = FileOpenFutureMorselPlanner::new(opener, file); + Ok(vec![Box::new(planner)]) + } +} + +/// Adapter for `FileOpenFuture` that allows it to be used as a `MorselPlanner` +/// for backwards compatibility. +struct FileOpenFutureMorselPlanner { + file_opener: Arc, + stream: Arc>>>>, + file: Mutex>, +} + +impl Debug for FileOpenFutureMorselPlanner { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("FileOpenFutureMorselPlanner") + .field("file_opener", &"...") + .field("stream", &"...") + .field("file", &self.file) + .finish() + } +} + +impl FileOpenFutureMorselPlanner { + pub fn new(file_opener: Arc, file: PartitionedFile) -> Self { + Self { + file_opener, + stream: Arc::new(Mutex::new(None)), + file: Mutex::new(Some(file)), + } + } +} + +impl MorselPlanner for FileOpenFutureMorselPlanner { + fn plan(&mut self) -> Result> { + let mut morsel_plan = MorselPlan::new(); + let mut made_progress = false; + + // Note that plan should **not** do IO work so setup a callback if needed + if let Some(file) = self.file.lock().unwrap().take() { + let file_opener = Arc::clone(&self.file_opener); + let output_stream = Arc::clone(&self.stream); + let load_future = async move { + let stream = file_opener + // open the file to get a stream + .open(file)? + // create the stream + .await?; + // store the stream for later retrieval + *(output_stream.lock().unwrap()) = Some(stream); + Ok(()) + }; + morsel_plan = morsel_plan.with_io_future(load_future.boxed()); + made_progress = true; + } + + // If the stream is ready, return it as a morsel + if let Some(stream) = self.stream.lock().unwrap().take() { + let morsel = FileStreamMorsel::new(stream); + morsel_plan = morsel_plan.with_morsels(vec![Box::new(morsel)]); + made_progress = true; + } + + Ok(made_progress.then_some(morsel_plan)) + } +} + +struct FileStreamMorsel { + stream: BoxStream<'static, Result>, +} + +impl Debug for FileStreamMorsel { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("FileStreamMorsel") + .field("stream", &"...") + .finish() + } +} + +impl FileStreamMorsel { + pub fn new(stream: BoxStream<'static, Result>) -> Self { + Self { stream } + } +} + +impl Morsel for FileStreamMorsel { + fn into_stream(self: Box) -> BoxStream<'static, Result> { + self.stream + } + + fn split(&mut self) -> Result>> { + Ok(vec![]) // no splitting supported + } +} diff --git a/datafusion/datasource/src/morsel/mod.rs b/datafusion/datasource/src/morsel/mod.rs new file mode 100644 index 000000000000..5a6569d5307c --- /dev/null +++ b/datafusion/datasource/src/morsel/mod.rs @@ -0,0 +1,181 @@ +// 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. + +//! Structures for Morsel Driven IO +//! +//! Morsel Driven IO is a technique for parallelizing the reading of large files +//! by dividing them into smaller "morsels" that can be processed independently. +//! It is inspired by the paper [Morsel-Driven Parallelism: A NUMA-Aware Query +//! Evaluation Framework for the Many-Core Age](https://db.in.tum.de/~leis/papers/morsels.pdf) + +mod adapters; +#[cfg(test)] +pub(crate) mod test_utils; + +use crate::PartitionedFile; +use arrow::array::RecordBatch; +use datafusion_common::error::Result; +use futures::future::BoxFuture; +use futures::stream::BoxStream; +use std::fmt::Debug; + +pub use adapters::FileOpenerMorselizer; + +/// A Morsel of work ready to resolve to a stream of [`RecordBatch`]es +/// +/// This represents a single morsel of work that is ready to be processed. It +/// has all data necessary (does not need any I/O) and is ready to be turned +/// into a stream of RecordBatches for processing by the execution engine. +pub trait Morsel: Send + Debug { + /// Consume this morsel and produce a stream of RecordBatches for processing. + /// + /// This should not do any IO work, such as reading from the file. + fn into_stream(self: Box) -> BoxStream<'static, Result>; + + /// If supported, split this morsel into smaller morsels. + /// + /// If not possible or not supported, return an empty Vector. + /// + /// This is used for dynamic load balancing of work where there are some + /// tasks that have nothing else scheduled. + fn split(&mut self) -> Result>>; +} + +/// A Morselizer takes a single PartitionedFile and breaks it down into smaller chunks +/// that can be planned and read in parallel by the execution engine. This is the entry point for +/// morsel driven IO. +pub trait Morselizer: Send + Sync + Debug { + /// Return MorselPlanners for this file. + /// + /// Each MorselPlanner is responsible for I/O and planning morsels for a + /// single scan of the file. Returning multiple MorselPlanners allows for + /// multiple concurrent scans of the same file. + /// + /// This may involve CPU work, such as parsing parquet metadata and + /// evaluating pruning predicates. It should NOT do any IO work, such as + /// reading from the file. If IO is required, it should return a future that + /// the caller can poll to drive the IO work to completion, and once the + /// future is complete, the caller can call `morselize` again to get the + /// next morsels. + fn morselize(&self, file: PartitionedFile) -> Result>>; +} + +/// A Morsel Planner is responsible for creating morsels for a given scan. +/// +/// The MorselPlanner is the unit of I/O -- there is only ever a single IO +/// outstanding for a specific MorselPlanner. DataFusion will potentially run +/// multiple MorselPlanners in parallel which corresponds to multiple parallel +/// I/O requests. +/// +/// It is not a Rust `Stream` so that it can explicitly separate CPU bound +/// work from IO work. +/// +/// The design is similar to `ParquetPushDecoder` -- when `plan` is called, it +/// should do CPU work to produce the next morsels or discover the next I/O +/// phase. +/// +/// Best practice is to spawn IO in a tokio Task in a separate IO runtime to +/// ensure that CPU work doesn't block/slowdown IO work, but this is not +/// strictly required by the API. +pub trait MorselPlanner: Send + Debug { + /// Attempt to plan morsels. This may involve CPU work, such as parsing + /// parquet metadata and evaluating pruning predicates. + /// + /// It should NOT do any IO work, such as reading from the file. If IO is + /// required, the returned [`MorselPlan`] should contain a future that the + /// caller polls to drive the IO work to completion. Once the future is + /// complete, the caller can call `plan` again to get the next morsels. + /// + /// Note this function is not async to make it clear explicitly that if IO + /// is required, it should be done in the returned `io_future`. + /// + /// Returns `None` if the MorselPlanner has no more work to do (is done). + /// + /// # Empty Morsel Plans + /// + /// It may return Some(..) with an empty MorselPlan, which means it is ready + /// for more CPU work and should be called again. + /// + /// # Output Ordering + /// + /// See the comments on [`MorselPlan`] for the logical output order + fn plan(&mut self) -> Result>; +} + +/// Return result of [`MorselPlanner::plan`] +/// +/// # Logical Ordering +/// For plans where the output order of rows is maintained, the output order of +/// a [`MorselPlanner`] is logically defined as follows: +/// 1. All morsels that are directly produced +/// 2. (recursively) All morsels produced by the returned `planners` +#[derive(Default)] +pub struct MorselPlan { + /// Any Morsels that are ready for processing. + morsels: Vec>, + /// Any newly-created planners that are ready for CPU work. + planners: Vec>, + /// A future that will drive any IO work to completion + /// + /// DataFusion will poll this future occasionally to drive the IO work to + /// completion. Once the future resolves, DataFusion will call `plan` again + /// to get the next morsels. Best practice is to run this in a task on a + /// separate IO runtime to ensure that CPU work is not blocked by IO work, + /// but this is not strictly required by the API. + io_future: Option>>, +} + +impl MorselPlan { + pub fn new() -> Self { + Default::default() + } + + pub fn with_morsels(mut self, morsels: Vec>) -> Self { + self.morsels = morsels; + self + } + + pub fn with_planners(mut self, planners: Vec>) -> Self { + self.planners = planners; + self + } + + pub fn with_io_future(mut self, io_future: BoxFuture<'static, Result<()>>) -> Self { + self.io_future = Some(io_future); + self + } + + pub fn take_io_future(&mut self) -> Option>> { + self.io_future.take() + } + + pub fn set_io_future(&mut self, io_future: BoxFuture<'static, Result<()>>) { + self.io_future = Some(io_future); + } + + pub fn take_morsels(&mut self) -> Vec> { + std::mem::take(&mut self.morsels) + } + + pub fn take_planners(&mut self) -> Vec> { + std::mem::take(&mut self.planners) + } + + pub fn has_io_future(&self) -> bool { + self.io_future.is_some() + } +} diff --git a/datafusion/datasource/src/morsel/test_utils.rs b/datafusion/datasource/src/morsel/test_utils.rs new file mode 100644 index 000000000000..a5a066d16dc5 --- /dev/null +++ b/datafusion/datasource/src/morsel/test_utils.rs @@ -0,0 +1,705 @@ +// 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. + +//! Test-only mocks for exercising `FileStream`'s morsel scheduler. +//! +//! These utilities let tests describe morsel-planning behavior directly, +//! without depending on a particular file format implementation. They are used +//! to verify: +//! - the sequence in which `FileStream` calls `morselize`, `plan`, and polls I/O +//! - the order in which morsels and child planners are consumed +//! - the eventual order of `RecordBatch` output produced by the scheduler + +use crate::PartitionedFile; +use crate::morsel::{Morsel, MorselPlan, MorselPlanner, Morselizer}; +use arrow::array::{Int32Array, RecordBatch}; +use arrow::datatypes::{DataType, Field, Schema}; +use datafusion_common::Result; +use futures::future::BoxFuture; +use futures::stream::BoxStream; +use futures::{Future, Stream}; +use std::collections::{HashMap, VecDeque}; +use std::fmt::{Debug, Display, Formatter}; +use std::pin::Pin; +use std::sync::{Arc, Mutex}; +use std::task::{Context, Poll}; + +/// Identifier for a mock planner in test traces. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct PlannerId(pub usize); + +/// Identifier for a mock morsel in test traces. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct MorselId(pub usize); + +/// Identifier for a mock I/O future in test traces. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct IoFutureId(pub usize); + +/// Identifier for a produced batch in test traces. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct BatchId(pub usize); + +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum MorselEvent { + /// `FileStream` asked the `Morselizer` to start work for a file. + MorselizeFile { path: String }, + /// A root planner was created from a test specification. + PlannerCreated { planner_id: PlannerId }, + /// `MorselPlanner::plan()` was invoked. + PlannerPlanCalled { planner_id: PlannerId }, + /// A planner returned one or more child planners. + PlannerProducedChild { + planner_id: PlannerId, + child_planner_id: PlannerId, + }, + /// A planner returned an I/O future. + IoFutureCreated { + planner_id: PlannerId, + io_future_id: IoFutureId, + }, + /// `FileStream` polled a planner's outstanding I/O future. + IoFuturePolled { + planner_id: PlannerId, + io_future_id: IoFutureId, + }, + /// A planner's I/O future completed successfully. + IoFutureResolved { + planner_id: PlannerId, + io_future_id: IoFutureId, + }, + /// A planner produced a morsel that is ready for CPU work. + MorselProduced { + planner_id: PlannerId, + morsel_id: MorselId, + }, + /// `FileStream` began reading a morsel. + MorselStreamStarted { morsel_id: MorselId }, + /// A morsel stream yielded one batch. + MorselStreamBatchProduced { + morsel_id: MorselId, + batch_id: BatchId, + }, + /// A morsel stream reached EOF. + MorselStreamFinished { morsel_id: MorselId }, +} + +/// Observer of `MorselEvent`s emitted by the test harness. +#[derive(Debug, Default, Clone)] +pub struct MorselObserver { + events: Arc>>, +} + +impl MorselObserver { + pub fn new() -> Self { + Self::default() + } + + /// Clear any previously buffered events + pub fn clear(&self) { + self.events.lock().unwrap().clear(); + } + + /// Push a new [`MorselEvent`] + pub fn push(&self, event: MorselEvent) { + self.events.lock().unwrap().push(event); + } + + /// Return a copy of the current list of [`MorselEvents`] + pub fn events(&self) -> Vec { + self.events.lock().unwrap().clone() + } + + /// Format the recorded events as a stable, human-readable snapshot. + /// + /// We prefer snapshotting the event trace in tests rather than asserting it + /// programmatically because `FileStream` scheduling behavior is easier to + /// review as a full ordered trace. When the scheduler changes, updating the + /// snapshot is typically simpler and more informative than rebuilding a + /// hand-authored sequence of enum constructors. + pub fn format_events(&self) -> String { + self.events() + .into_iter() + .map(|event| event.to_string()) + .collect::>() + .join("\n") + } + + /// Format only the high-level scheduler events. + /// + /// This is useful for tests where the exact `plan()` / `poll()` interleave + /// is not the main point of the assertion. Summary snapshots are easier to + /// review when validating broader behavior such as ordering or outstanding + /// I/O limits. + pub fn format_summary_events(&self) -> String { + self.events() + .into_iter() + .filter(MorselEvent::is_summary_event) + .map(|event| event.to_string()) + .collect::>() + .join("\n") + } +} + +impl MorselEvent { + /// Return true for the higher-level events that are most useful in compact + /// scheduler snapshots. + pub fn is_summary_event(&self) -> bool { + matches!( + self, + MorselEvent::MorselizeFile { .. } + | MorselEvent::PlannerCreated { .. } + | MorselEvent::PlannerProducedChild { .. } + | MorselEvent::IoFutureCreated { .. } + | MorselEvent::IoFutureResolved { .. } + | MorselEvent::MorselProduced { .. } + | MorselEvent::MorselStreamBatchProduced { .. } + ) + } +} + +/// Test [`Morselizer`] that maps file paths to a fixed set of planner specs. +/// +/// This lets tests describe file-level morselization behavior without any real +/// file metadata or object-store access. +#[derive(Debug, Clone)] +pub struct MockMorselizer { + observer: MorselObserver, + files: HashMap, +} + +impl MockMorselizer { + pub fn new() -> Self { + Self { + observer: MorselObserver::new(), + files: HashMap::new(), + } + } + + /// Return a reference to the observer + pub fn observer(&self) -> &MorselObserver { + &self.observer + } + + /// Add the description of how a file will be planned + pub fn with_file(mut self, path: impl Into, planner: MockPlanner) -> Self { + self.files.insert(path.into(), planner); + self + } +} + +impl Morselizer for MockMorselizer { + fn morselize(&self, file: PartitionedFile) -> Result>> { + let path = file.object_meta.location.to_string(); + self.observer + .push(MorselEvent::MorselizeFile { path: path.clone() }); + + Ok(self + .files + .get(&path) + .cloned() + .into_iter() + .map(|planner| { + self.observer.push(MorselEvent::PlannerCreated { + planner_id: planner.planner_id, + }); + Box::new(MockMorselPlanner::new(self.observer.clone(), planner)) + as Box + }) + .collect()) + } +} + +/// Steps for a MockPlanner +/// +/// Tests build these specs up front, and `MockMorselizer` turns them into real +/// planners when `FileStream` starts work on a file. +#[derive(Debug, Clone)] +pub struct MockPlanner { + planner_id: PlannerId, + steps: Vec, +} + +impl MockPlanner { + /// Create a fluent builder for a planner specification. + pub fn builder() -> MockPlannerBuilder { + MockPlannerBuilder::default() + } +} + +/// One scheduler-visible step in a mock planner's lifecycle. +/// +/// A single step can produce morsels, child planners, and an I/O future, which +/// makes it possible to model the generic `MorselPlan` API closely in tests. +#[derive(Debug, Clone)] +pub enum PlannerStep { + ReturnPlan { + morsels: Vec, + planners: Vec, + /// Optional identifier for the I/O future returned by this step. + /// + /// Tests use this to assert the ordering of multiple outstanding I/O + /// phases. It must be `Some` when `io_polls > 0`. + io_future_id: Option, + io_polls: usize, + }, + ReturnNone, +} + +/// Builder for [`MockPlanner`]. +/// +/// This keeps `FileStream` scheduler tests readable as they grow to include +/// child planners and multiple I/O phases. +#[derive(Debug, Default)] +pub struct MockPlannerBuilder { + planner_id: Option, + steps: Vec, +} + +impl MockPlannerBuilder { + pub fn with_id(mut self, planner_id: PlannerId) -> Self { + self.planner_id = Some(planner_id); + self + } + + pub fn return_morsel(self, morsel_id: MorselId, batch_id: i32) -> Self { + self.return_plan( + ReturnPlanBuilder::new() + .with_morsel(MockMorselSpec::single_batch(morsel_id, batch_id)), + ) + } + + pub fn return_plan(mut self, plan: ReturnPlanBuilder) -> Self { + self.steps.push(plan.build()); + self + } + + pub fn return_none(mut self) -> Self { + self.steps.push(PlannerStep::ReturnNone); + self + } + + pub fn build(self) -> MockPlanner { + MockPlanner { + planner_id: self + .planner_id + .expect("MockPlannerBuilder requires planner_id"), + steps: self.steps, + } + } +} + +/// Builder for `PlannerStep::ReturnPlan`. +#[derive(Debug, Default)] +pub struct ReturnPlanBuilder { + /// Morsels that should be returned immediately by this planner step. + morsels: Vec, + /// Child planners that should be returned immediately by this planner step. + planners: Vec, + /// Identifier for the mock I/O future returned by this step, if any. + io_future_id: Option, + /// Number of `Poll::Pending` results the mock I/O future should yield + /// before resolving successfully. + /// + /// This is a deterministic test-only knob. It does not model elapsed time + /// or bytes read; it only controls how many scheduler polls are required + /// before the mock I/O future becomes ready. + io_polls: usize, +} + +impl ReturnPlanBuilder { + /// Create an empty return-plan builder. + pub fn new() -> Self { + Self::default() + } + + /// Add a single-batch morsel that should be returned immediately by this + /// planner step. + pub fn return_morsel(self, morsel_id: MorselId, batch_id: i32) -> Self { + self.with_morsel(MockMorselSpec::single_batch(morsel_id, batch_id)) + } + + /// Add a morsel that should be returned immediately by this planner step. + pub fn with_morsel(mut self, morsel: MockMorselSpec) -> Self { + self.morsels.push(morsel); + self + } + + /// Add a child planner that should be returned immediately by this step. + pub fn with_planner(mut self, planner: MockPlanner) -> Self { + self.planners.push(planner); + self + } + + /// Return a mock I/O future from this step. + /// + /// `io_future_id` is recorded in the emitted `MorselEvent`s so tests can + /// distinguish multiple I/O phases from the same planner. + /// + /// `io_polls` controls how many times that future returns `Poll::Pending` + /// before it resolves with `Poll::Ready(Ok(()))`. + /// + /// For example, `with_io(id, 1)` means: + /// - first poll: `Poll::Pending` + /// - second poll: `Poll::Ready(Ok(()))` + /// + /// So the total number of polls observed in the trace is `io_polls + 1`. + pub fn with_io(mut self, io_future_id: IoFutureId, io_polls: usize) -> Self { + self.io_future_id = Some(io_future_id); + self.io_polls = io_polls; + self + } + + /// Build the corresponding [`PlannerStep::ReturnPlan`] + pub fn build(self) -> PlannerStep { + PlannerStep::ReturnPlan { + morsels: self.morsels, + planners: self.planners, + io_future_id: self.io_future_id, + io_polls: self.io_polls, + } + } +} + +/// Declarative description of a mock morsel and the batches it should yield. +/// +/// Each batch id is turned into a one-row `RecordBatch`, which makes output +/// order easy to assert in `FileStream` tests. +#[derive(Debug, Clone)] +pub struct MockMorselSpec { + morsel_id: MorselId, + batch_ids: Vec, +} + +impl MockMorselSpec { + pub fn single_batch(morsel_id: MorselId, batch_id: i32) -> Self { + Self { + morsel_id, + batch_ids: vec![batch_id], + } + } +} + +/// Concrete `MorselPlanner` used by `FileStream` tests. +/// +/// It consumes a queue of `PlannerStep`s so tests can deterministically control +/// when a planner emits morsels, yields child planners, blocks on I/O, or +/// finishes. +struct MockMorselPlanner { + observer: MorselObserver, + planner_id: PlannerId, + steps: VecDeque, +} + +impl MockMorselPlanner { + fn new(observer: MorselObserver, spec: MockPlanner) -> Self { + Self { + observer, + planner_id: spec.planner_id, + steps: spec.steps.into(), + } + } +} + +impl Debug for MockMorselPlanner { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("MockMorselPlanner") + .field("planner_id", &self.planner_id) + .finish() + } +} + +impl MorselPlanner for MockMorselPlanner { + fn plan(&mut self) -> Result> { + self.observer.push(MorselEvent::PlannerPlanCalled { + planner_id: self.planner_id, + }); + + let Some(step) = self.steps.pop_front() else { + return Ok(None); + }; + + match step { + PlannerStep::ReturnPlan { + morsels, + planners, + io_future_id, + io_polls, + } => { + let mut plan = MorselPlan::new(); + + if !morsels.is_empty() { + for morsel in &morsels { + self.observer.push(MorselEvent::MorselProduced { + planner_id: self.planner_id, + morsel_id: morsel.morsel_id, + }); + } + plan = plan.with_morsels( + morsels + .into_iter() + .map(|morsel| { + Box::new(MockMorsel::new(self.observer.clone(), morsel)) + as Box + }) + .collect(), + ); + } + + if !planners.is_empty() { + for planner in &planners { + self.observer.push(MorselEvent::PlannerProducedChild { + planner_id: self.planner_id, + child_planner_id: planner.planner_id, + }); + } + plan = plan.with_planners( + planners + .into_iter() + .map(|planner| { + Box::new(MockMorselPlanner::new( + self.observer.clone(), + planner, + )) + as Box + }) + .collect(), + ); + } + + if io_polls > 0 { + let io_future_id = io_future_id.expect( + "PlannerStep::ReturnPlan with io_polls > 0 must specify io_future_id", + ); + self.observer.push(MorselEvent::IoFutureCreated { + planner_id: self.planner_id, + io_future_id, + }); + plan = plan.with_io_future(Box::pin(MockIoFuture::new( + self.observer.clone(), + self.planner_id, + io_future_id, + io_polls, + )) + as BoxFuture<'static, Result<()>>); + } + + Ok(Some(plan)) + } + PlannerStep::ReturnNone => Ok(None), + } + } +} + +/// Concrete `Morsel` used by the test harness. +/// +/// It yields a deterministic sequence of one-row batches and records lifecycle +/// events so tests can correlate scheduler activity with produced output. +struct MockMorsel { + observer: MorselObserver, + morsel_id: MorselId, + batch_ids: Vec, +} + +impl MockMorsel { + fn new(observer: MorselObserver, spec: MockMorselSpec) -> Self { + Self { + observer, + morsel_id: spec.morsel_id, + batch_ids: spec.batch_ids, + } + } +} + +impl Debug for MockMorsel { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("MockMorsel") + .field("morsel_id", &self.morsel_id) + .finish() + } +} + +impl Morsel for MockMorsel { + fn into_stream(self: Box) -> BoxStream<'static, Result> { + self.observer.push(MorselEvent::MorselStreamStarted { + morsel_id: self.morsel_id, + }); + Box::pin(MockMorselStream::new( + self.observer.clone(), + self.morsel_id, + self.batch_ids, + )) + } + + fn split(&mut self) -> Result>> { + Ok(vec![]) + } +} + +/// Stream returned by `MockMorsel::into_stream`. +/// +/// This stream exists so tests can observe exactly when a morsel starts, +/// produces batches, and finishes. +struct MockMorselStream { + observer: MorselObserver, + morsel_id: MorselId, + batches: VecDeque, + finished: bool, +} + +impl MockMorselStream { + fn new(observer: MorselObserver, morsel_id: MorselId, batch_ids: Vec) -> Self { + Self { + observer, + morsel_id, + batches: batch_ids.into(), + finished: false, + } + } +} + +impl Stream for MockMorselStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + if let Some(batch_id) = self.batches.pop_front() { + self.observer.push(MorselEvent::MorselStreamBatchProduced { + morsel_id: self.morsel_id, + batch_id: BatchId(batch_id as usize), + }); + return Poll::Ready(Some(Ok(single_value_batch(batch_id)))); + } + + if !self.finished { + self.finished = true; + self.observer.push(MorselEvent::MorselStreamFinished { + morsel_id: self.morsel_id, + }); + } + + Poll::Ready(None) + } +} + +/// Deterministic future used to simulate planner I/O in tests. +/// +/// It resolves after a configured number of pending polls and self-wakes so +/// `FileStream` can make forward progress without timers or real async I/O. +struct MockIoFuture { + observer: MorselObserver, + planner_id: PlannerId, + io_future_id: IoFutureId, + pending_polls_remaining: usize, +} + +impl MockIoFuture { + fn new( + observer: MorselObserver, + planner_id: PlannerId, + io_future_id: IoFutureId, + pending_polls: usize, + ) -> Self { + Self { + observer, + planner_id, + io_future_id, + pending_polls_remaining: pending_polls, + } + } +} + +impl Future for MockIoFuture { + type Output = Result<()>; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.observer.push(MorselEvent::IoFuturePolled { + planner_id: self.planner_id, + io_future_id: self.io_future_id, + }); + if self.pending_polls_remaining > 0 { + self.pending_polls_remaining -= 1; + cx.waker().wake_by_ref(); + return Poll::Pending; + } + + self.observer.push(MorselEvent::IoFutureResolved { + planner_id: self.planner_id, + io_future_id: self.io_future_id, + }); + Poll::Ready(Ok(())) + } +} + +fn single_value_batch(value: i32) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, false)])); + RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(vec![value]))]).unwrap() +} + +impl Display for MorselEvent { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + MorselEvent::MorselizeFile { path } => write!(f, "morselize_file: {path}"), + MorselEvent::PlannerCreated { planner_id } => { + write!(f, "planner_created: {planner_id:?}") + } + MorselEvent::PlannerPlanCalled { planner_id } => { + write!(f, "planner_called: {planner_id:?}") + } + MorselEvent::PlannerProducedChild { + planner_id, + child_planner_id, + } => write!( + f, + "planner_produced_child: {planner_id:?} -> {child_planner_id:?}" + ), + MorselEvent::IoFutureCreated { + planner_id, + io_future_id, + } => write!(f, "io_future_created: {planner_id:?}, {io_future_id:?}"), + MorselEvent::IoFuturePolled { + planner_id, + io_future_id, + } => write!(f, "io_future_polled: {planner_id:?}, {io_future_id:?}"), + MorselEvent::IoFutureResolved { + planner_id, + io_future_id, + } => write!(f, "io_future_resolved: {planner_id:?}, {io_future_id:?}"), + MorselEvent::MorselProduced { + planner_id, + morsel_id, + } => write!(f, "morsel_produced: {planner_id:?}, {morsel_id:?}"), + MorselEvent::MorselStreamStarted { morsel_id } => { + write!(f, "morsel_stream_started: {morsel_id:?}") + } + MorselEvent::MorselStreamBatchProduced { + morsel_id, + batch_id, + } => { + write!( + f, + "morsel_stream_batch_produced: {morsel_id:?}, {batch_id:?}" + ) + } + MorselEvent::MorselStreamFinished { morsel_id } => { + write!(f, "morsel_stream_finished: {morsel_id:?}") + } + } + } +}