Skip to content
Draft
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
98 changes: 60 additions & 38 deletions datafusion/datasource-parquet/src/metadata.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,7 @@
//! [`DFParquetMetadata`] for fetching Parquet file metadata, statistics
//! and schema information.

use crate::{
ObjectStoreFetch, apply_file_schema_type_coercions, coerce_int96_to_resolution,
};
use crate::{apply_file_schema_type_coercions, coerce_int96_to_resolution};
use arrow::array::{Array, ArrayRef, BooleanArray};
use arrow::compute::and;
use arrow::compute::kernels::cmp::eq;
Expand All @@ -44,10 +42,11 @@ use object_store::{ObjectMeta, ObjectStore};
use parquet::arrow::arrow_reader::statistics::StatisticsConverter;
use parquet::arrow::{parquet_column, parquet_to_arrow_schema};
use parquet::file::metadata::{
PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData,
PageIndexPolicy, ParquetMetaData, ParquetMetaDataPushDecoder, RowGroupMetaData,
SortingColumn,
};
use parquet::schema::types::SchemaDescriptor;
use parquet::DecodeResult;
use std::any::Any;
use std::collections::HashMap;
use std::sync::Arc;
Expand Down Expand Up @@ -114,25 +113,15 @@ impl<'a> DFParquetMetadata<'a> {

/// Fetch parquet metadata from the remote object store
pub async fn fetch_metadata(&self) -> Result<Arc<ParquetMetaData>> {
let Self {
store,
object_meta,
metadata_size_hint,
decryption_properties,
file_metadata_cache,
coerce_int96: _,
} = self;

let fetch = ObjectStoreFetch::new(*store, object_meta);

// implementation to fetch parquet metadata
let cache_metadata =
!cfg!(feature = "parquet_encryption") || decryption_properties.is_none();
let cache_metadata = !cfg!(feature = "parquet_encryption")
|| self.decryption_properties.is_none();

if cache_metadata
&& let Some(file_metadata_cache) = file_metadata_cache.as_ref()
&& let Some(cached) = file_metadata_cache.get(&object_meta.location)
&& cached.is_valid_for(object_meta)
&& let Some(file_metadata_cache) = self.file_metadata_cache.as_ref()
&& let Some(cached) =
file_metadata_cache.get(&self.object_meta.location)
&& cached.is_valid_for(self.object_meta)
&& let Some(cached_parquet) = cached
.file_metadata
.as_any()
Expand All @@ -141,32 +130,65 @@ impl<'a> DFParquetMetadata<'a> {
return Ok(Arc::clone(cached_parquet.parquet_metadata()));
}

let mut reader =
ParquetMetaDataReader::new().with_prefetch_hint(*metadata_size_hint);
let file_size = self.object_meta.size as u64;
let mut decoder = ParquetMetaDataPushDecoder::try_new(file_size)
.map_err(DataFusionError::from)?;

#[cfg(feature = "parquet_encryption")]
if let Some(decryption_properties) = decryption_properties {
reader = reader
.with_decryption_properties(Some(Arc::clone(decryption_properties)));
}

if cache_metadata && file_metadata_cache.is_some() {
if cache_metadata && self.file_metadata_cache.is_some() {
// Need to retrieve the entire metadata for the caching to be effective.
reader = reader.with_page_index_policy(PageIndexPolicy::Optional);
decoder = decoder.with_page_index_policy(PageIndexPolicy::Optional);
} else {
decoder = decoder.with_page_index_policy(PageIndexPolicy::Skip);
}

let metadata = Arc::new(
reader
.load_and_finish(fetch, object_meta.size)
// If we have a size hint, prefetch that many bytes from the end of the file
if let Some(hint) = self.metadata_size_hint {
let prefetch_start = file_size.saturating_sub(hint as u64);
let prefetch_range = prefetch_start..file_size;
let data = self
.store
.get_ranges(
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This and next requests can go on a queue.

&self.object_meta.location,
&[prefetch_range.clone()],
)
.await
.map_err(DataFusionError::from)?,
);
.map_err(DataFusionError::from)?;
decoder
.push_ranges(vec![prefetch_range], data)
.map_err(DataFusionError::from)?;
}

let metadata = loop {
match decoder.try_decode().map_err(DataFusionError::from)? {
DecodeResult::Data(metadata) => break metadata,
DecodeResult::NeedsData(ranges) => {
let buffers = self
.store
.get_ranges(&self.object_meta.location, &ranges)
.await
.map_err(DataFusionError::from)?;
decoder
.push_ranges(ranges, buffers)
.map_err(DataFusionError::from)?;
}
DecodeResult::Finished => {
return Err(DataFusionError::Internal(
"ParquetMetaDataPushDecoder finished without producing metadata"
.to_string(),
));
}
}
};

let metadata = Arc::new(metadata);

if cache_metadata && let Some(file_metadata_cache) = file_metadata_cache {
if cache_metadata
&& let Some(file_metadata_cache) = &self.file_metadata_cache
{
file_metadata_cache.put(
&object_meta.location,
&self.object_meta.location,
CachedFileMetadataEntry::new(
(*object_meta).clone(),
self.object_meta.clone(),
Arc::new(CachedParquetMetaData::new(Arc::clone(&metadata))),
),
);
Expand Down
Loading