From da2f28c44bbc7d662b9ff0ca4ffb16f480a2ddd3 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 28 Jul 2025 15:39:46 -0300 Subject: [PATCH] Revert "Antalya 25.6.5: Forward port of #795 - Parquet Metadata Caching" --- programs/server/Server.cpp | 9 --- src/Access/Common/AccessType.h | 1 - src/Common/ProfileEvents.cpp | 3 +- src/Core/FormatFactorySettings.h | 3 +- src/Core/ServerSettings.cpp | 3 +- src/Core/SettingsChangesHistory.cpp | 5 -- src/Interpreters/InterpreterSystemQuery.cpp | 15 ---- src/Parsers/ASTSystemQuery.cpp | 1 - src/Parsers/ASTSystemQuery.h | 1 - src/Processors/Formats/IInputFormat.h | 3 - .../Formats/Impl/ParquetBlockInputFormat.cpp | 76 +------------------ .../Formats/Impl/ParquetBlockInputFormat.h | 14 ---- .../Formats/Impl/ParquetFileMetaDataCache.cpp | 20 ----- .../Formats/Impl/ParquetFileMetaDataCache.h | 30 -------- .../Impl/ParquetMetadataInputFormat.cpp | 53 +------------ .../Formats/Impl/ParquetMetadataInputFormat.h | 10 --- .../StorageObjectStorageSource.cpp | 3 - .../01271_show_privileges.reference | 1 - .../0_stateless/02995_settings_25_2_3.tsv | 1 - ...et_object_storage_metadata_cache.reference | 8 -- ..._parquet_object_storage_metadata_cache.sql | 61 --------------- 21 files changed, 10 insertions(+), 311 deletions(-) delete mode 100644 src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp delete mode 100644 src/Processors/Formats/Impl/ParquetFileMetaDataCache.h delete mode 100644 tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.reference delete mode 100644 tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1334b4fcad41..cb2286cc28e8 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -156,10 +156,6 @@ # include #endif -#if USE_PARQUET -# include -#endif - #include /// A minimal file used when the server is run without installation @@ -330,7 +326,6 @@ namespace ServerSetting extern const ServerSettingsUInt64 os_cpu_busy_time_threshold; extern const ServerSettingsFloat min_os_cpu_wait_time_ratio_to_drop_connection; extern const ServerSettingsFloat max_os_cpu_wait_time_ratio_to_drop_connection; - extern const ServerSettingsUInt64 input_format_parquet_metadata_cache_max_size; } namespace ErrorCodes @@ -2428,10 +2423,6 @@ try auto replicas_reconnector = ReplicasReconnector::init(global_context); -#if USE_PARQUET - ParquetFileMetaDataCache::instance()->setMaxSizeInBytes(server_settings[ServerSetting::input_format_parquet_metadata_cache_max_size]); -#endif - /// Set current database name before loading tables and databases because /// system logs may copy global context. std::string default_database = server_settings[ServerSetting::default_database]; diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 5d5b57c4f71c..961a90283160 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -184,7 +184,6 @@ enum class AccessType : uint8_t M(SYSTEM_DROP_SCHEMA_CACHE, "SYSTEM DROP SCHEMA CACHE, DROP SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_FORMAT_SCHEMA_CACHE, "SYSTEM DROP FORMAT SCHEMA CACHE, DROP FORMAT SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_S3_CLIENT_CACHE, "SYSTEM DROP S3 CLIENT, DROP S3 CLIENT CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ - M(SYSTEM_DROP_PARQUET_METADATA_CACHE, "SYSTEM DROP PARQUET METADATA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_CACHE, "DROP CACHE", GROUP, SYSTEM) \ M(SYSTEM_RELOAD_CONFIG, "RELOAD CONFIG", GLOBAL, SYSTEM_RELOAD) \ M(SYSTEM_RELOAD_USERS, "RELOAD USERS", GLOBAL, SYSTEM_RELOAD) \ diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index a3839adba860..687969b5f678 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -1041,8 +1041,7 @@ The server successfully detected this situation and will download merged part fr M(IndexGenericExclusionSearchAlgorithm, "Number of times the generic exclusion search algorithm is used over the index marks", ValueType::Number) \ M(ParallelReplicasQueryCount, "Number of (sub)queries executed using parallel replicas during a query execution", ValueType::Number) \ M(DistributedConnectionReconnectCount, "Number of reconnects to other servers done during distributed query execution. It can happen when a stale connection has been acquired from connection pool", ValueType::Number) \ - M(ParquetMetaDataCacheHits, "Number of times the read from filesystem cache hit the cache.", ValueType::Number) \ - M(ParquetMetaDataCacheMisses, "Number of times the read from filesystem cache miss the cache.", ValueType::Number) \ + #ifdef APPLY_FOR_EXTERNAL_EVENTS #define APPLY_FOR_EVENTS(M) APPLY_FOR_BUILTIN_EVENTS(M) APPLY_FOR_EXTERNAL_EVENTS(M) diff --git a/src/Core/FormatFactorySettings.h b/src/Core/FormatFactorySettings.h index 0b318be81eee..022194d20c57 100644 --- a/src/Core/FormatFactorySettings.h +++ b/src/Core/FormatFactorySettings.h @@ -1348,7 +1348,8 @@ Limits the size of the blocks formed during data parsing in input formats in byt DECLARE(Bool, input_format_parquet_allow_geoparquet_parser, true, R"( Use geo column parser to convert Array(UInt8) into Point/Linestring/Polygon/MultiLineString/MultiPolygon types )", 0) \ - DECLARE(Bool, input_format_parquet_use_metadata_cache, true, R"(Enable parquet file metadata caching)", 0) \ + + // End of FORMAT_FACTORY_SETTINGS #define OBSOLETE_FORMAT_SETTINGS(M, ALIAS) \ diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index dc08087b8efe..fc964e21892f 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1064,7 +1064,8 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ See [Controlling behavior on server CPU overload](/operations/settings/server-overload) for more details. )", 0) \ DECLARE(Float, distributed_cache_keep_up_free_connections_ratio, 0.1f, "Soft limit for number of active connection distributed cache will try to keep free. After the number of free connections goes below distributed_cache_keep_up_free_connections_ratio * max_connections, connections with oldest activity will be closed until the number goes above the limit.", 0) \ - DECLARE(UInt64, input_format_parquet_metadata_cache_max_size, 500000000, "Maximum size of parquet file metadata cache", 0) \ + + // clang-format on /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 069d1dfb4b21..6e6bfe580117 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -182,11 +182,6 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"parallel_hash_join_threshold", 0, 0, "New setting"}, /// Release closed. Please use 25.4 }); - addSettingsChanges(settings_changes_history, "24.12.2.20000", - { - // Altinity Antalya modifications atop of 24.12 - {"input_format_parquet_use_metadata_cache", true, true, "New setting, turned ON by default"}, // https://github.com/Altinity/ClickHouse/pull/586 - }); addSettingsChanges(settings_changes_history, "25.2", { /// Release closed. Please use 25.3 diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 7bac5d20964b..f11e9598308e 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -79,10 +79,6 @@ #include #endif -#if USE_PARQUET -#include -#endif - #if USE_AWS_S3 #include #endif @@ -437,16 +433,6 @@ BlockIO InterpreterSystemQuery::execute() getContext()->clearQueryResultCache(query.query_result_cache_tag); break; } - case Type::DROP_PARQUET_METADATA_CACHE: - { -#if USE_PARQUET - getContext()->checkAccess(AccessType::SYSTEM_DROP_PARQUET_METADATA_CACHE); - ParquetFileMetaDataCache::instance()->clear(); - break; -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "The server was compiled without the support for Parquet"); -#endif - } case Type::DROP_COMPILED_EXPRESSION_CACHE: #if USE_EMBEDDED_COMPILER getContext()->checkAccess(AccessType::SYSTEM_DROP_COMPILED_EXPRESSION_CACHE); @@ -1547,7 +1533,6 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::DROP_PAGE_CACHE: case Type::DROP_SCHEMA_CACHE: case Type::DROP_FORMAT_SCHEMA_CACHE: - case Type::DROP_PARQUET_METADATA_CACHE: case Type::DROP_S3_CLIENT_CACHE: { required_access.emplace_back(AccessType::SYSTEM_DROP_CACHE); diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 3ee6161c7c20..fac766a01180 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -467,7 +467,6 @@ void ASTSystemQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & setti case Type::DROP_COMPILED_EXPRESSION_CACHE: case Type::DROP_S3_CLIENT_CACHE: case Type::DROP_ICEBERG_METADATA_CACHE: - case Type::DROP_PARQUET_METADATA_CACHE: case Type::RESET_COVERAGE: case Type::RESTART_REPLICAS: case Type::JEMALLOC_PURGE: diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 73e1182cb6be..5d650df64ff3 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -42,7 +42,6 @@ class ASTSystemQuery : public IAST, public ASTQueryWithOnCluster DROP_SCHEMA_CACHE, DROP_FORMAT_SCHEMA_CACHE, DROP_S3_CLIENT_CACHE, - DROP_PARQUET_METADATA_CACHE, STOP_LISTEN, START_LISTEN, RESTART_REPLICAS, diff --git a/src/Processors/Formats/IInputFormat.h b/src/Processors/Formats/IInputFormat.h index 70fdd9a3e23a..c59f1e714038 100644 --- a/src/Processors/Formats/IInputFormat.h +++ b/src/Processors/Formats/IInputFormat.h @@ -70,9 +70,6 @@ class IInputFormat : public SourceWithKeyCondition void needOnlyCount() { need_only_count = true; } - /// Set additional info/key/id related to underlying storage of the ReadBuffer - virtual void setStorageRelatedUniqueKey(const Settings & /*settings*/, const String & /*key*/) {} - protected: ReadBuffer & getReadBuffer() const { chassert(in); return *in; } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 22bfc9b7d595..67994192a6b3 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -3,9 +3,6 @@ #if USE_PARQUET #include -#include -#include -#include #include #include #include @@ -32,7 +29,6 @@ #include #include #include -#include #include #include @@ -42,8 +38,6 @@ namespace ProfileEvents extern const Event ParquetFetchWaitTimeMicroseconds; extern const Event ParquetReadRowGroups; extern const Event ParquetPrunedRowGroups; - extern const Event ParquetMetaDataCacheHits; - extern const Event ParquetMetaDataCacheMisses; } namespace CurrentMetrics @@ -60,16 +54,6 @@ namespace CurrentMetrics namespace DB { -namespace Setting -{ - extern const SettingsBool input_format_parquet_use_metadata_cache; -} - -namespace ServerSetting -{ - extern const ServerSettingsUInt64 input_format_parquet_metadata_cache_max_size; -} - namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -529,49 +513,6 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa return hyperrectangle; } -std::shared_ptr ParquetBlockInputFormat::readMetadataFromFile() -{ - createArrowFileIfNotCreated(); - return parquet::ReadMetaData(arrow_file); -} - -std::shared_ptr ParquetBlockInputFormat::getFileMetaData() -{ - // in-memory cache is not implemented for local file operations, only for remote files - // there is a chance the user sets `input_format_parquet_use_metadata_cache=1` for a local file operation - // and the cache_key won't be set. Therefore, we also need to check for metadata_cache.key - if (!metadata_cache.use_cache || metadata_cache.key.empty()) - { - return readMetadataFromFile(); - } - - auto [parquet_file_metadata, loaded] = ParquetFileMetaDataCache::instance()->getOrSet( - metadata_cache.key, - [&]() - { - return readMetadataFromFile(); - } - ); - if (loaded) - ProfileEvents::increment(ProfileEvents::ParquetMetaDataCacheMisses); - else - ProfileEvents::increment(ProfileEvents::ParquetMetaDataCacheHits); - return parquet_file_metadata; -} - -void ParquetBlockInputFormat::createArrowFileIfNotCreated() -{ - if (arrow_file) - { - return; - } - - // Create arrow file adapter. - // TODO: Make the adapter do prefetching on IO threads, based on the full set of ranges that - // we'll need to read (which we know in advance). Use max_download_threads for that. - arrow_file = asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); -} - std::unordered_set getBloomFilterFilteringColumnKeys(const KeyCondition::RPN & rpn) { std::unordered_set column_keys; @@ -671,7 +612,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() if (is_stopped) return; - metadata = getFileMetaData(); + metadata = parquet::ReadMetaData(arrow_file); const bool prefetch_group = supportPrefetch(); std::shared_ptr schema; @@ -771,8 +712,6 @@ void ParquetBlockInputFormat::initializeIfNeeded() } } - bool has_row_groups_to_read = false; - auto skip_row_group_based_on_filters = [&](int row_group) { if (!format_settings.parquet.filter_push_down && !format_settings.parquet.bloom_filter_push_down) @@ -824,20 +763,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() row_group_batches.back().total_bytes_compressed += row_group_size; auto rows = adaptive_chunk_size(row_group); row_group_batches.back().adaptive_chunk_size = rows ? rows : format_settings.parquet.max_block_size; - - has_row_groups_to_read = true; } - - if (has_row_groups_to_read) - { - createArrowFileIfNotCreated(); - } -} - -void ParquetBlockInputFormat::setStorageRelatedUniqueKey(const Settings & settings, const String & key_) -{ - metadata_cache.key = key_; - metadata_cache.use_cache = settings[Setting::input_format_parquet_use_metadata_cache]; } void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_batch_idx) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index a1b7eade4303..f941bb70ebce 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -72,8 +72,6 @@ class ParquetBlockInputFormat : public IInputFormat size_t getApproxBytesReadForChunk() const override { return previous_approx_bytes_read_for_chunk; } - void setStorageRelatedUniqueKey(const Settings & settings, const String & key_) override; - private: Chunk read() override; @@ -92,11 +90,6 @@ class ParquetBlockInputFormat : public IInputFormat void threadFunction(size_t row_group_batch_idx); - void createArrowFileIfNotCreated(); - std::shared_ptr readMetadataFromFile(); - - std::shared_ptr getFileMetaData(); - inline bool supportPrefetch() const; // Data layout in the file: @@ -345,13 +338,6 @@ class ParquetBlockInputFormat : public IInputFormat std::exception_ptr background_exception = nullptr; std::atomic is_stopped{0}; bool is_initialized = false; - struct Cache - { - String key; - bool use_cache = false; - }; - - Cache metadata_cache; }; class ParquetSchemaReader : public ISchemaReader diff --git a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp deleted file mode 100644 index da8ad825f505..000000000000 --- a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp +++ /dev/null @@ -1,20 +0,0 @@ -#include - -#if USE_PARQUET - -namespace DB -{ - -ParquetFileMetaDataCache::ParquetFileMetaDataCache() - : CacheBase(CurrentMetrics::end(), CurrentMetrics::end(), 0) -{} - -ParquetFileMetaDataCache * ParquetFileMetaDataCache::instance() -{ - static ParquetFileMetaDataCache instance; - return &instance; -} - -} - -#endif diff --git a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h deleted file mode 100644 index fb5fc1bb0217..000000000000 --- a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_PARQUET - -namespace parquet -{ - -class FileMetaData; - -} - -#include - -namespace DB -{ - -class ParquetFileMetaDataCache : public CacheBase -{ -public: - static ParquetFileMetaDataCache * instance(); - -private: - ParquetFileMetaDataCache(); -}; - -} - -#endif diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp index ccdaf4274ab4..d9b55b2fc79c 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp @@ -22,17 +22,8 @@ #include #include "ArrowBufferedStreams.h" #include -#include -#include -#include -namespace ProfileEvents -{ -extern const Event ParquetMetaDataCacheHits; -extern const Event ParquetMetaDataCacheMisses; -} - namespace DB { @@ -41,11 +32,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -namespace Setting -{ -extern const SettingsBool input_format_parquet_use_metadata_cache; -} - static NamesAndTypesList getHeaderForParquetMetadata() { NamesAndTypesList names_and_types{ @@ -144,35 +130,10 @@ void checkHeader(const Block & header) static std::shared_ptr getFileMetadata( ReadBuffer & in, const FormatSettings & format_settings, - std::atomic & is_stopped, - ParquetMetadataInputFormat::Cache metadata_cache) + std::atomic & is_stopped) { - // in-memory cache is not implemented for local file operations, only for remote files - // there is a chance the user sets `input_format_parquet_use_metadata_cache=1` for a local file operation - // and the cache_key won't be set. Therefore, we also need to check for metadata_cache.key - if (!metadata_cache.use_cache || metadata_cache.key.empty()) - { - auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); - return parquet::ReadMetaData(arrow_file); - } - - auto [parquet_file_metadata, loaded] = ParquetFileMetaDataCache::instance()->getOrSet( - metadata_cache.key, - [&]() - { - auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); - return parquet::ReadMetaData(arrow_file); - } - ); - - if (loaded) - ProfileEvents::increment(ProfileEvents::ParquetMetaDataCacheMisses); - else - ProfileEvents::increment(ProfileEvents::ParquetMetaDataCacheHits); - - return parquet_file_metadata; - - + auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); + return parquet::ReadMetaData(arrow_file); } ParquetMetadataInputFormat::ParquetMetadataInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) @@ -187,7 +148,7 @@ Chunk ParquetMetadataInputFormat::read() if (done) return res; - auto metadata = getFileMetadata(*in, format_settings, is_stopped, metadata_cache); + auto metadata = getFileMetadata(*in, format_settings, is_stopped); const auto & header = getPort().getHeader(); auto names_and_types = getHeaderForParquetMetadata(); @@ -528,12 +489,6 @@ void ParquetMetadataInputFormat::resetParser() done = false; } -void ParquetMetadataInputFormat::setStorageRelatedUniqueKey(const Settings & settings, const String & key_) -{ - metadata_cache.key = key_; - metadata_cache.use_cache = settings[Setting::input_format_parquet_use_metadata_cache]; -} - ParquetMetadataSchemaReader::ParquetMetadataSchemaReader(ReadBuffer & in_) : ISchemaReader(in_) { diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h index 5cd3f4173bf5..5d2d89898596 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h @@ -62,14 +62,6 @@ class ParquetMetadataInputFormat : public IInputFormat void resetParser() override; - void setStorageRelatedUniqueKey(const Settings & settings, const String & key_) override; - - struct Cache - { - String key; - bool use_cache = false; - }; - private: Chunk read() override; @@ -86,8 +78,6 @@ class ParquetMetadataInputFormat : public IInputFormat const FormatSettings format_settings; bool done = false; std::atomic is_stopped{0}; - - Cache metadata_cache; }; class ParquetMetadataSchemaReader : public ISchemaReader diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 70ee134f980d..4e10bd12e40d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -517,9 +517,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade if (need_only_count) input_format->needOnlyCount(); - if (!object_info->getPath().empty()) - input_format->setStorageRelatedUniqueKey(context_->getSettingsRef(), object_info->getPath() + ":" + object_info->metadata->etag); - builder.init(Pipe(input_format)); if (auto transformer = configuration->getSchemaTransformer(context_, object_info->getPath())) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index b49e579898e6..268959bb444c 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -133,7 +133,6 @@ SYSTEM DROP PAGE CACHE ['SYSTEM DROP PAGE CACHE','DROP PAGE CACHE'] GLOBAL SYSTE SYSTEM DROP SCHEMA CACHE ['SYSTEM DROP SCHEMA CACHE','DROP SCHEMA CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP FORMAT SCHEMA CACHE ['SYSTEM DROP FORMAT SCHEMA CACHE','DROP FORMAT SCHEMA CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP S3 CLIENT CACHE ['SYSTEM DROP S3 CLIENT','DROP S3 CLIENT CACHE'] GLOBAL SYSTEM DROP CACHE -SYSTEM DROP PARQUET METADATA CACHE ['SYSTEM DROP PARQUET METADATA CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP CACHE ['DROP CACHE'] \N SYSTEM SYSTEM RELOAD CONFIG ['RELOAD CONFIG'] GLOBAL SYSTEM RELOAD SYSTEM RELOAD USERS ['RELOAD USERS'] GLOBAL SYSTEM RELOAD diff --git a/tests/queries/0_stateless/02995_settings_25_2_3.tsv b/tests/queries/0_stateless/02995_settings_25_2_3.tsv index ca227db9355a..e926ef502a2a 100644 --- a/tests/queries/0_stateless/02995_settings_25_2_3.tsv +++ b/tests/queries/0_stateless/02995_settings_25_2_3.tsv @@ -538,7 +538,6 @@ input_format_parquet_bloom_filter_push_down 0 input_format_parquet_case_insensitive_column_matching 0 input_format_parquet_enable_row_group_prefetch 1 input_format_parquet_filter_push_down 1 -input_format_parquet_use_metadata_cache 1 input_format_parquet_import_nested 0 input_format_parquet_local_file_min_bytes_for_seek 8192 input_format_parquet_max_block_size 65409 diff --git a/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.reference b/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.reference deleted file mode 100644 index c87ad9008b60..000000000000 --- a/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.reference +++ /dev/null @@ -1,8 +0,0 @@ -10 -10 -10 -10 -10 -10 -0 -10 diff --git a/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.sql b/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.sql deleted file mode 100644 index 2a1934e7c963..000000000000 --- a/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.sql +++ /dev/null @@ -1,61 +0,0 @@ --- Tags: no-parallel, no-fasttest - -DROP TABLE IF EXISTS t_parquet_03262; - -CREATE TABLE t_parquet_03262 (a UInt64) -ENGINE = S3(s3_conn, filename = 'test_03262_{_partition_id}', format = Parquet) -PARTITION BY a; - -INSERT INTO t_parquet_03262 SELECT number FROM numbers(10) SETTINGS s3_truncate_on_insert=1; - -SELECT COUNT(*) -FROM s3(s3_conn, filename = 'test_03262_*', format = Parquet) -SETTINGS input_format_parquet_use_metadata_cache=1, optimize_count_from_files=0; - -SELECT COUNT(*) -FROM s3(s3_conn, filename = 'test_03262_*', format = Parquet) -SETTINGS input_format_parquet_use_metadata_cache=1, optimize_count_from_files=0, log_comment='test_03262_parquet_metadata_cache'; - -SELECT COUNT(*) -FROM s3(s3_conn, filename = 'test_03262_*', format = ParquetMetadata) -SETTINGS input_format_parquet_use_metadata_cache=1, log_comment='test_03262_parquet_metadata_format_metadata_cache'; - -SYSTEM FLUSH LOGS; - -SELECT ProfileEvents['ParquetMetaDataCacheHits'] -FROM system.query_log -where log_comment = 'test_03262_parquet_metadata_cache' -AND type = 'QueryFinish' -ORDER BY event_time desc -LIMIT 1; - -SELECT ProfileEvents['ParquetMetaDataCacheHits'] -FROM system.query_log -where log_comment = 'test_03262_parquet_metadata_format_metadata_cache' -AND type = 'QueryFinish' -ORDER BY event_time desc -LIMIT 1; - -SYSTEM DROP PARQUET METADATA CACHE; - -SELECT COUNT(*) -FROM s3(s3_conn, filename = 'test_03262_*', format = Parquet) -SETTINGS input_format_parquet_use_metadata_cache=1, optimize_count_from_files=0, log_comment='test_03262_parquet_metadata_cache_cache_empty'; - -SYSTEM FLUSH LOGS; - -SELECT ProfileEvents['ParquetMetaDataCacheHits'] -FROM system.query_log -where log_comment = 'test_03262_parquet_metadata_cache_cache_empty' -AND type = 'QueryFinish' -ORDER BY event_time desc -LIMIT 1; - -SELECT ProfileEvents['ParquetMetaDataCacheMisses'] -FROM system.query_log -where log_comment = 'test_03262_parquet_metadata_cache_cache_empty' -AND type = 'QueryFinish' -ORDER BY event_time desc -LIMIT 1; - -DROP TABLE t_parquet_03262;