This is an automated email from the ASF dual-hosted git repository.
liurenjie1024 pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg-rust.git
The following commit(s) were added to refs/heads/main by this push:
new 8cf581b2 fix(arrow): Use new ParquetMetaDataReader instead (#661)
8cf581b2 is described below
commit 8cf581b2edc574fbd591cb9090dff55efb0f8eb7
Author: Xuanwo <[email protected]>
AuthorDate: Thu Oct 10 10:11:43 2024 +0800
fix(arrow): Use new ParquetMetaDataReader instead (#661)
Signed-off-by: Xuanwo <[email protected]>
---
crates/iceberg/src/arrow/reader.rs | 20 +++++++++++---------
1 file changed, 11 insertions(+), 9 deletions(-)
diff --git a/crates/iceberg/src/arrow/reader.rs
b/crates/iceberg/src/arrow/reader.rs
index ed422be9..f6680e31 100644
--- a/crates/iceberg/src/arrow/reader.rs
+++ b/crates/iceberg/src/arrow/reader.rs
@@ -31,11 +31,11 @@ use bytes::Bytes;
use fnv::FnvHashSet;
use futures::channel::mpsc::{channel, Sender};
use futures::future::BoxFuture;
-use futures::{try_join, SinkExt, StreamExt, TryFutureExt, TryStreamExt};
+use futures::{try_join, FutureExt, SinkExt, StreamExt, TryFutureExt,
TryStreamExt};
use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions,
RowFilter, RowSelection};
-use parquet::arrow::async_reader::{AsyncFileReader, MetadataLoader};
+use parquet::arrow::async_reader::AsyncFileReader;
use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask,
PARQUET_FIELD_ID_META_KEY};
-use parquet::file::metadata::ParquetMetaData;
+use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader};
use parquet::schema::types::{SchemaDescriptor, Type as ParquetType};
use crate::arrow::{arrow_schema_to_schema, get_arrow_datum};
@@ -1078,12 +1078,14 @@ impl<R: FileRead> AsyncFileReader for
ArrowFileReader<R> {
}
fn get_metadata(&mut self) -> BoxFuture<'_,
parquet::errors::Result<Arc<ParquetMetaData>>> {
- Box::pin(async move {
- let file_size = self.meta.size;
- let mut loader = MetadataLoader::load(self, file_size as usize,
None).await?;
- loader.load_page_index(false, false).await?;
- Ok(Arc::new(loader.finish()))
- })
+ async move {
+ let reader = ParquetMetaDataReader::new();
+ let size = self.meta.size as usize;
+ let meta = reader.load_and_finish(self, size).await?;
+
+ Ok(Arc::new(meta))
+ }
+ .boxed()
}
}