alamb commented on code in PR #7334:
URL: https://github.com/apache/arrow-rs/pull/7334#discussion_r2018907063
##########
parquet/src/file/metadata/reader.rs:
##########
@@ -638,6 +676,53 @@ impl ParquetMetaDataReader {
}
}
+ #[cfg(all(feature = "async", feature = "arrow"))]
+ async fn load_metadata_via_suffix<F: MetadataSuffixFetch>(
+ &self,
+ fetch: &mut F,
+ prefetch: usize,
+ ) -> Result<(ParquetMetaData, Option<(usize, Bytes)>)> {
+ let suffix = fetch.fetch_suffix(prefetch.max(FOOTER_SIZE)).await?;
Review Comment:
It looks like `prefetch` is always the value of self.get_prefetch_size()
which already has at least FOOTER_SIZE bytes
Thus I think this line is unecessary.
I think it would be clearer if you removed the `prefetch` parameter, and
instead directly called `self.prefetch_size()` in the method
##########
parquet/src/file/metadata/reader.rs:
##########
@@ -638,6 +676,53 @@ impl ParquetMetaDataReader {
}
}
+ #[cfg(all(feature = "async", feature = "arrow"))]
+ async fn load_metadata_via_suffix<F: MetadataSuffixFetch>(
+ &self,
+ fetch: &mut F,
+ prefetch: usize,
+ ) -> Result<(ParquetMetaData, Option<(usize, Bytes)>)> {
+ let suffix = fetch.fetch_suffix(prefetch.max(FOOTER_SIZE)).await?;
+ let suffix_len = suffix.len();
Review Comment:
The code from here on down seems like it is copy/pasted from `load_metadata`
-- could you please try and avoid duplication if possible by refactoring the
common code into a shared method?
##########
parquet/src/arrow/async_reader/store.rs:
##########
@@ -45,29 +46,35 @@ use tokio::runtime::Handle;
/// println!("Found Blob with {}B at {}", meta.size, meta.location);
///
/// // Show Parquet metadata
-/// let reader = ParquetObjectReader::new(storage_container, meta);
+/// let reader = ParquetObjectReader::new(storage_container, meta.location,
Some(meta.size));
/// let builder = ParquetRecordBatchStreamBuilder::new(reader).await.unwrap();
/// print_parquet_metadata(&mut stdout(), builder.metadata());
/// # }
/// ```
#[derive(Clone, Debug)]
pub struct ParquetObjectReader {
store: Arc<dyn ObjectStore>,
- meta: ObjectMeta,
+ path: Path,
+ file_size: Option<usize>,
metadata_size_hint: Option<usize>,
preload_column_index: bool,
preload_offset_index: bool,
runtime: Option<Handle>,
}
impl ParquetObjectReader {
- /// Creates a new [`ParquetObjectReader`] for the provided [`ObjectStore`]
and [`ObjectMeta`]
+ /// Creates a new [`ParquetObjectReader`] for the provided [`ObjectStore`]
and [`Path`].
///
- /// [`ObjectMeta`] can be obtained using [`ObjectStore::list`] or
[`ObjectStore::head`]
- pub fn new(store: Arc<dyn ObjectStore>, meta: ObjectMeta) -> Self {
+ /// The file size is optional, and if provided, it will ensure that only
bounded range requests
+ /// are used. If file size is not provided, the reader will use suffix
range requests to fetch
+ /// the metadata.
+ ///
+ /// The file size can be obtained using [`ObjectStore::list`] or
[`ObjectStore::head`].
+ pub fn new(store: Arc<dyn ObjectStore>, path: Path, file_size:
Option<usize>) -> Self {
Review Comment:
What do you think about not including the optional`file_size` in the
constructor, and instead add a `with_file_size` method? I think that might be
more consistent with the `metadata_size_hint` and the other options on this
struct.
```suggestion
pub fn new(store: Arc<dyn ObjectStore>, path: Path) -> Self {
```
And then you would make one of these things like
```rust
let reader = ParquetObjectReader::new(store, meta.location)
.with_size(Some(meta.size))
.with_runtime(rt.handle().clone());
```
I think as long as we left documentation that clearly stated that providing
the size up front was an important optimization to avoid extra calls, this
would be ok
--
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]