33#if USE_PARQUET
44
55#include < Columns/ColumnNullable.h>
6+ #include < Core/Settings.h>
7+ #include < Core/ServerSettings.h>
8+ #include < Common/ProfileEvents.h>
69#include < Common/logger_useful.h>
710#include < Common/ThreadPool.h>
811#include < Formats/FormatFactory.h>
3538namespace ProfileEvents
3639{
3740 extern const Event ParquetFetchWaitTimeMicroseconds;
41+ extern const Event ParquetMetaDataCacheHits;
42+ extern const Event ParquetMetaDataCacheMisses;
3843}
3944
4045namespace CurrentMetrics
@@ -51,6 +56,16 @@ namespace CurrentMetrics
5156namespace DB
5257{
5358
59+ namespace Setting
60+ {
61+ extern const SettingsBool input_format_parquet_use_metadata_cache;
62+ }
63+
64+ namespace ServerSetting
65+ {
66+ extern const ServerSettingsUInt64 input_format_parquet_metadata_cache_max_size;
67+ }
68+
5469namespace ErrorCodes
5570{
5671 extern const int BAD_ARGUMENTS;
@@ -510,6 +525,58 @@ static std::vector<Range> getHyperrectangleForRowGroup(const parquet::FileMetaDa
510525 return hyperrectangle;
511526}
512527
528+ ParquetFileMetaDataCache::ParquetFileMetaDataCache (UInt64 max_size_bytes)
529+ : CacheBase(max_size_bytes) {}
530+
531+ ParquetFileMetaDataCache * ParquetFileMetaDataCache::instance (UInt64 max_size_bytes)
532+ {
533+ static ParquetFileMetaDataCache instance (max_size_bytes);
534+ return &instance;
535+ }
536+
537+ std::shared_ptr<parquet::FileMetaData> ParquetBlockInputFormat::readMetadataFromFile ()
538+ {
539+ createArrowFileIfNotCreated ();
540+ return parquet::ReadMetaData (arrow_file);
541+ }
542+
543+ std::shared_ptr<parquet::FileMetaData> ParquetBlockInputFormat::getFileMetaData ()
544+ {
545+ // in-memory cache is not implemented for local file operations, only for remote files
546+ // there is a chance the user sets `input_format_parquet_use_metadata_cache=1` for a local file operation
547+ // and the cache_key won't be set. Therefore, we also need to check for metadata_cache.key
548+ if (!metadata_cache.use_cache || metadata_cache.key .empty ())
549+ {
550+ return readMetadataFromFile ();
551+ }
552+
553+ auto [parquet_file_metadata, loaded] = ParquetFileMetaDataCache::instance (metadata_cache.max_size_bytes )->getOrSet (
554+ metadata_cache.key ,
555+ [&]()
556+ {
557+ return readMetadataFromFile ();
558+ }
559+ );
560+ if (loaded)
561+ ProfileEvents::increment (ProfileEvents::ParquetMetaDataCacheMisses);
562+ else
563+ ProfileEvents::increment (ProfileEvents::ParquetMetaDataCacheHits);
564+ return parquet_file_metadata;
565+ }
566+
567+ void ParquetBlockInputFormat::createArrowFileIfNotCreated ()
568+ {
569+ if (arrow_file)
570+ {
571+ return ;
572+ }
573+
574+ // Create arrow file adapter.
575+ // TODO: Make the adapter do prefetching on IO threads, based on the full set of ranges that
576+ // we'll need to read (which we know in advance). Use max_download_threads for that.
577+ arrow_file = asArrowFile (*in, format_settings, is_stopped, " Parquet" , PARQUET_MAGIC_BYTES, /* avoid_buffering */ true );
578+ }
579+
513580std::unordered_set<std::size_t > getBloomFilterFilteringColumnKeys (const KeyCondition::RPN & rpn)
514581{
515582 std::unordered_set<std::size_t > column_keys;
@@ -609,7 +676,7 @@ void ParquetBlockInputFormat::initializeIfNeeded()
609676 if (is_stopped)
610677 return ;
611678
612- metadata = parquet::ReadMetaData (arrow_file );
679+ metadata = getFileMetaData ( );
613680 const bool prefetch_group = supportPrefetch ();
614681
615682 std::shared_ptr<arrow::Schema> schema;
@@ -709,6 +776,8 @@ void ParquetBlockInputFormat::initializeIfNeeded()
709776 }
710777 }
711778
779+ bool has_row_groups_to_read = false ;
780+
712781 auto skip_row_group_based_on_filters = [&](int row_group)
713782 {
714783 if (!format_settings.parquet .filter_push_down && !format_settings.parquet .bloom_filter_push_down )
@@ -758,9 +827,23 @@ void ParquetBlockInputFormat::initializeIfNeeded()
758827 row_group_batches.back ().total_bytes_compressed += row_group_size;
759828 auto rows = adaptive_chunk_size (row_group);
760829 row_group_batches.back ().adaptive_chunk_size = rows ? rows : format_settings.parquet .max_block_size ;
830+
831+ has_row_groups_to_read = true ;
832+ }
833+
834+ if (has_row_groups_to_read)
835+ {
836+ createArrowFileIfNotCreated ();
761837 }
762838}
763839
840+ void ParquetBlockInputFormat::setStorageRelatedUniqueKey (const ServerSettings & server_settings, const Settings & settings, const String & key_)
841+ {
842+ metadata_cache.key = key_;
843+ metadata_cache.use_cache = settings[Setting::input_format_parquet_use_metadata_cache];
844+ metadata_cache.max_size_bytes = server_settings[ServerSetting::input_format_parquet_metadata_cache_max_size];
845+ }
846+
764847void ParquetBlockInputFormat::initializeRowGroupBatchReader (size_t row_group_batch_idx)
765848{
766849 const bool row_group_prefetch = supportPrefetch ();
0 commit comments