@@ -42,6 +42,7 @@ use datafusion_common::instant::Instant;
4242use futures:: future:: BoxFuture ;
4343use futures:: stream:: BoxStream ;
4444use futures:: { FutureExt as _, Stream , StreamExt as _, ready} ;
45+ use crate :: morsel:: Morselizer ;
4546
4647/// A stream that iterates record batch by record batch, file over file.
4748pub struct FileStream {
@@ -50,8 +51,10 @@ pub struct FileStream {
5051 /// The stream schema (file schema including partition columns and after
5152 /// projection).
5253 projected_schema : SchemaRef ,
53- /// The remaining number of records to parse, None if no limit
54+ /// The remaining number of records to parse until limit is reached , None if no limit
5455 remain : Option < usize > ,
56+ /// A type specific [`Morselizer`] that examines the input files and produces a stream of `Morsels`
57+ morselizer : Option < Box < dyn Morselizer > > ,
5558 /// A dynamic [`FileOpener`]. Calling `open()` returns a [`FileOpenFuture`],
5659 /// which can be resolved to a stream of `RecordBatch`.
5760 file_opener : Arc < dyn FileOpener > ,
@@ -81,6 +84,7 @@ impl FileStream {
8184 file_iter : file_group. into_inner ( ) . into_iter ( ) . collect ( ) ,
8285 projected_schema,
8386 remain : config. limit ,
87+ morselizer : None ,
8488 file_opener,
8589 state : FileStreamState :: Idle ,
8690 file_stream_metrics : FileStreamMetrics :: new ( metrics, partition) ,
@@ -292,6 +296,9 @@ impl RecordBatchStream for FileStream {
292296}
293297
294298/// A fallible future that resolves to a stream of [`RecordBatch`]
299+ ///
300+ /// This is typically an `async` function that opens the file, and returns a
301+ /// stream that reads the file and produces `RecordBatch`es.
295302pub type FileOpenFuture =
296303 BoxFuture < ' static , Result < BoxStream < ' static , Result < RecordBatch > > > > ;
297304
@@ -308,10 +315,15 @@ pub enum OnError {
308315/// Generic API for opening a file using an [`ObjectStore`] and resolving to a
309316/// stream of [`RecordBatch`]
310317///
318+ /// TODO: rename this to MorselOpener
319+ /// Also, add documentation here describing IO expectations
320+ ///
311321/// [`ObjectStore`]: object_store::ObjectStore
312322pub trait FileOpener : Unpin + Send + Sync {
313323 /// Asynchronously open the specified file and return a stream
314324 /// of [`RecordBatch`]
325+ ///
326+ /// TODO: describe prefetching behavior here, and expectations around IO
315327 fn open ( & self , partitioned_file : PartitionedFile ) -> Result < FileOpenFuture > ;
316328}
317329
@@ -330,7 +342,7 @@ pub enum FileStreamState {
330342 /// for a given file
331343 Open {
332344 /// A [`FileOpenFuture`] returned by [`FileOpener::open`]
333- future : FileOpenFuture ,
345+ future : FileOpenFuture ,
334346 } ,
335347 /// Scanning the [`BoxStream`] returned by the completion of a [`FileOpenFuture`]
336348 /// returned by [`FileOpener::open`]
0 commit comments