alamb commented on code in PR #21342:
URL: https://github.com/apache/datafusion/pull/21342#discussion_r3039573319


##########
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:
   One 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.
   
   My initial proposal (following @Dandandan 's original design" is that when 
possible the files are put into a shared queue so that when a FileStream is 
ready it gets the next file
   
   I think once we get that structure in place, we can contemplate more 
sophisticated designs (like one filestream preparing a parquet file, and then 
divying up the record batches between other cores)



##########
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>>,

Review Comment:
   > My understanding of the state machine is File ->  MorselPlanner (via 
Morselizer, an IO operation) and then MorselPlanner -> Morsel (a CPU operation) 
and finally Morsel -> RecordBatch(es) (IO). Is that right?
   
   That is basically right, except the last step `Morsel --> RecordBatch(es)` 
should not have IO (though you are right that it does now)
   
   > Operations like Morsel -> RecordBatch are still a mix of IO/CPU 
(especially with filter pushdown on).
   
   My idea is that we change that so that the RecordBatches don't actually flow 
until we have all data buffered and ready to decode
   
   This is possible to do with the arrow-rs parquet reader when:
   1. Each MorselPlanner is for a single RowGroup
   2. We don't produce the morsel stream until we start getting batches (there 
is some version of that in https://github.com/apache/datafusion/pull/20820)



##########
datafusion/datasource/src/file_stream/builder.rs:
##########
@@ -51,8 +54,18 @@ impl<'a> FileStreamBuilder<'a> {
     }
 
     /// Configure the [`FileOpener`] used to open files.
+    ///
+    /// This will overwrite any setting from [`Self::with_morselizer`]
     pub fn with_file_opener(mut self, file_opener: Arc<dyn FileOpener>) -> 
Self {

Review Comment:
   This method is the way we could keep using FileOpener (as it is simpler)
   
   I am not sure how we could still allow using FileOpener but not keep this 
method



##########
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>>>,
+    /// Planner-owned asynchronous I/O that must complete before more planner
+    /// CPU work can be scheduled.
+    pending_open: Option<PendingOpen>,
+    /// 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_open: 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, consuming ready morsels,
+    /// planners, pending planner I/O, or unopened files from `self`.
+    ///
+    /// The return [`ScanAndReturn`] tells `poll_inner` how to update the
+    /// outer `FileStreamState`.
+    pub(super) fn poll_scan(&mut self, cx: &mut Context<'_>) -> ScanAndReturn {
+        let _processing_timer: ScopedTimerGuard<'_> =
+            self.metrics.time_processing.timer();
+        if let Some(pending_open) = self.pending_open.take() {
+            let PendingOpen {
+                planner,
+                mut io_future,
+            } = pending_open;
+            match io_future.poll_unpin(cx) {
+                Poll::Pending => {
+                    self.pending_open = Some(PendingOpen { planner, io_future 
});
+                    return ScanAndReturn::Return(Poll::Pending);
+                }
+                Poll::Ready(Ok(())) => {
+                    self.ready_planners.push_back(planner);
+                    return ScanAndReturn::Continue;
+                }
+                Poll::Ready(Err(err)) => {
+                    self.metrics.file_open_errors.add(1);
+                    self.metrics.time_opening.stop();
+                    return match self.on_error {
+                        OnError::Skip => {
+                            self.metrics.files_processed.add(1);
+                            ScanAndReturn::Continue
+                        }
+                        OnError::Fail => ScanAndReturn::Error(err),
+                    };
+                }
+            }
+        }
+
+        if let Some(reader) = self.reader.as_mut() {
+            match reader.poll_next_unpin(cx) {
+                Poll::Pending => return ScanAndReturn::Return(Poll::Pending),
+                Poll::Ready(Some(Ok(batch))) => {
+                    self.metrics.time_scanning_until_data.stop();
+                    self.metrics.time_scanning_total.stop();
+                    let (batch, finished) = match &mut self.remain {
+                        Some(remain) => {
+                            if *remain > batch.num_rows() {
+                                *remain -= batch.num_rows();
+                                self.metrics.time_scanning_total.start();
+                                (batch, false)
+                            } else {
+                                let batch = batch.slice(0, *remain);
+                                let done = 1 + self.file_iter.len();
+                                self.metrics.files_processed.add(done);
+                                *remain = 0;
+                                (batch, true)
+                            }
+                        }
+                        None => {
+                            self.metrics.time_scanning_total.start();
+                            (batch, false)
+                        }
+                    };
+                    return if finished {
+                        ScanAndReturn::Done(Some(Ok(batch)))
+                    } else {
+                        ScanAndReturn::Return(Poll::Ready(Some(Ok(batch))))
+                    };
+                }
+                Poll::Ready(Some(Err(err))) => {
+                    self.reader = None;
+                    self.metrics.file_scan_errors.add(1);
+                    self.metrics.time_scanning_until_data.stop();
+                    self.metrics.time_scanning_total.stop();
+                    return match self.on_error {
+                        OnError::Skip => {
+                            self.metrics.files_processed.add(1);
+                            ScanAndReturn::Continue
+                        }
+                        OnError::Fail => ScanAndReturn::Error(err),
+                    };
+                }
+                Poll::Ready(None) => {
+                    self.reader = None;
+                    self.metrics.files_processed.add(1);
+                    self.metrics.time_scanning_until_data.stop();
+                    self.metrics.time_scanning_total.stop();
+                    return ScanAndReturn::Continue;
+                }
+            }
+        }
+
+        if let Some(morsel) = self.ready_morsels.pop_front() {
+            self.metrics.files_opened.add(1);

Review Comment:
   You are right -- the files opened should be for files "morselized"



-- 
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]

Reply via email to