alamb commented on code in PR #21342:
URL: https://github.com/apache/datafusion/pull/21342#discussion_r3059459942
##########
datafusion/datasource/src/file_stream/mod.rs:
##########
@@ -89,105 +79,31 @@ impl FileStream {
/// 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;
+ if let FileStreamState::Scan { scan_state } = &mut self.state {
+ scan_state.set_on_error(on_error);
+ }
self
Review Comment:
Done. This is a good call
##########
datafusion/datasource/src/file_stream/mod.rs:
##########
@@ -38,33 +38,24 @@ use datafusion_physical_plan::metrics::{BaselineMetrics,
ExecutionPlanMetricsSet
use arrow::record_batch::RecordBatch;
+use futures::Stream;
use futures::future::BoxFuture;
use futures::stream::BoxStream;
-use futures::{FutureExt as _, Stream, StreamExt as _, ready};
+
+use self::scan_state::{ScanAndReturn, ScanState};
pub use builder::FileStreamBuilder;
pub use metrics::{FileStreamMetrics, StartableTime};
/// A stream that iterates record batch by record batch, file over file.
pub struct FileStream {
- /// An iterator over input files.
Review Comment:
The state machine responsible for opening files and interacting with the
Morsels is now in Scan State
##########
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 {
+ /// Files that still need to be planned.
+ file_iter: VecDeque<PartitionedFile>,
+ /// Remaining record limit, if any.
+ remain: Option<usize>,
+ /// The file-format-specific morselizer used to plan files.
+ morselizer: Box<dyn Morselizer>,
+ /// Describes the 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>>>,
Review Comment:
> Is there one ScanState across all partitions or one per partition? I'm
guessing the latter: file_iter: VecDeque<PartitionedFile> is the files for this
partition
yes, it is one ScanState per partition
> we pump all of the files into one output stream of RecordBatch (reader).
But we can have multiple planners / morsels ready and merge those all into a
single stream of RecordBatch on the way out.
Yes this is right
--
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]