alamb commented on code in PR #17127:
URL: https://github.com/apache/datafusion/pull/17127#discussion_r2292148773


##########
datafusion/datasource-parquet/src/reader.rs:
##########
@@ -256,44 +258,19 @@ impl AsyncFileReader for CachedParquetFileReader {
             #[cfg(not(feature = "parquet_encryption"))]
             let file_decryption_properties = None;
 
-            fetch_parquet_metadata(
-                &mut self.inner,
-                &file_meta.object_meta,
-                None,
-                file_decryption_properties,
-                Some(metadata_cache),
-            )
-            .await
-            .map_err(|e| {
-                parquet::errors::ParquetError::General(format!(
-                    "Failed to fetch metadata for file {}: {e}",
-                    file_meta.object_meta.location,
-                ))
-            })
+            // TODO should there be metadata prefetch hint here?

Review Comment:
   - Filed https://github.com/apache/datafusion/issues/17279



##########
datafusion/datasource-parquet/src/metadata.rs:
##########
@@ -0,0 +1,564 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! [`DFParquetMetadata`] for fetching Parquet file metadata, statistics
+//! and schema information.
+
+use crate::{
+    apply_file_schema_type_coercions, coerce_int96_to_resolution, 
ObjectStoreFetch,
+};
+use arrow::array::{ArrayRef, BooleanArray};
+use arrow::compute::and;
+use arrow::compute::kernels::cmp::eq;
+use arrow::compute::sum;
+use arrow::datatypes::{DataType, Schema, SchemaRef, TimeUnit};
+use datafusion_common::encryption::FileDecryptionProperties;
+use datafusion_common::stats::Precision;
+use datafusion_common::{
+    ColumnStatistics, DataFusionError, Result, ScalarValue, Statistics,
+};
+use datafusion_execution::cache::cache_manager::{FileMetadata, 
FileMetadataCache};
+use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator};
+use datafusion_physical_plan::Accumulator;
+use log::debug;
+use object_store::path::Path;
+use object_store::{ObjectMeta, ObjectStore};
+use parquet::arrow::arrow_reader::statistics::StatisticsConverter;
+use parquet::arrow::parquet_to_arrow_schema;
+use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader, 
RowGroupMetaData};
+use std::any::Any;
+use std::collections::HashMap;
+use std::sync::Arc;
+
+/// Handles fetching Parquet file schema, metadata and statistics
+/// from object store.
+///
+/// This component is exposed for low level integrations through
+/// [`ParquetFileReaderFactory`].
+///
+/// [`ParquetFileReaderFactory`]: crate::ParquetFileReaderFactory
+#[derive(Debug)]
+pub struct DFParquetMetadata<'a> {
+    store: &'a dyn ObjectStore,
+    object_meta: &'a ObjectMeta,
+    metadata_size_hint: Option<usize>,
+    decryption_properties: Option<&'a FileDecryptionProperties>,
+    file_metadata_cache: Option<Arc<dyn FileMetadataCache>>,
+    /// timeunit to coerce INT96 timestamps to
+    pub coerce_int96: Option<TimeUnit>,
+}
+
+impl<'a> DFParquetMetadata<'a> {
+    pub fn new(store: &'a dyn ObjectStore, object_meta: &'a ObjectMeta) -> 
Self {
+        Self {
+            store,
+            object_meta,
+            metadata_size_hint: None,
+            decryption_properties: None,
+            file_metadata_cache: None,
+            coerce_int96: None,
+        }
+    }
+
+    /// set metadata size hint
+    pub fn with_metadata_size_hint(mut self, metadata_size_hint: 
Option<usize>) -> Self {
+        self.metadata_size_hint = metadata_size_hint;
+        self
+    }
+
+    /// set decryption properties
+    pub fn with_decryption_properties(
+        mut self,
+        decryption_properties: Option<&'a FileDecryptionProperties>,
+    ) -> Self {
+        self.decryption_properties = decryption_properties;
+        self
+    }
+
+    /// set file metadata cache
+    pub fn with_file_metadata_cache(
+        mut self,
+        file_metadata_cache: Option<Arc<dyn FileMetadataCache>>,
+    ) -> Self {
+        self.file_metadata_cache = file_metadata_cache;
+        self
+    }
+
+    /// Set timeunit to coerce INT96 timestamps to
+    pub fn with_coerce_int96(mut self, time_unit: Option<TimeUnit>) -> Self {
+        self.coerce_int96 = time_unit;
+        self
+    }
+
+    /// fetch parquet metadata
+    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();
+
+        if cache_metadata {
+            if let Some(parquet_metadata) = file_metadata_cache
+                .as_ref()
+                .and_then(|file_metadata_cache| 
file_metadata_cache.get(object_meta))
+                .and_then(|file_metadata| {
+                    file_metadata
+                        .as_any()
+                        .downcast_ref::<CachedParquetMetaData>()
+                        .map(|cached_parquet_metadata| {
+                            
Arc::clone(cached_parquet_metadata.parquet_metadata())
+                        })
+                })
+            {
+                return Ok(parquet_metadata);
+            }
+        }
+
+        let mut reader =
+            
ParquetMetaDataReader::new().with_prefetch_hint(*metadata_size_hint);
+
+        #[cfg(feature = "parquet_encryption")]
+        if let Some(decryption_properties) = decryption_properties {
+            reader = 
reader.with_decryption_properties(Some(decryption_properties));
+        }
+
+        if cache_metadata && file_metadata_cache.is_some() {
+            // Need to retrieve the entire metadata for the caching to be 
effective.

Review Comment:
   Yes, I think that is exactly what it it means, which is confusing. 



##########
datafusion/datasource-parquet/src/metadata.rs:
##########
@@ -0,0 +1,564 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! [`DFParquetMetadata`] for fetching Parquet file metadata, statistics
+//! and schema information.
+
+use crate::{
+    apply_file_schema_type_coercions, coerce_int96_to_resolution, 
ObjectStoreFetch,
+};
+use arrow::array::{ArrayRef, BooleanArray};
+use arrow::compute::and;
+use arrow::compute::kernels::cmp::eq;
+use arrow::compute::sum;
+use arrow::datatypes::{DataType, Schema, SchemaRef, TimeUnit};
+use datafusion_common::encryption::FileDecryptionProperties;
+use datafusion_common::stats::Precision;
+use datafusion_common::{
+    ColumnStatistics, DataFusionError, Result, ScalarValue, Statistics,
+};
+use datafusion_execution::cache::cache_manager::{FileMetadata, 
FileMetadataCache};
+use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator};
+use datafusion_physical_plan::Accumulator;
+use log::debug;
+use object_store::path::Path;
+use object_store::{ObjectMeta, ObjectStore};
+use parquet::arrow::arrow_reader::statistics::StatisticsConverter;
+use parquet::arrow::parquet_to_arrow_schema;
+use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader, 
RowGroupMetaData};
+use std::any::Any;
+use std::collections::HashMap;
+use std::sync::Arc;
+
+/// Handles fetching Parquet file schema, metadata and statistics
+/// from object store.
+///
+/// This component is exposed for low level integrations through
+/// [`ParquetFileReaderFactory`].
+///
+/// [`ParquetFileReaderFactory`]: crate::ParquetFileReaderFactory
+#[derive(Debug)]
+pub struct DFParquetMetadata<'a> {
+    store: &'a dyn ObjectStore,
+    object_meta: &'a ObjectMeta,
+    metadata_size_hint: Option<usize>,
+    decryption_properties: Option<&'a FileDecryptionProperties>,
+    file_metadata_cache: Option<Arc<dyn FileMetadataCache>>,
+    /// timeunit to coerce INT96 timestamps to
+    pub coerce_int96: Option<TimeUnit>,
+}
+
+impl<'a> DFParquetMetadata<'a> {
+    pub fn new(store: &'a dyn ObjectStore, object_meta: &'a ObjectMeta) -> 
Self {
+        Self {
+            store,
+            object_meta,
+            metadata_size_hint: None,
+            decryption_properties: None,
+            file_metadata_cache: None,
+            coerce_int96: None,
+        }
+    }
+
+    /// set metadata size hint
+    pub fn with_metadata_size_hint(mut self, metadata_size_hint: 
Option<usize>) -> Self {
+        self.metadata_size_hint = metadata_size_hint;
+        self
+    }
+
+    /// set decryption properties
+    pub fn with_decryption_properties(
+        mut self,
+        decryption_properties: Option<&'a FileDecryptionProperties>,
+    ) -> Self {
+        self.decryption_properties = decryption_properties;
+        self
+    }
+
+    /// set file metadata cache
+    pub fn with_file_metadata_cache(
+        mut self,
+        file_metadata_cache: Option<Arc<dyn FileMetadataCache>>,
+    ) -> Self {
+        self.file_metadata_cache = file_metadata_cache;
+        self
+    }
+
+    /// Set timeunit to coerce INT96 timestamps to
+    pub fn with_coerce_int96(mut self, time_unit: Option<TimeUnit>) -> Self {
+        self.coerce_int96 = time_unit;
+        self
+    }
+
+    /// fetch parquet metadata
+    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();
+
+        if cache_metadata {
+            if let Some(parquet_metadata) = file_metadata_cache
+                .as_ref()
+                .and_then(|file_metadata_cache| 
file_metadata_cache.get(object_meta))
+                .and_then(|file_metadata| {
+                    file_metadata
+                        .as_any()
+                        .downcast_ref::<CachedParquetMetaData>()
+                        .map(|cached_parquet_metadata| {
+                            
Arc::clone(cached_parquet_metadata.parquet_metadata())
+                        })
+                })
+            {
+                return Ok(parquet_metadata);
+            }
+        }
+
+        let mut reader =
+            
ParquetMetaDataReader::new().with_prefetch_hint(*metadata_size_hint);
+
+        #[cfg(feature = "parquet_encryption")]
+        if let Some(decryption_properties) = decryption_properties {
+            reader = 
reader.with_decryption_properties(Some(decryption_properties));
+        }
+
+        if cache_metadata && file_metadata_cache.is_some() {
+            // Need to retrieve the entire metadata for the caching to be 
effective.
+            reader = reader.with_page_indexes(true);
+        }
+
+        let metadata = Arc::new(
+            reader
+                .load_and_finish(fetch, object_meta.size)
+                .await
+                .map_err(DataFusionError::from)?,
+        );
+
+        if cache_metadata {
+            if let Some(file_metadata_cache) = file_metadata_cache {
+                file_metadata_cache.put(
+                    object_meta,
+                    
Arc::new(CachedParquetMetaData::new(Arc::clone(&metadata))),
+                );
+            }
+        }
+
+        Ok(metadata)
+    }
+
+    /// Read and parse the schema of the Parquet file
+    pub async fn fetch_schema(&self) -> Result<Schema> {
+        let metadata = self.fetch_metadata().await?;

Review Comment:
   TLDR is I am not 100% sure, but I am to find out with @BlakeOrth 
   - https://github.com/apache/datafusion/pull/17266



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org
For additional commands, e-mail: github-h...@datafusion.apache.org

Reply via email to