Skip to content

Commit

Permalink
better check for invalid prefect
Browse files Browse the repository at this point in the history
  • Loading branch information
samuelcolvin committed Sep 13, 2024
1 parent 36f0af4 commit d041b13
Showing 1 changed file with 15 additions and 4 deletions.
19 changes: 15 additions & 4 deletions parquet/src/arrow/async_reader/metadata.rs
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,12 @@ impl<F: MetadataFetch> MetadataLoader<F> {
return Err(ParquetError::EOF(format!(
"file size of {file_size} is less than footer"
)));
} else if let Some(size_hint) = prefetch {
if size_hint < 8 {
return Err(ParquetError::EOF(format!(
"prefetch size of {size_hint} is less than footer size"
)));
}
}

// If a size hint is provided, read more than the minimum size
Expand All @@ -71,10 +77,7 @@ impl<F: MetadataFetch> MetadataLoader<F> {
let suffix_len = suffix.len();

let mut footer = [0; 8];
let Some(footer_start) = suffix_len.checked_sub(8) else {
return Err(ParquetError::General("Invalid Parquet file. Corrupt footer".to_string()));
};
footer.copy_from_slice(&suffix[footer_start..]);
footer.copy_from_slice(&suffix[suffix_len - 8..suffix_len]);

let length = decode_footer(&footer)?;

Expand Down Expand Up @@ -276,6 +279,14 @@ mod tests {
assert_eq!(actual.file_metadata().schema(), expected);
assert_eq!(fetch_count.load(Ordering::SeqCst), 2);

// Metadata hint invalid - below 8
fetch_count.store(0, Ordering::SeqCst);
let err = fetch_parquet_metadata(&mut fetch, len, Some(7))
.await
.unwrap_err()
.to_string();
assert_eq!(err, "EOF: prefetch size of 7 is less than footer size");

// Metadata hint too small
fetch_count.store(0, Ordering::SeqCst);
let actual = fetch_parquet_metadata(&mut fetch, len, Some(10))
Expand Down

0 comments on commit d041b13

Please sign in to comment.