alamb commented on code in PR #21342: URL: https://github.com/apache/datafusion/pull/21342#discussion_r3067822075
########## datafusion/datasource/src/file_stream/scan_state.rs: ########## @@ -0,0 +1,261 @@ +// 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 std::collections::VecDeque; +use std::task::{Context, Poll}; + +use crate::PartitionedFile; +use crate::morsel::{Morsel, MorselPlanner, Morselizer}; +use arrow::record_batch::RecordBatch; +use datafusion_common::{DataFusionError, Result}; +use datafusion_physical_plan::metrics::ScopedTimerGuard; +use futures::future::BoxFuture; +use futures::stream::BoxStream; +use futures::{FutureExt as _, StreamExt as _}; + +use super::{FileStreamMetrics, OnError}; + +/// Planner-owned asynchronous I/O discovered while planning a file. +/// +/// Once `io_future` completes, `planner` becomes CPU-ready again and can be +/// pushed back onto the scan queue for further planning. +struct PendingOpen { + /// The planner to resume after the I/O completes. + planner: Box<dyn MorselPlanner>, + /// The outstanding I/O future for `planner`. + io_future: BoxFuture<'static, Result<()>>, +} + +/// All mutable state for the active `FileStreamState::Scan` lifecycle. +/// +/// This groups together ready planners, ready morsels, the active reader, +/// pending planner I/O, the remaining files and limit, and the metrics +/// associated with processing that work. +pub(super) struct ScanState { Review Comment: I have added a diagram - let me know if that helps or if there is something else I can do ########## datafusion/datasource/src/morsel/mocks.rs: ########## @@ -0,0 +1,556 @@ +// Licensed to the Apache Software Foundation (ASF) under one Review Comment: Basically it makes a mock morselizer that records its steps so that the control flow of FileStream can be tested / verified ########## datafusion/datasource-parquet/src/opener.rs: ########## @@ -1906,8 +1814,45 @@ mod test { encryption_factory: None, max_predicate_cache_size: self.max_predicate_cache_size, reverse_row_groups: self.reverse_row_groups, + } + } + } + + /// Test helper that drives a [`ParquetMorselizer`] to completion and returns + /// the first stream morsel it produces. + /// + /// This mirrors how `FileStream` consumes the morsel APIs: it repeatedly + /// plans CPU work, awaits any discovered I/O futures, and feeds the planner + /// back into the ready queue until a stream morsel is ready. + async fn open_file( Review Comment: this is needed so that all the tests in this file don't need to be rewritten to the morselizer API ########## datafusion/datasource-parquet/src/source.rs: ########## @@ -542,37 +554,34 @@ impl FileSource for ParquetSource { .as_ref() .map(|time_unit| parse_coerce_int96_string(time_unit.as_str()).unwrap()); - let opener = Arc::new(ParquetOpener { - morselizer: ParquetMorselizer { - partition_index: partition, - projection: self.projection.clone(), - batch_size: self - .batch_size - .expect("Batch size must set before creating ParquetOpener"), - limit: base_config.limit, - preserve_order: base_config.preserve_order, - predicate: self.predicate.clone(), - table_schema: self.table_schema.clone(), - metadata_size_hint: self.metadata_size_hint, - metrics: self.metrics().clone(), - parquet_file_reader_factory, - pushdown_filters: self.pushdown_filters(), - reorder_filters: self.reorder_filters(), - force_filter_selections: self.force_filter_selections(), - enable_page_index: self.enable_page_index(), - 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(), - max_predicate_cache_size: self.max_predicate_cache_size(), - reverse_row_groups: self.reverse_row_groups, - }, - }); - Ok(opener) + Ok(Box::new(ParquetMorselizer { Review Comment: this passes the new morselizer directly rather than wrapping it first ########## datafusion/datasource/src/file_stream/scan_state.rs: ########## @@ -0,0 +1,283 @@ +// 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 std::collections::VecDeque; +use std::task::{Context, Poll}; + +use crate::PartitionedFile; +use crate::morsel::{Morsel, MorselPlanner, Morselizer, PendingMorselPlanner}; +use arrow::record_batch::RecordBatch; +use datafusion_common::{DataFusionError, Result}; +use datafusion_physical_plan::metrics::ScopedTimerGuard; +use futures::stream::BoxStream; +use futures::{FutureExt as _, StreamExt as _}; + +use super::{FileStreamMetrics, OnError}; + +/// State [`FileStreamState::Scan`]. +/// +/// There is one `ScanState` per `FileStream`, and thus per output partition. +/// +/// It groups together the lifecycle of scanning that partition's files: +/// unopened files, CPU-ready planners, pending planner I/O, ready morsels, +/// the active reader, and the metrics associated with processing that work. +/// +/// # State Transitions +/// +/// ```text +/// file_iter +/// | +/// v +/// morselizer.plan_file(file) +/// | +/// v +/// ready_planners ---> plan() ---> ready_morsels ---> into_stream() ---> reader ---> RecordBatches +/// ^ | +/// | v +/// | pending_planner +/// | | +/// | v +/// +-------- poll until ready +/// ``` +/// +/// [`FileStreamState::Scan`]: super::FileStreamState::Scan +pub(super) struct ScanState { + /// Files that still need to be planned. + file_iter: VecDeque<PartitionedFile>, + /// Remaining row limit, if any. + remain: Option<usize>, + /// The morselizer used to plan files. + morselizer: Box<dyn Morselizer>, + /// Behavior if opening or scanning a file fails. + on_error: OnError, + /// CPU-ready planners for the current file. + ready_planners: VecDeque<Box<dyn MorselPlanner>>, + /// Ready morsels for the current file. + ready_morsels: VecDeque<Box<dyn Morsel>>, + /// The active reader, if any. + reader: Option<BoxStream<'static, Result<RecordBatch>>>, + /// The single planner currently blocked on I/O, if any. + pending_planner: Option<PendingMorselPlanner>, + /// Metrics for the active scan queues. + metrics: FileStreamMetrics, +} + +impl ScanState { + pub(super) fn new( + file_iter: impl Into<VecDeque<PartitionedFile>>, + remain: Option<usize>, + morselizer: Box<dyn Morselizer>, + on_error: OnError, + metrics: FileStreamMetrics, + ) -> Self { + let file_iter = file_iter.into(); + Self { + file_iter, + remain, + morselizer, + on_error, + ready_planners: Default::default(), + ready_morsels: Default::default(), + reader: None, + pending_planner: None, + metrics, + } + } + + /// Updates how scan errors are handled while the stream is still active. + pub(super) fn set_on_error(&mut self, on_error: OnError) { + self.on_error = on_error; + } + + /// Drives one iteration of the active scan state. + /// + /// Work is attempted in this order: + /// 1. resolve any pending planner I/O + /// 2. poll the active reader + /// 3. turn a ready morsel into the active reader + /// 4. run CPU planning on a ready planner + /// 5. morselize the next unopened file + /// + /// The return [`ScanAndReturn`] tells `poll_inner` how to update the + /// outer `FileStreamState`. + pub(super) fn poll_scan(&mut self, cx: &mut Context<'_>) -> ScanAndReturn { Review Comment: This is the main state machine loop that shows how the file stream transitions between states ########## datafusion/datasource-parquet/src/opener.rs: ########## @@ -2013,12 +1953,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<Item = Result<arrow::array::RecordBatch, DataFusionError>> - + Send, - >, - >, + mut stream: BoxStream<'static, Result<RecordBatch>>, Review Comment: just a simpler way to write this -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
