This is an automated email from the ASF dual-hosted git repository.
mneumann pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow-rs.git
The following commit(s) were added to refs/heads/master by this push:
new 341ec357e7 stop panic in `MetadataLoader` on invalid data (#6367)
341ec357e7 is described below
commit 341ec357e74e897f50250930b44f453bce54a19a
Author: Samuel Colvin <[email protected]>
AuthorDate: Mon Sep 16 15:52:06 2024 +0100
stop panic in `MetadataLoader` on invalid data (#6367)
* stop panic in MetadataLoader on invalid data
* better check for invalid prefect
* limit hint instead of erroring
* import FOOTER_SIZE
---
parquet/src/arrow/async_reader/metadata.rs | 31 ++++++++++++++++++++----------
1 file changed, 21 insertions(+), 10 deletions(-)
diff --git a/parquet/src/arrow/async_reader/metadata.rs
b/parquet/src/arrow/async_reader/metadata.rs
index 9224ea3f68..b4bf77f260 100644
--- a/parquet/src/arrow/async_reader/metadata.rs
+++ b/parquet/src/arrow/async_reader/metadata.rs
@@ -21,6 +21,7 @@ use crate::file::footer::{decode_footer, decode_metadata};
use crate::file::metadata::ParquetMetaData;
use crate::file::page_index::index::Index;
use crate::file::page_index::index_reader::{acc_range, decode_column_index,
decode_offset_index};
+use crate::file::FOOTER_SIZE;
use bytes::Bytes;
use futures::future::BoxFuture;
use futures::FutureExt;
@@ -53,7 +54,7 @@ impl<F: MetadataFetch> MetadataLoader<F> {
///
/// See [`fetch_parquet_metadata`] for the meaning of the individual
parameters
pub async fn load(mut fetch: F, file_size: usize, prefetch: Option<usize>)
-> Result<Self> {
- if file_size < 8 {
+ if file_size < FOOTER_SIZE {
return Err(ParquetError::EOF(format!(
"file size of {file_size} is less than footer"
)));
@@ -62,20 +63,22 @@ impl<F: MetadataFetch> MetadataLoader<F> {
// If a size hint is provided, read more than the minimum size
// to try and avoid a second fetch.
let footer_start = if let Some(size_hint) = prefetch {
+ // check for hint smaller than footer
+ let size_hint = std::cmp::max(size_hint, FOOTER_SIZE);
file_size.saturating_sub(size_hint)
} else {
- file_size - 8
+ file_size - FOOTER_SIZE
};
let suffix = fetch.fetch(footer_start..file_size).await?;
let suffix_len = suffix.len();
- let mut footer = [0; 8];
- footer.copy_from_slice(&suffix[suffix_len - 8..suffix_len]);
+ let mut footer = [0; FOOTER_SIZE];
+ footer.copy_from_slice(&suffix[suffix_len - FOOTER_SIZE..suffix_len]);
let length = decode_footer(&footer)?;
- if file_size < length + 8 {
+ if file_size < length + FOOTER_SIZE {
return Err(ParquetError::EOF(format!(
"file size of {} is less than footer + metadata {}",
file_size,
@@ -84,14 +87,14 @@ impl<F: MetadataFetch> MetadataLoader<F> {
}
// Did not fetch the entire file metadata in the initial read, need to
make a second request
- let (metadata, remainder) = if length > suffix_len - 8 {
- let metadata_start = file_size - length - 8;
- let meta = fetch.fetch(metadata_start..file_size - 8).await?;
+ let (metadata, remainder) = if length > suffix_len - FOOTER_SIZE {
+ let metadata_start = file_size - length - FOOTER_SIZE;
+ let meta = fetch.fetch(metadata_start..file_size -
FOOTER_SIZE).await?;
(decode_metadata(&meta)?, None)
} else {
- let metadata_start = file_size - length - 8 - footer_start;
+ let metadata_start = file_size - length - FOOTER_SIZE -
footer_start;
- let slice = &suffix[metadata_start..suffix_len - 8];
+ let slice = &suffix[metadata_start..suffix_len - FOOTER_SIZE];
(
decode_metadata(slice)?,
Some((footer_start, suffix.slice(..metadata_start))),
@@ -273,6 +276,14 @@ mod tests {
assert_eq!(actual.file_metadata().schema(), expected);
assert_eq!(fetch_count.load(Ordering::SeqCst), 2);
+ // Metadata hint too small - below footer size
+ fetch_count.store(0, Ordering::SeqCst);
+ let actual = fetch_parquet_metadata(&mut fetch, len, Some(7))
+ .await
+ .unwrap();
+ assert_eq!(actual.file_metadata().schema(), expected);
+ assert_eq!(fetch_count.load(Ordering::SeqCst), 2);
+
// Metadata hint too small
fetch_count.store(0, Ordering::SeqCst);
let actual = fetch_parquet_metadata(&mut fetch, len, Some(10))