yjshen commented on a change in pull request #1905:
URL: https://github.com/apache/arrow-datafusion/pull/1905#discussion_r820119627
##########
File path: datafusion/src/datasource/object_store/local.rs
##########
@@ -82,23 +112,12 @@ impl ObjectReader for LocalFileReader {
)
}
- fn sync_chunk_reader(
- &self,
- start: u64,
- length: usize,
- ) -> Result<Box<dyn Read + Send + Sync>> {
- // A new file descriptor is opened for each chunk reader.
- // This okay because chunks are usually fairly large.
- let mut file = File::open(&self.file.path)?;
Review comment:
Yes, the original problem is about too much open, and we do solve it in
our HDFS object store implementations similar to your suggestion above.
However, as I think more of the ObjectReader API and its use, I think I've
brought in one extra layer of abstraction, the "chunk" reader layer, into
ObjectReader without benefits. I prefer the `chunk reader` is only Parquet
related, and object readers should only care about like seeks and reads.
Therefore the current PR stops creating new readers from `ObjectReader`, but
directly reads in the `ObjectReader` itself. And If we are seeking an ability
to fetch multi parts from a parquet file simultaneously, we can utilize the
struct `PartitionedFile`.
```
/// A single file that should be read, along with its schema, statistics
/// and partition column values that need to be appended to each row.
pub struct PartitionedFile {
/// Path for the file (e.g. URL, filesystem path, etc)
pub file_meta: FileMeta,
/// Values of partition columns to be appended to each row
pub partition_values: Vec<ScalarValue>,
// We may include row group range here for a more fine-grained parallel
execution
}
```
We could have a `max_bytes_per_partition` configuration during query
planing, combine multiple parquet files into one partition (like we do now), or
split a large parquet file into many ranges, and have each partition handle
only several ranges. with the help of
https://github.com/apache/arrow-rs/issues/158.
And the last several comments are about how to avoid `Mutex` from the
`ObjectReaderWrapper` . Since we read parquet file sequentially in a partition,
Mutex may incur unnecessary overhead. Just have to write like this way to
achieve interior mutability since ChunkReader API in parquet-rs needs so:
```rust
pub trait ChunkReader: Length + Send + Sync {
type T: Read + Send;
/// get a serialy readeable slice of the current reader
/// This should fail if the slice exceeds the current bounds
fn get_read(&self, start: u64, length: usize) -> Result<Self::T>; //
&self as well as send imposes the need for interior mutability
}
```
--
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]