Skip to content

Commit 755a0d2

Browse files
authored
feat: Add byte hint for fetching parquet metadata (#1108)
## What changes are included in this PR? Added hints for the reader when fetching parquet metadata.
1 parent 5e03d9f commit 755a0d2

File tree

1 file changed

+35
-11
lines changed

1 file changed

+35
-11
lines changed

crates/iceberg/src/arrow/reader.rs

+35-11
Original file line numberDiff line numberDiff line change
@@ -1120,24 +1120,45 @@ impl<'a> BoundPredicateVisitor for PredicateConverter<'a> {
11201120
}
11211121

11221122
/// ArrowFileReader is a wrapper around a FileRead that impls parquets AsyncFileReader.
1123-
///
1124-
/// # TODO
1125-
///
1126-
/// [ParquetObjectReader](https://docs.rs/parquet/latest/src/parquet/arrow/async_reader/store.rs.html#64)
1127-
/// contains the following hints to speed up metadata loading, we can consider adding them to this struct:
1128-
///
1129-
/// - `metadata_size_hint`: Provide a hint as to the size of the parquet file's footer.
1130-
/// - `preload_column_index`: Load the Column Index as part of [`Self::get_metadata`].
1131-
/// - `preload_offset_index`: Load the Offset Index as part of [`Self::get_metadata`].
11321123
pub struct ArrowFileReader<R: FileRead> {
11331124
meta: FileMetadata,
1125+
preload_column_index: bool,
1126+
preload_offset_index: bool,
1127+
metadata_size_hint: Option<usize>,
11341128
r: R,
11351129
}
11361130

11371131
impl<R: FileRead> ArrowFileReader<R> {
11381132
/// Create a new ArrowFileReader
11391133
pub fn new(meta: FileMetadata, r: R) -> Self {
1140-
Self { meta, r }
1134+
Self {
1135+
meta,
1136+
preload_column_index: false,
1137+
preload_offset_index: false,
1138+
metadata_size_hint: None,
1139+
r,
1140+
}
1141+
}
1142+
1143+
/// Enable or disable preloading of the column index
1144+
pub fn with_preload_column_index(mut self, preload: bool) -> Self {
1145+
self.preload_column_index = preload;
1146+
self
1147+
}
1148+
1149+
/// Enable or disable preloading of the offset index
1150+
pub fn with_preload_offset_index(mut self, preload: bool) -> Self {
1151+
self.preload_offset_index = preload;
1152+
self
1153+
}
1154+
1155+
/// Provide a hint as to the number of bytes to prefetch for parsing the Parquet metadata
1156+
///
1157+
/// This hint can help reduce the number of fetch requests. For more details see the
1158+
/// [ParquetMetaDataReader documentation](https://docs.rs/parquet/latest/parquet/file/metadata/struct.ParquetMetaDataReader.html#method.with_prefetch_hint).
1159+
pub fn with_metadata_size_hint(mut self, hint: usize) -> Self {
1160+
self.metadata_size_hint = Some(hint);
1161+
self
11411162
}
11421163
}
11431164

@@ -1152,7 +1173,10 @@ impl<R: FileRead> AsyncFileReader for ArrowFileReader<R> {
11521173

11531174
fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result<Arc<ParquetMetaData>>> {
11541175
async move {
1155-
let reader = ParquetMetaDataReader::new();
1176+
let reader = ParquetMetaDataReader::new()
1177+
.with_prefetch_hint(self.metadata_size_hint)
1178+
.with_column_indexes(self.preload_column_index)
1179+
.with_offset_indexes(self.preload_offset_index);
11561180
let size = self.meta.size as usize;
11571181
let meta = reader.load_and_finish(self, size).await?;
11581182

0 commit comments

Comments
 (0)