Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 4 additions & 1 deletion datafusion/common/src/config.rs
Original file line number Diff line number Diff line change
Expand Up @@ -621,7 +621,10 @@ config_namespace! {
/// bytes of the parquet file optimistically. If not specified, two reads are required:
/// One read to fetch the 8-byte parquet footer and
/// another to fetch the metadata length encoded in the footer
pub metadata_size_hint: Option<usize>, default = None
/// Default setting to 512 KiB, which should be sufficient for most parquet files,
/// it can reduce one I/O operation per parquet file. If the metadata is larger than
/// the hint, two reads will still be performed.
pub metadata_size_hint: Option<usize>, default = Some(512 * 1024)

/// (reading) If true, filter expressions are be applied during the parquet decoding operation to
/// reduce the number of rows decoded. This optimization is sometimes called "late materialization".
Expand Down
14 changes: 14 additions & 0 deletions datafusion/core/src/datasource/file_format/options.rs
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,8 @@ pub struct ParquetReadOptions<'a> {
pub file_sort_order: Vec<Vec<SortExpr>>,
/// Properties for decryption of Parquet files that use modular encryption
pub file_decryption_properties: Option<ConfigFileDecryptionProperties>,
/// Metadata size hint for Parquet files reading (in bytes)
pub metadata_size_hint: Option<usize>,
}

impl Default for ParquetReadOptions<'_> {
Expand All @@ -281,6 +283,7 @@ impl Default for ParquetReadOptions<'_> {
schema: None,
file_sort_order: vec![],
file_decryption_properties: None,
metadata_size_hint: None,
}
}
}
Expand Down Expand Up @@ -340,6 +343,12 @@ impl<'a> ParquetReadOptions<'a> {
self.file_decryption_properties = Some(file_decryption_properties);
self
}

/// Configure metadata size hint for Parquet files reading (in bytes)
pub fn metadata_size_hint(mut self, size_hint: Option<usize>) -> Self {
self.metadata_size_hint = size_hint;
self
}
}

/// Options that control the reading of ARROW files.
Expand Down Expand Up @@ -606,6 +615,11 @@ impl ReadOptions<'_> for ParquetReadOptions<'_> {
if let Some(file_decryption_properties) = &self.file_decryption_properties {
options.crypto.file_decryption = Some(file_decryption_properties.clone());
}
// This can be overridden per-read in ParquetReadOptions, if setting.
if let Some(metadata_size_hint) = self.metadata_size_hint {
options.global.metadata_size_hint = Some(metadata_size_hint);
}

let mut file_format = ParquetFormat::new().with_options(options);

if let Some(parquet_pruning) = self.parquet_pruning {
Expand Down
7 changes: 5 additions & 2 deletions datafusion/core/src/datasource/file_format/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -546,7 +546,8 @@ mod tests {
let (files, _file_names) = store_parquet(vec![batch1], false).await?;

let state = SessionContext::new().state();
let format = ParquetFormat::default();
// Make metadata size hint None to keep original behavior
let format = ParquetFormat::default().with_metadata_size_hint(None);
let _schema = format.infer_schema(&state, &store.upcast(), &files).await?;
assert_eq!(store.request_count(), 3);
// No increase, cache being used.
Expand Down Expand Up @@ -620,7 +621,9 @@ mod tests {

let mut state = SessionContext::new().state();
state = set_view_state(state, force_views);
let format = ParquetFormat::default().with_force_view_types(force_views);
let format = ParquetFormat::default()
.with_force_view_types(force_views)
.with_metadata_size_hint(None);
let schema = format.infer_schema(&state, &store.upcast(), &files).await?;
assert_eq!(store.request_count(), 6);

Expand Down
Loading