From 0dcb3133c7591d81a9901c879699fd836dca7e08 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Dec 2023 13:54:58 +0100 Subject: [PATCH 1/6] Allow to dynamically change fs cache size --- src/Interpreters/Cache/FileCache.cpp | 76 +++++++++++++----- src/Interpreters/Cache/FileCacheFactory.cpp | 32 +++++--- src/Interpreters/Cache/FileCacheFactory.h | 2 + src/Interpreters/Cache/IFileCachePriority.h | 10 ++- .../Cache/LRUFileCachePriority.cpp | 52 +++++++++++- src/Interpreters/Cache/LRUFileCachePriority.h | 2 + src/Interpreters/Cache/Metadata.cpp | 7 +- .../Cache/SLRUFileCachePriority.cpp | 22 +++++- .../Cache/SLRUFileCachePriority.h | 5 +- tests/config/config.d/storage_conf.xml | 11 +++ ...lly_change_filesystem_cache_size.reference | 20 +++++ ...ynamically_change_filesystem_cache_size.sh | 79 +++++++++++++++++++ 12 files changed, 272 insertions(+), 46 deletions(-) create mode 100644 tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference create mode 100755 tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index b6a3a0dadc5..0e799df1615 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -707,7 +707,7 @@ KeyMetadata::iterator FileCache::addFileSegment( stash_records.emplace( stash_key, stash->queue->add(locked_key.getKeyMetadata(), offset, 0, *lock)); - if (stash->queue->getElementsCount(*lock) > stash->queue->getElementsLimit()) + if (stash->queue->getElementsCount(*lock) > stash->queue->getElementsLimit(*lock)) stash->queue->pop(*lock); result_state = FileSegment::State::DETACHED; @@ -748,7 +748,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa LOG_TEST( log, "Trying to reserve space ({} bytes) for {}:{}, current usage {}/{}", size, file_segment.key(), file_segment.offset(), - main_priority->getSize(cache_lock), main_priority->getSizeLimit()); + main_priority->getSize(cache_lock), main_priority->getSizeLimit(cache_lock)); /// In case of per query cache limit (by default disabled), we add/remove entries from both /// (main_priority and query_priority) priority queues, but iterate entries in order of query_priority, @@ -760,7 +760,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa { query_priority = &query_context->getPriority(); - const bool query_limit_exceeded = query_priority->getSize(cache_lock) + size > query_priority->getSizeLimit(); + const bool query_limit_exceeded = query_priority->getSize(cache_lock) + size > query_priority->getSizeLimit(cache_lock); if (query_limit_exceeded && !query_context->recacheOnFileCacheQueryLimitExceeded()) { LOG_TEST(log, "Query limit exceeded, space reservation failed, " @@ -771,7 +771,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa LOG_TEST( log, "Using query limit, current usage: {}/{} (while reserving for {}:{})", - query_priority->getSize(cache_lock), query_priority->getSizeLimit(), + query_priority->getSize(cache_lock), query_priority->getSizeLimit(cache_lock), file_segment.key(), file_segment.offset()); } @@ -1057,9 +1057,11 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir) bool limits_satisfied; IFileCachePriority::IteratorPtr cache_it; + size_t size_limit = 0; { auto lock = lockCache(); + size_limit = main_priority->getSizeLimit(lock); limits_satisfied = main_priority->canFit(size, lock); if (limits_satisfied) @@ -1109,7 +1111,7 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir) log, "Cache capacity changed (max size: {}), " "cached file `{}` does not fit in cache anymore (size: {})", - main_priority->getSizeLimit(), offset_it->path().string(), size); + size_limit, offset_it->path().string(), size); fs::remove(offset_it->path()); } @@ -1215,7 +1217,8 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, std::lock_guard lock(apply_settings_mutex); - if (metadata.setBackgroundDownloadQueueSizeLimit(new_settings.background_download_queue_size_limit)) + if (new_settings.background_download_queue_size_limit != actual_settings.background_download_queue_size_limit + && metadata.setBackgroundDownloadQueueSizeLimit(new_settings.background_download_queue_size_limit)) { LOG_INFO(log, "Changed background_download_queue_size from {} to {}", actual_settings.background_download_queue_size_limit, @@ -1224,24 +1227,57 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, actual_settings.background_download_queue_size_limit = new_settings.background_download_queue_size_limit; } - bool updated; - try + if (new_settings.background_download_threads != actual_settings.background_download_threads) { - updated = metadata.setBackgroundDownloadThreads(new_settings.background_download_threads); - } - catch (...) - { - actual_settings.background_download_threads = metadata.getBackgroundDownloadThreads(); - throw; + bool updated = false; + try + { + updated = metadata.setBackgroundDownloadThreads(new_settings.background_download_threads); + } + catch (...) + { + actual_settings.background_download_threads = metadata.getBackgroundDownloadThreads(); + throw; + } + + if (updated) + { + LOG_INFO(log, "Changed background_download_threads from {} to {}", + actual_settings.background_download_threads, + new_settings.background_download_threads); + + actual_settings.background_download_threads = new_settings.background_download_threads; + } } - if (updated) - { - LOG_INFO(log, "Changed background_download_threads from {} to {}", - actual_settings.background_download_threads, - new_settings.background_download_threads); - actual_settings.background_download_threads = new_settings.background_download_threads; + if (new_settings.max_size != actual_settings.max_size + || new_settings.max_elements != actual_settings.max_elements) + { + auto cache_lock = lockCache(); + + bool updated = false; + try + { + updated = main_priority->modifySizeLimits( + new_settings.max_size, new_settings.max_elements, new_settings.slru_size_ratio, cache_lock); + } + catch (...) + { + actual_settings.max_size = main_priority->getSizeLimit(cache_lock); + actual_settings.max_elements = main_priority->getElementsLimit(cache_lock); + throw; + } + + if (updated) + { + LOG_INFO(log, "Changed max_size from {} to {}, max_elements from {} to {}", + actual_settings.max_size, new_settings.max_size, + actual_settings.max_elements, new_settings.max_elements); + + actual_settings.max_size = main_priority->getSizeLimit(cache_lock); + actual_settings.max_elements = main_priority->getElementsLimit(cache_lock); + } } } diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index 9ba96de26dc..84eafde9afd 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -25,6 +25,12 @@ FileCacheSettings FileCacheFactory::FileCacheData::getSettings() const return settings; } +void FileCacheFactory::FileCacheData::setSettings(const FileCacheSettings & new_settings) +{ + std::lock_guard lock(settings_mutex); + settings = new_settings; +} + FileCacheFactory & FileCacheFactory::instance() { static FileCacheFactory ret; @@ -100,21 +106,23 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig FileCacheSettings new_settings; new_settings.loadFromConfig(config, cache_info->config_path); - FileCacheSettings old_settings; - { - std::lock_guard lock(cache_info->settings_mutex); - if (new_settings == cache_info->settings) - continue; + FileCacheSettings old_settings = cache_info->getSettings(); + if (old_settings == new_settings) + continue; - old_settings = cache_info->settings; + try + { + cache_info->cache->applySettingsIfPossible(new_settings, old_settings); + } + catch (...) + { + /// Settings changes could be partially applied in case of exception, + /// make sure cache_info->settings show correct state of applied settings. + cache_info->setSettings(old_settings); + throw; } - cache_info->cache->applySettingsIfPossible(new_settings, old_settings); - - { - std::lock_guard lock(cache_info->settings_mutex); - cache_info->settings = old_settings; - } + cache_info->setSettings(old_settings); } } diff --git a/src/Interpreters/Cache/FileCacheFactory.h b/src/Interpreters/Cache/FileCacheFactory.h index 2148e520fd1..c60b247005b 100644 --- a/src/Interpreters/Cache/FileCacheFactory.h +++ b/src/Interpreters/Cache/FileCacheFactory.h @@ -24,6 +24,8 @@ public: FileCacheSettings getSettings() const; + void setSettings(const FileCacheSettings & new_settings); + const FileCachePtr cache; const std::string config_path; diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 0f407a3082c..52de36849ae 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -55,9 +55,9 @@ public: virtual ~IFileCachePriority() = default; - size_t getElementsLimit() const { return max_elements; } + size_t getElementsLimit(const CacheGuard::Lock &) const { return max_elements; } - size_t getSizeLimit() const { return max_size; } + size_t getSizeLimit(const CacheGuard::Lock &) const { return max_size; } virtual size_t getSize(const CacheGuard::Lock &) const = 0; @@ -86,9 +86,11 @@ public: FinalizeEvictionFunc & finalize_eviction_func, const CacheGuard::Lock &) = 0; + virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) = 0; + protected: - const size_t max_size = 0; - const size_t max_elements = 0; + size_t max_size = 0; + size_t max_elements = 0; }; } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index a6abaea11c3..ba9d624ca9b 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -16,6 +16,9 @@ namespace ProfileEvents { extern const Event FilesystemCacheEvictionSkippedFileSegments; extern const Event FilesystemCacheEvictionTries; + extern const Event FilesystemCacheEvictMicroseconds; + extern const Event FilesystemCacheEvictedBytes; + extern const Event FilesystemCacheEvictedFileSegments; } namespace DB @@ -36,7 +39,7 @@ IFileCachePriority::IteratorPtr LRUFileCachePriority::add( /// NOLINT return std::make_shared(add(Entry(key_metadata->key, offset, size, key_metadata), lock)); } -LRUFileCachePriority::LRUIterator LRUFileCachePriority::add(Entry && entry, const CacheGuard::Lock &) +LRUFileCachePriority::LRUIterator LRUFileCachePriority::add(Entry && entry, const CacheGuard::Lock & lock) { if (entry.size == 0) { @@ -59,7 +62,7 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::add(Entry && entry, cons } #endif - const auto & size_limit = getSizeLimit(); + const auto & size_limit = getSizeLimit(lock); if (size_limit && current_size + entry.size > size_limit) { throw Exception( @@ -288,6 +291,51 @@ std::vector LRUFileCachePriority::dump(FileCache & cache, const return res; } +bool LRUFileCachePriority::modifySizeLimits( + size_t max_size_, size_t max_elements_, double /* size_ratio_ */, const CacheGuard::Lock & lock) +{ + if (max_size == max_size_ && max_elements == max_elements_) + return false; /// Nothing to change. + + auto check_limits_satisfied = [&]() + { + return (max_size_ == 0 || current_size <= max_size_) + && (max_elements_ == 0 || current_elements_num <= max_elements_); + }; + + if (check_limits_satisfied()) + { + max_size = max_size_; + max_elements = max_elements_; + return true; + } + + auto iterate_func = [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) + { + chassert(segment_metadata->file_segment->assertCorrectness()); + + if (!segment_metadata->releasable()) + return IterationResult::CONTINUE; + + auto segment = segment_metadata->file_segment; + locked_key.removeFileSegment(segment->offset(), segment->lock()); + + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->getDownloadedSize()); + return IterationResult::REMOVE_AND_CONTINUE; + }; + + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds); + iterate( + [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) + { return check_limits_satisfied() ? IterationResult::BREAK : iterate_func(locked_key, segment_metadata); }, + lock); + + max_size = max_size_; + max_elements = max_elements_; + return true; +} + void LRUFileCachePriority::LRUIterator::remove(const CacheGuard::Lock & lock) { assertValid(); diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 5ff6c61eb4d..2450ecbff39 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -48,6 +48,8 @@ public: void pop(const CacheGuard::Lock & lock) { remove(queue.begin(), lock); } + bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override; + private: void updateElementsCount(int64_t num); void updateSize(int64_t size); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 08c6151f1cb..f4aa7cc7ab1 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -681,7 +681,7 @@ void CacheMetadata::startup() download_threads.emplace_back(std::make_shared()); download_threads.back()->thread = std::make_unique([this, thread = download_threads.back()] { downloadThreadFunc(thread->stop_flag); }); } - cleanup_thread = std::make_unique(std::function{ [this]{ cleanupThreadFunc(); }}); + cleanup_thread = std::make_unique([this]{ cleanupThreadFunc(); }); } void CacheMetadata::shutdown() @@ -708,10 +708,10 @@ bool CacheMetadata::setBackgroundDownloadThreads(size_t threads_num) if (threads_num == download_threads_num) return false; + SCOPE_EXIT({ download_threads_num = download_threads.size(); }); + if (threads_num > download_threads_num) { - SCOPE_EXIT({ download_threads_num = download_threads.size(); }); - size_t add_threads = threads_num - download_threads_num; for (size_t i = 0; i < add_threads; ++i) { @@ -739,7 +739,6 @@ bool CacheMetadata::setBackgroundDownloadThreads(size_t threads_num) } download_queue->cv.notify_all(); - SCOPE_EXIT({ download_threads_num = download_threads.size(); }); for (size_t i = 0; i < remove_threads; ++i) { diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 71b8d44d438..56af33b98f3 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -21,14 +21,15 @@ namespace SLRUFileCachePriority::SLRUFileCachePriority( size_t max_size_, size_t max_elements_, - double size_ratio) + double size_ratio_) : IFileCachePriority(max_size_, max_elements_) + , size_ratio(size_ratio_) , protected_queue(LRUFileCachePriority(getRatio(max_size_, size_ratio), getRatio(max_elements_, size_ratio))) , probationary_queue(LRUFileCachePriority(getRatio(max_size_, 1 - size_ratio), getRatio(max_elements_, 1 - size_ratio))) { LOG_DEBUG( log, "Using probationary queue size: {}, protected queue size: {}", - probationary_queue.getSizeLimit(), protected_queue.getSizeLimit()); + probationary_queue.max_size, protected_queue.max_elements); } size_t SLRUFileCachePriority::getSize(const CacheGuard::Lock & lock) const @@ -151,7 +152,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach /// Entry is in probationary queue. /// We need to move it to protected queue. const size_t size = iterator.getEntry().size; - if (size > protected_queue.getSizeLimit()) + if (size > protected_queue.getSizeLimit(lock)) { /// Entry size is bigger than the whole protected queue limit. /// This is only possible if protected_queue_size_limit is less than max_file_segment_size, @@ -235,6 +236,21 @@ void SLRUFileCachePriority::shuffle(const CacheGuard::Lock & lock) probationary_queue.shuffle(lock); } +bool SLRUFileCachePriority::modifySizeLimits( + size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock & lock) +{ + if (max_size == max_size_ && max_elements == max_elements_ && size_ratio == size_ratio_) + return false; /// Nothing to change. + + protected_queue.modifySizeLimits(getRatio(max_size_, size_ratio_), getRatio(max_elements_, size_ratio_), 0, lock); + probationary_queue.modifySizeLimits(getRatio(max_size_, 1 - size_ratio_), getRatio(max_elements_, 1 - size_ratio_), 0, lock); + + max_size = max_size_; + max_elements = max_elements_; + size_ratio = size_ratio_; + return true; +} + SLRUFileCachePriority::SLRUIterator::SLRUIterator( SLRUFileCachePriority * cache_priority_, LRUFileCachePriority::LRUIterator && lru_iterator_, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 45fc7ad8333..373e37a8fc9 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -18,7 +18,7 @@ private: public: class SLRUIterator; - SLRUFileCachePriority(size_t max_size_, size_t max_elements_, double size_ratio); + SLRUFileCachePriority(size_t max_size_, size_t max_elements_, double size_ratio_); size_t getSize(const CacheGuard::Lock & lock) const override; @@ -45,7 +45,10 @@ public: std::vector dump(FileCache & cache, const CacheGuard::Lock &) override; + bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override; + private: + double size_ratio; LRUFileCachePriority protected_queue; LRUFileCachePriority probationary_queue; Poco::Logger * log = &Poco::Logger::get("SLRUFileCachePriority"); diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 18652826d83..72248f668a7 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -29,6 +29,17 @@ 0 0 + + cache + s3disk + s3_cache_02944/ + 100 + 100 + 10 + 10 + 100 + 0 + local_blob_storage diff --git a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference new file mode 100644 index 00000000000..7fa32ec2b09 --- /dev/null +++ b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference @@ -0,0 +1,20 @@ +100 10 10 10 0 0 98 10 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +0 +10 +98 +set max_size from 100 to 10 +10 10 10 10 0 0 8 1 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +1 +8 +set max_size from 10 to 100 +100 10 10 10 0 0 8 1 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +10 +98 +set max_elements from 10 to 2 +100 2 10 10 0 0 18 2 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +2 +18 +set max_elements from 2 to 10 +100 10 10 10 0 0 18 2 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +10 +98 diff --git a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh new file mode 100755 index 00000000000..021493eaa82 --- /dev/null +++ b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh @@ -0,0 +1,79 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-s3-storage + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +disk_name="s3_cache_02944" + +$CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" + +$CLICKHOUSE_CLIENT -nm --query " +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String) engine=MergeTree() ORDER BY tuple() SETTINGS disk = '$disk_name'; +INSERT INTO test SELECT randomString(100); +SYSTEM DROP FILESYSTEM CACHE; +" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" + +$CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" +$CLICKHOUSE_CLIENT --query "SELECT sum(size) FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" + +config_path=/etc/clickhouse-server/config.d/storage_conf.xml +config_path_tmp=$config_path.tmp + +echo 'set max_size from 100 to 10' +cat $config_path \ +| sed "s|100<\/max_size>|10<\/max_size>|" \ +> $config_path_tmp +mv $config_path_tmp $config_path + +$CLICKHOUSE_CLIENT -nm --query "SYSTEM RELOAD CONFIG" +$CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" +$CLICKHOUSE_CLIENT --query "SELECT sum(size) FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" + +echo 'set max_size from 10 to 100' +cat $config_path \ +| sed "s|10<\/max_size>|100<\/max_size>|" \ +> $config_path_tmp +mv $config_path_tmp $config_path + +$CLICKHOUSE_CLIENT -nm --query "SYSTEM RELOAD CONFIG" +$CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" + +$CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" +$CLICKHOUSE_CLIENT --query "SELECT sum(size) FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" + +echo 'set max_elements from 10 to 2' +cat $config_path \ +| sed "s|10<\/max_elements>|2<\/max_elements>|" \ +> $config_path_tmp +mv $config_path_tmp $config_path + +$CLICKHOUSE_CLIENT -nm --query "SYSTEM RELOAD CONFIG" +$CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" +$CLICKHOUSE_CLIENT --query "SELECT sum(size) FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" + +echo 'set max_elements from 2 to 10' +cat $config_path \ +| sed "s|2<\/max_elements>|10<\/max_elements>|" \ +> $config_path_tmp +mv $config_path_tmp $config_path + +$CLICKHOUSE_CLIENT -nm --query "SYSTEM RELOAD CONFIG" +$CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" + +$CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" +$CLICKHOUSE_CLIENT --query "SELECT sum(size) FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" From e05cc51ef0a5ad61879919e37a5c9970f1e444f7 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 15 Dec 2023 16:17:30 +0100 Subject: [PATCH 2/6] Update storage_conf.xml --- tests/config/config.d/storage_conf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 72248f668a7..7a5caad9139 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -31,7 +31,7 @@ cache - s3disk + s3_disk s3_cache_02944/ 100 100 From 44f1644622a126ebafc1473f18dfefac00d9fd97 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 18 Dec 2023 11:01:32 +0100 Subject: [PATCH 3/6] Update 02944_dynamically_change_filesystem_cache_size.reference --- ..._dynamically_change_filesystem_cache_size.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference index 7fa32ec2b09..cd155b6ca29 100644 --- a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference +++ b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference @@ -1,20 +1,20 @@ -100 10 10 10 0 0 98 10 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +100 10 10 10 0 0 98 10 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 0 10 98 set max_size from 100 to 10 -10 10 10 10 0 0 8 1 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +10 10 10 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 1 8 set max_size from 10 to 100 -100 10 10 10 0 0 8 1 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +100 10 10 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 10 98 set max_elements from 10 to 2 -100 2 10 10 0 0 18 2 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +100 2 10 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 2 18 set max_elements from 2 to 10 -100 10 10 10 0 0 18 2 /var/lib/clickhouse/caches/s3_cache_02944/ 5 5000 0 1 +100 10 10 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 10 98 From 566aeaa17aaf247f973fb4516ba7c46c3460624e Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 18 Dec 2023 17:23:41 +0100 Subject: [PATCH 4/6] Better config --- tests/config/config.d/storage_conf.xml | 11 --------- tests/config/config.d/storage_conf_02944.xml | 25 ++++++++++++++++++++ tests/config/install.sh | 1 + 3 files changed, 26 insertions(+), 11 deletions(-) create mode 100644 tests/config/config.d/storage_conf_02944.xml diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 7a5caad9139..18652826d83 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -29,17 +29,6 @@ 0 0 - - cache - s3_disk - s3_cache_02944/ - 100 - 100 - 10 - 10 - 100 - 0 - local_blob_storage diff --git a/tests/config/config.d/storage_conf_02944.xml b/tests/config/config.d/storage_conf_02944.xml new file mode 100644 index 00000000000..5f45640a923 --- /dev/null +++ b/tests/config/config.d/storage_conf_02944.xml @@ -0,0 +1,25 @@ + + + + + s3 + s3_disk/ + http://localhost:11111/test/test_02944/ + clickhouse + clickhouse + 20000 + + + cache + s3_disk + s3_cache_02944/ + 100 + 10 + 10 + 10 + 100 + 0 + + + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 6046f05c922..2f9fd44c9b0 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -156,6 +156,7 @@ if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then fi ln -sf $SRC_PATH/config.d/storage_conf.xml $DEST_SERVER_PATH/config.d/ + ln -sf $SRC_PATH/config.d/storage_conf_02944.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/s3_cache.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/s3_cache_new.xml $DEST_SERVER_PATH/users.d/ fi From 0a45ffef3090a1245a0f9853646edd33c1bcd230 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 18 Dec 2023 19:14:45 +0100 Subject: [PATCH 5/6] Update 02944_dynamically_change_filesystem_cache_size.sh --- .../02944_dynamically_change_filesystem_cache_size.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh index 021493eaa82..e47e13a7e40 100755 --- a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh +++ b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" $CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" $CLICKHOUSE_CLIENT --query "SELECT sum(size) FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" -config_path=/etc/clickhouse-server/config.d/storage_conf.xml +config_path=/etc/clickhouse-server/config.d/storage_conf_02944.xml config_path_tmp=$config_path.tmp echo 'set max_size from 100 to 10' From 06a2e86983fe46c203d20878f23b8e86d3f3e41c Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Dec 2023 10:02:45 +0100 Subject: [PATCH 6/6] Fix test --- ...lly_change_filesystem_cache_size.reference | 2 +- ...ynamically_change_filesystem_cache_size.sh | 19 ++++++++++++++----- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference index cd155b6ca29..8620171cb99 100644 --- a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference +++ b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference @@ -1,4 +1,4 @@ -100 10 10 10 0 0 98 10 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 +100 10 10 10 0 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 0 10 98 diff --git a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh index e47e13a7e40..2e344a6b6e5 100755 --- a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh +++ b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-s3-storage +# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -7,6 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) disk_name="s3_cache_02944" +$CLICKHOUSE_CLIENT --query "SYSTEM DROP FILESYSTEM CACHE" $CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" $CLICKHOUSE_CLIENT -nm --query " @@ -32,7 +33,9 @@ cat $config_path \ > $config_path_tmp mv $config_path_tmp $config_path -$CLICKHOUSE_CLIENT -nm --query "SYSTEM RELOAD CONFIG" +$CLICKHOUSE_CLIENT -nm --query " +set send_logs_level='fatal'; +SYSTEM RELOAD CONFIG" $CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" $CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" @@ -44,7 +47,9 @@ cat $config_path \ > $config_path_tmp mv $config_path_tmp $config_path -$CLICKHOUSE_CLIENT -nm --query "SYSTEM RELOAD CONFIG" +$CLICKHOUSE_CLIENT -nm --query " +set send_logs_level='fatal'; +SYSTEM RELOAD CONFIG" $CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" $CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" @@ -58,7 +63,9 @@ cat $config_path \ > $config_path_tmp mv $config_path_tmp $config_path -$CLICKHOUSE_CLIENT -nm --query "SYSTEM RELOAD CONFIG" +$CLICKHOUSE_CLIENT -nm --query " +set send_logs_level='fatal'; +SYSTEM RELOAD CONFIG" $CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" $CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" @@ -70,7 +77,9 @@ cat $config_path \ > $config_path_tmp mv $config_path_tmp $config_path -$CLICKHOUSE_CLIENT -nm --query "SYSTEM RELOAD CONFIG" +$CLICKHOUSE_CLIENT -nm --query " +set send_logs_level='fatal'; +SYSTEM RELOAD CONFIG" $CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" $CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null"