alamb commented on code in PR #20820: URL: https://github.com/apache/datafusion/pull/20820#discussion_r2908721774
########## datafusion/datasource/src/morsel.rs: ########## @@ -0,0 +1,57 @@ +// 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) + +use std::any::Any; +use std::sync::Arc; +use futures::future::BoxFuture; +use datafusion_common::error::Result; +use futures::stream::BoxStream; +use crate::PartitionedFile; + +/// A Morsel represents a portion of a file that can be processed independently. +#[derive(Debug)] +pub struct Morsel { + /// The original [`PartitionedFile`] that this morsel belongs to + file: Arc<PartitionedFile>, + /// File format specific information that describes the morsel, such as byte range, row group, etc. + info: Box<dyn Any>, +} + +/// A fallible future that resolves to a stream of [`Morsel`] +/// +/// This is typically an `async` function that opens the file, and returns a +/// stream of Morsels that can be processed independently. The stream may yield +/// an error if the file cannot be opened or read. +pub type MorselOpenFuture = BoxFuture<'static, Result<BoxStream<'static, Morsel>>>; Review Comment: > smaller ones if we're running out of them (i.e. no more files to morselize and number of morsels is < threads). Is this the same thing as just using smaller morsels to begin with (rather than reading entire row groups, for example, pair down the plan to just read 2MB chunks "morsels")? I realize we would have to work out of to avoid blowing out the IOs in this case -- 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]
