File tree Expand file tree Collapse file tree 2 files changed +16
-2
lines changed
datafusion/datasource-parquet/src Expand file tree Collapse file tree 2 files changed +16
-2
lines changed Original file line number Diff line number Diff line change @@ -1008,10 +1008,20 @@ pub async fn fetch_parquet_metadata(
10081008 #[ cfg( feature = "parquet_encryption" ) ]
10091009 let reader = reader. with_decryption_properties ( decryption_properties) ;
10101010
1011- reader
1011+ let metadata = reader
10121012 . load_and_finish ( fetch, file_size)
10131013 . await
1014- . map_err ( DataFusionError :: from)
1014+ . map_err ( DataFusionError :: from) ?;
1015+
1016+ if cache_metadata {
1017+ if let Some ( cache) = file_metadata_cache {
1018+ let cached_metadata =
1019+ Arc :: new ( CachedParquetMetaData :: new ( Arc :: new ( metadata. clone ( ) ) ) ) ;
1020+ cache. put ( meta, cached_metadata) ;
1021+ }
1022+ }
1023+
1024+ Ok ( metadata)
10151025}
10161026
10171027/// Read and parse the schema of the Parquet file at location `path`
Original file line number Diff line number Diff line change @@ -285,6 +285,10 @@ impl AsyncFileReader for CachedParquetFileReader {
285285pub struct CachedParquetMetaData ( Arc < ParquetMetaData > ) ;
286286
287287impl CachedParquetMetaData {
288+ pub fn new ( metadata : Arc < ParquetMetaData > ) -> Self {
289+ Self ( metadata)
290+ }
291+
288292 pub fn parquet_metadata ( & self ) -> & Arc < ParquetMetaData > {
289293 & self . 0
290294 }
You can’t perform that action at this time.
0 commit comments