Skip to content

Commit

Permalink
fix(arrow): Use new ParquetMetaDataReader instead (apache#661)
Browse files Browse the repository at this point in the history
Signed-off-by: Xuanwo <[email protected]>
  • Loading branch information
Xuanwo authored and shaeqahmed committed Dec 9, 2024
1 parent d6ed3c5 commit 433a7ed
Showing 1 changed file with 11 additions and 9 deletions.
20 changes: 11 additions & 9 deletions crates/iceberg/src/arrow/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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};
Expand Down Expand Up @@ -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()
}
}

Expand Down

0 comments on commit 433a7ed

Please sign in to comment.