mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 04:12:19 +00:00
Merge pull request #57897 from ClickHouse/allow-to-dynamically-change-fs-cache-size
Support dynamic reloading of filesystem cache size
This commit is contained in:
commit
d4e71f96c1
@ -707,7 +707,7 @@ KeyMetadata::iterator FileCache::addFileSegment(
|
|||||||
stash_records.emplace(
|
stash_records.emplace(
|
||||||
stash_key, stash->queue->add(locked_key.getKeyMetadata(), offset, 0, *lock));
|
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);
|
stash->queue->pop(*lock);
|
||||||
|
|
||||||
result_state = FileSegment::State::DETACHED;
|
result_state = FileSegment::State::DETACHED;
|
||||||
@ -748,7 +748,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
|
|||||||
LOG_TEST(
|
LOG_TEST(
|
||||||
log, "Trying to reserve space ({} bytes) for {}:{}, current usage {}/{}",
|
log, "Trying to reserve space ({} bytes) for {}:{}, current usage {}/{}",
|
||||||
size, file_segment.key(), file_segment.offset(),
|
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
|
/// 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,
|
/// (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();
|
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())
|
if (query_limit_exceeded && !query_context->recacheOnFileCacheQueryLimitExceeded())
|
||||||
{
|
{
|
||||||
LOG_TEST(log, "Query limit exceeded, space reservation failed, "
|
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_TEST(
|
||||||
log, "Using query limit, current usage: {}/{} (while reserving for {}:{})",
|
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());
|
file_segment.key(), file_segment.offset());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1066,9 +1066,11 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
|
|||||||
|
|
||||||
bool limits_satisfied;
|
bool limits_satisfied;
|
||||||
IFileCachePriority::IteratorPtr cache_it;
|
IFileCachePriority::IteratorPtr cache_it;
|
||||||
|
size_t size_limit = 0;
|
||||||
|
|
||||||
{
|
{
|
||||||
auto lock = lockCache();
|
auto lock = lockCache();
|
||||||
|
size_limit = main_priority->getSizeLimit(lock);
|
||||||
|
|
||||||
limits_satisfied = main_priority->canFit(size, lock);
|
limits_satisfied = main_priority->canFit(size, lock);
|
||||||
if (limits_satisfied)
|
if (limits_satisfied)
|
||||||
@ -1118,7 +1120,7 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
|
|||||||
log,
|
log,
|
||||||
"Cache capacity changed (max size: {}), "
|
"Cache capacity changed (max size: {}), "
|
||||||
"cached file `{}` does not fit in cache anymore (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());
|
fs::remove(offset_it->path());
|
||||||
}
|
}
|
||||||
@ -1222,7 +1224,8 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings,
|
|||||||
|
|
||||||
std::lock_guard lock(apply_settings_mutex);
|
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 {}",
|
LOG_INFO(log, "Changed background_download_queue_size from {} to {}",
|
||||||
actual_settings.background_download_queue_size_limit,
|
actual_settings.background_download_queue_size_limit,
|
||||||
@ -1231,7 +1234,9 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings,
|
|||||||
actual_settings.background_download_queue_size_limit = new_settings.background_download_queue_size_limit;
|
actual_settings.background_download_queue_size_limit = new_settings.background_download_queue_size_limit;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool updated;
|
if (new_settings.background_download_threads != actual_settings.background_download_threads)
|
||||||
|
{
|
||||||
|
bool updated = false;
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
updated = metadata.setBackgroundDownloadThreads(new_settings.background_download_threads);
|
updated = metadata.setBackgroundDownloadThreads(new_settings.background_download_threads);
|
||||||
@ -1252,6 +1257,37 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
FileCache::QueryContextHolder::QueryContextHolder(
|
FileCache::QueryContextHolder::QueryContextHolder(
|
||||||
const String & query_id_,
|
const String & query_id_,
|
||||||
FileCache * cache_,
|
FileCache * cache_,
|
||||||
|
@ -25,6 +25,12 @@ FileCacheSettings FileCacheFactory::FileCacheData::getSettings() const
|
|||||||
return settings;
|
return settings;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void FileCacheFactory::FileCacheData::setSettings(const FileCacheSettings & new_settings)
|
||||||
|
{
|
||||||
|
std::lock_guard lock(settings_mutex);
|
||||||
|
settings = new_settings;
|
||||||
|
}
|
||||||
|
|
||||||
FileCacheFactory & FileCacheFactory::instance()
|
FileCacheFactory & FileCacheFactory::instance()
|
||||||
{
|
{
|
||||||
static FileCacheFactory ret;
|
static FileCacheFactory ret;
|
||||||
@ -100,21 +106,23 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig
|
|||||||
FileCacheSettings new_settings;
|
FileCacheSettings new_settings;
|
||||||
new_settings.loadFromConfig(config, cache_info->config_path);
|
new_settings.loadFromConfig(config, cache_info->config_path);
|
||||||
|
|
||||||
FileCacheSettings old_settings;
|
FileCacheSettings old_settings = cache_info->getSettings();
|
||||||
{
|
if (old_settings == new_settings)
|
||||||
std::lock_guard lock(cache_info->settings_mutex);
|
|
||||||
if (new_settings == cache_info->settings)
|
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
old_settings = cache_info->settings;
|
try
|
||||||
}
|
|
||||||
|
|
||||||
cache_info->cache->applySettingsIfPossible(new_settings, old_settings);
|
|
||||||
|
|
||||||
{
|
{
|
||||||
std::lock_guard lock(cache_info->settings_mutex);
|
cache_info->cache->applySettingsIfPossible(new_settings, old_settings);
|
||||||
cache_info->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->setSettings(old_settings);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -24,6 +24,8 @@ public:
|
|||||||
|
|
||||||
FileCacheSettings getSettings() const;
|
FileCacheSettings getSettings() const;
|
||||||
|
|
||||||
|
void setSettings(const FileCacheSettings & new_settings);
|
||||||
|
|
||||||
const FileCachePtr cache;
|
const FileCachePtr cache;
|
||||||
const std::string config_path;
|
const std::string config_path;
|
||||||
|
|
||||||
|
@ -55,9 +55,9 @@ public:
|
|||||||
|
|
||||||
virtual ~IFileCachePriority() = default;
|
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;
|
virtual size_t getSize(const CacheGuard::Lock &) const = 0;
|
||||||
|
|
||||||
@ -86,9 +86,11 @@ public:
|
|||||||
FinalizeEvictionFunc & finalize_eviction_func,
|
FinalizeEvictionFunc & finalize_eviction_func,
|
||||||
const CacheGuard::Lock &) = 0;
|
const CacheGuard::Lock &) = 0;
|
||||||
|
|
||||||
|
virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) = 0;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
const size_t max_size = 0;
|
size_t max_size = 0;
|
||||||
const size_t max_elements = 0;
|
size_t max_elements = 0;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -16,6 +16,9 @@ namespace ProfileEvents
|
|||||||
{
|
{
|
||||||
extern const Event FilesystemCacheEvictionSkippedFileSegments;
|
extern const Event FilesystemCacheEvictionSkippedFileSegments;
|
||||||
extern const Event FilesystemCacheEvictionTries;
|
extern const Event FilesystemCacheEvictionTries;
|
||||||
|
extern const Event FilesystemCacheEvictMicroseconds;
|
||||||
|
extern const Event FilesystemCacheEvictedBytes;
|
||||||
|
extern const Event FilesystemCacheEvictedFileSegments;
|
||||||
}
|
}
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -36,7 +39,7 @@ IFileCachePriority::IteratorPtr LRUFileCachePriority::add( /// NOLINT
|
|||||||
return std::make_shared<LRUIterator>(add(Entry(key_metadata->key, offset, size, key_metadata), lock));
|
return std::make_shared<LRUIterator>(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)
|
if (entry.size == 0)
|
||||||
{
|
{
|
||||||
@ -59,7 +62,7 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::add(Entry && entry, cons
|
|||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
const auto & size_limit = getSizeLimit();
|
const auto & size_limit = getSizeLimit(lock);
|
||||||
if (size_limit && current_size + entry.size > size_limit)
|
if (size_limit && current_size + entry.size > size_limit)
|
||||||
{
|
{
|
||||||
throw Exception(
|
throw Exception(
|
||||||
@ -288,6 +291,51 @@ std::vector<FileSegmentInfo> LRUFileCachePriority::dump(const CacheGuard::Lock &
|
|||||||
return res;
|
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)
|
void LRUFileCachePriority::LRUIterator::remove(const CacheGuard::Lock & lock)
|
||||||
{
|
{
|
||||||
assertValid();
|
assertValid();
|
||||||
|
@ -48,6 +48,8 @@ public:
|
|||||||
|
|
||||||
void pop(const CacheGuard::Lock & lock) { remove(queue.begin(), lock); }
|
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:
|
private:
|
||||||
void updateElementsCount(int64_t num);
|
void updateElementsCount(int64_t num);
|
||||||
void updateSize(int64_t size);
|
void updateSize(int64_t size);
|
||||||
|
@ -687,7 +687,7 @@ void CacheMetadata::startup()
|
|||||||
download_threads.emplace_back(std::make_shared<DownloadThread>());
|
download_threads.emplace_back(std::make_shared<DownloadThread>());
|
||||||
download_threads.back()->thread = std::make_unique<ThreadFromGlobalPool>([this, thread = download_threads.back()] { downloadThreadFunc(thread->stop_flag); });
|
download_threads.back()->thread = std::make_unique<ThreadFromGlobalPool>([this, thread = download_threads.back()] { downloadThreadFunc(thread->stop_flag); });
|
||||||
}
|
}
|
||||||
cleanup_thread = std::make_unique<ThreadFromGlobalPool>(std::function{ [this]{ cleanupThreadFunc(); }});
|
cleanup_thread = std::make_unique<ThreadFromGlobalPool>([this]{ cleanupThreadFunc(); });
|
||||||
}
|
}
|
||||||
|
|
||||||
void CacheMetadata::shutdown()
|
void CacheMetadata::shutdown()
|
||||||
@ -714,10 +714,10 @@ bool CacheMetadata::setBackgroundDownloadThreads(size_t threads_num)
|
|||||||
if (threads_num == download_threads_num)
|
if (threads_num == download_threads_num)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (threads_num > download_threads_num)
|
|
||||||
{
|
|
||||||
SCOPE_EXIT({ download_threads_num = download_threads.size(); });
|
SCOPE_EXIT({ download_threads_num = download_threads.size(); });
|
||||||
|
|
||||||
|
if (threads_num > download_threads_num)
|
||||||
|
{
|
||||||
size_t add_threads = threads_num - download_threads_num;
|
size_t add_threads = threads_num - download_threads_num;
|
||||||
for (size_t i = 0; i < add_threads; ++i)
|
for (size_t i = 0; i < add_threads; ++i)
|
||||||
{
|
{
|
||||||
@ -745,7 +745,6 @@ bool CacheMetadata::setBackgroundDownloadThreads(size_t threads_num)
|
|||||||
}
|
}
|
||||||
|
|
||||||
download_queue->cv.notify_all();
|
download_queue->cv.notify_all();
|
||||||
SCOPE_EXIT({ download_threads_num = download_threads.size(); });
|
|
||||||
|
|
||||||
for (size_t i = 0; i < remove_threads; ++i)
|
for (size_t i = 0; i < remove_threads; ++i)
|
||||||
{
|
{
|
||||||
|
@ -21,14 +21,15 @@ namespace
|
|||||||
SLRUFileCachePriority::SLRUFileCachePriority(
|
SLRUFileCachePriority::SLRUFileCachePriority(
|
||||||
size_t max_size_,
|
size_t max_size_,
|
||||||
size_t max_elements_,
|
size_t max_elements_,
|
||||||
double size_ratio)
|
double size_ratio_)
|
||||||
: IFileCachePriority(max_size_, max_elements_)
|
: IFileCachePriority(max_size_, max_elements_)
|
||||||
|
, size_ratio(size_ratio_)
|
||||||
, protected_queue(LRUFileCachePriority(getRatio(max_size_, size_ratio), getRatio(max_elements_, 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)))
|
, probationary_queue(LRUFileCachePriority(getRatio(max_size_, 1 - size_ratio), getRatio(max_elements_, 1 - size_ratio)))
|
||||||
{
|
{
|
||||||
LOG_DEBUG(
|
LOG_DEBUG(
|
||||||
log, "Using probationary queue size: {}, protected queue size: {}",
|
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
|
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.
|
/// Entry is in probationary queue.
|
||||||
/// We need to move it to protected queue.
|
/// We need to move it to protected queue.
|
||||||
const size_t size = iterator.getEntry().size;
|
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.
|
/// 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,
|
/// 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);
|
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::SLRUIterator::SLRUIterator(
|
||||||
SLRUFileCachePriority * cache_priority_,
|
SLRUFileCachePriority * cache_priority_,
|
||||||
LRUFileCachePriority::LRUIterator && lru_iterator_,
|
LRUFileCachePriority::LRUIterator && lru_iterator_,
|
||||||
|
@ -18,7 +18,7 @@ private:
|
|||||||
public:
|
public:
|
||||||
class SLRUIterator;
|
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;
|
size_t getSize(const CacheGuard::Lock & lock) const override;
|
||||||
|
|
||||||
@ -45,7 +45,10 @@ public:
|
|||||||
|
|
||||||
std::vector<FileSegmentInfo> dump(const CacheGuard::Lock &) override;
|
std::vector<FileSegmentInfo> dump(const CacheGuard::Lock &) override;
|
||||||
|
|
||||||
|
bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
double size_ratio;
|
||||||
LRUFileCachePriority protected_queue;
|
LRUFileCachePriority protected_queue;
|
||||||
LRUFileCachePriority probationary_queue;
|
LRUFileCachePriority probationary_queue;
|
||||||
Poco::Logger * log = &Poco::Logger::get("SLRUFileCachePriority");
|
Poco::Logger * log = &Poco::Logger::get("SLRUFileCachePriority");
|
||||||
|
25
tests/config/config.d/storage_conf_02944.xml
Normal file
25
tests/config/config.d/storage_conf_02944.xml
Normal file
@ -0,0 +1,25 @@
|
|||||||
|
<clickhouse>
|
||||||
|
<storage_configuration>
|
||||||
|
<disks>
|
||||||
|
<s3_disk_02944>
|
||||||
|
<type>s3</type>
|
||||||
|
<path>s3_disk/</path>
|
||||||
|
<endpoint>http://localhost:11111/test/test_02944/</endpoint>
|
||||||
|
<access_key_id>clickhouse</access_key_id>
|
||||||
|
<secret_access_key>clickhouse</secret_access_key>
|
||||||
|
<request_timeout_ms>20000</request_timeout_ms>
|
||||||
|
</s3_disk_02944>
|
||||||
|
<s3_cache_02944>
|
||||||
|
<type>cache</type>
|
||||||
|
<disk>s3_disk</disk>
|
||||||
|
<path>s3_cache_02944/</path>
|
||||||
|
<max_size>100</max_size>
|
||||||
|
<max_elements>10</max_elements>
|
||||||
|
<max_file_segment_size>10</max_file_segment_size>
|
||||||
|
<boundary_alignment>10</boundary_alignment>
|
||||||
|
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||||
|
<cache_on_write_operations>0</cache_on_write_operations>
|
||||||
|
</s3_cache_02944>
|
||||||
|
</disks>
|
||||||
|
</storage_configuration>
|
||||||
|
</clickhouse>
|
@ -156,6 +156,7 @@ if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then
|
|||||||
fi
|
fi
|
||||||
|
|
||||||
ln -sf $SRC_PATH/config.d/storage_conf.xml $DEST_SERVER_PATH/config.d/
|
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.xml $DEST_SERVER_PATH/users.d/
|
||||||
ln -sf $SRC_PATH/users.d/s3_cache_new.xml $DEST_SERVER_PATH/users.d/
|
ln -sf $SRC_PATH/users.d/s3_cache_new.xml $DEST_SERVER_PATH/users.d/
|
||||||
fi
|
fi
|
||||||
|
@ -0,0 +1,20 @@
|
|||||||
|
100 10 10 10 0 0 0 0 /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/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/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/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/filesystem_caches/s3_cache_02944/ 5 5000 0 1
|
||||||
|
10
|
||||||
|
98
|
88
tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh
Executable file
88
tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh
Executable file
@ -0,0 +1,88 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings
|
||||||
|
|
||||||
|
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 "SYSTEM DROP FILESYSTEM CACHE"
|
||||||
|
$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_02944.xml
|
||||||
|
config_path_tmp=$config_path.tmp
|
||||||
|
|
||||||
|
echo 'set max_size from 100 to 10'
|
||||||
|
cat $config_path \
|
||||||
|
| sed "s|<max_size>100<\/max_size>|<max_size>10<\/max_size>|" \
|
||||||
|
> $config_path_tmp
|
||||||
|
mv $config_path_tmp $config_path
|
||||||
|
|
||||||
|
$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'"
|
||||||
|
$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|<max_size>10<\/max_size>|<max_size>100<\/max_size>|" \
|
||||||
|
> $config_path_tmp
|
||||||
|
mv $config_path_tmp $config_path
|
||||||
|
|
||||||
|
$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"
|
||||||
|
|
||||||
|
$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|<max_elements>10<\/max_elements>|<max_elements>2<\/max_elements>|" \
|
||||||
|
> $config_path_tmp
|
||||||
|
mv $config_path_tmp $config_path
|
||||||
|
|
||||||
|
$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'"
|
||||||
|
$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|<max_elements>2<\/max_elements>|<max_elements>10<\/max_elements>|" \
|
||||||
|
> $config_path_tmp
|
||||||
|
mv $config_path_tmp $config_path
|
||||||
|
|
||||||
|
$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"
|
||||||
|
|
||||||
|
$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'"
|
Loading…
Reference in New Issue
Block a user