@@ -2,9 +2,10 @@ use std::sync::{Arc, Mutex};
22use datafusion:: execution:: cache:: cache_manager:: { FileMetadataCache , CacheManagerConfig } ;
33use datafusion:: execution:: cache:: cache_unit:: { DefaultFileStatisticsCache , DefaultFilesMetadataCache , DefaultListFilesCache } ;
44use datafusion:: execution:: cache:: CacheAccessor ;
5+ use datafusion:: datasource:: physical_plan:: parquet:: metadata:: DFParquetMetadata ;
56use tokio:: runtime:: Runtime ;
67use crate :: cache:: MutexFileMetadataCache ;
7- use crate :: util:: { create_object_meta_from_file, construct_file_metadata } ;
8+ use crate :: util:: { create_object_meta_from_file} ;
89
910/// Custom CacheManager that holds cache references directly
1011pub struct CustomCacheManager {
@@ -39,7 +40,8 @@ impl CustomCacheManager {
3940 let file_static_cache = Arc :: new ( DefaultFileStatisticsCache :: default ( ) ) ;
4041 // Add file metadata cache if available
4142 if let Some ( cache) = self . get_file_metadata_cache_for_datafusion ( ) {
42- config = config. with_file_metadata_cache ( Some ( cache) ) ;
43+ config = config. with_file_metadata_cache ( Some ( cache. clone ( ) ) )
44+ . with_metadata_cache_limit ( cache. cache_limit ( ) ) ;
4345 }
4446 config = config. with_files_statistics_cache ( Some ( file_static_cache. clone ( ) ) ) ;
4547 // Future: Add stats cache when implemented
@@ -256,31 +258,40 @@ impl CustomCacheManager {
256258
257259 let store = Arc :: new ( object_store:: local:: LocalFileSystem :: new ( ) ) ;
258260
259- //TODO: Use TokioRuntimePtr to block on the async operation
260- let metadata = Runtime :: new ( )
261+ // Get cache reference for DataFusion metadata loading
262+ let cache_ref = self . file_metadata_cache . as_ref ( )
263+ . ok_or_else ( || "No file metadata cache configured" . to_string ( ) ) ?;
264+
265+ let metadata_cache = cache_ref. clone ( ) as Arc < dyn FileMetadataCache > ;
266+
267+ // Use DataFusion's metadata loading by passing reference to file_metadata_cache to get complete metadata
268+ // IMPORTANT: When a cache is provided to DFParquetMetadata, fetch_metadata() will:
269+ // 1. Enable page index loading (with_page_indexes(true))
270+ // 2. Load the complete metadata including column and offset indexes
271+ // 3. Automatically put the metadata into the cache (lines 155-160 in datafusion's metadata.rs)
272+ // This ensures we cache exactly what DataFusion would cache during query execution
273+ let _parquet_metadata = Runtime :: new ( )
261274 . map_err ( |e| format ! ( "Failed to create Tokio Runtime: {}" , e) ) ?
262275 . block_on ( async {
263- construct_file_metadata ( store. as_ref ( ) , object_meta, data_format)
264- . await
265- . map_err ( |e| format ! ( "Failed to construct file metadata: {}" , e) )
266- } ) ?;
276+ let df_metadata = DFParquetMetadata :: new ( store. as_ref ( ) , object_meta)
277+ . with_file_metadata_cache ( Some ( metadata_cache) ) ;
267278
268- // Get the cache directly from our stored reference
269- let cache = self . file_metadata_cache . as_ref ( )
270- . ok_or_else ( || "No file metadata cache configured" . to_string ( ) ) ?;
271-
272- match cache. inner . lock ( ) {
273- Ok ( mut cache_guard) => {
274- cache_guard. put ( object_meta, metadata) ;
279+ // fetch_metadata() performs the cache put operation internally
280+ df_metadata. fetch_metadata ( ) . await
281+ . map_err ( |e| format ! ( "Failed to fetch metadata: {}" , e) )
282+ } ) ?;
275283
284+ // Verify the metadata was cached properly
285+ match cache_ref. inner . lock ( ) {
286+ Ok ( cache_guard) => {
276287 if cache_guard. contains_key ( object_meta) {
277288 Ok ( true )
278289 } else {
279- println ! ( "Failed to cache metadata for: {}" , file_path) ;
290+ println ! ( "[CACHE ERROR] Failed to cache metadata for: {}" , file_path) ;
280291 Ok ( false )
281292 }
282293 }
283- Err ( e) => Err ( format ! ( "Cache put failed : {}" , e) )
294+ Err ( e) => Err ( format ! ( "Failed to verify cache : {}" , e) )
284295 }
285296 }
286297}
0 commit comments