Allow to change some cache settings without server restart

This commit is contained in:
kssenii 2023-12-05 11:07:14 +01:00
parent 8db3bac7bd
commit f44f7c8c28
11 changed files with 163 additions and 48 deletions

View File

@ -65,7 +65,7 @@ void registerDiskCache(DiskFactory & factory, bool /* global_skip_access_check *
}
}
auto cache = FileCacheFactory::instance().getOrCreate(name, file_cache_settings);
auto cache = FileCacheFactory::instance().getOrCreate(name, file_cache_settings, predefined_configuration ? "" : config_prefix);
auto disk = disk_it->second;
if (!dynamic_cast<const DiskObjectStorage *>(disk.get()))
throw Exception(ErrorCodes::BAD_ARGUMENTS,

View File

@ -53,15 +53,13 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & settings)
: max_file_segment_size(settings.max_file_segment_size)
, bypass_cache_threshold(settings.enable_bypass_cache_with_threshold ? settings.bypass_cache_threshold : 0)
, boundary_alignment(settings.boundary_alignment)
, background_download_threads(settings.background_download_threads)
, metadata_download_threads(settings.load_metadata_threads)
FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & settings_)
: settings(settings_)
, log(&Poco::Logger::get("FileCache(" + cache_name + ")"))
, metadata(settings.base_path, settings.background_download_queue_size_limit)
{
settings.bypass_cache_threshold = settings.enable_bypass_cache_with_threshold ? settings.bypass_cache_threshold : 0;
main_priority = std::make_unique<LRUFileCachePriority>(settings.max_size, settings.max_elements);
if (settings.cache_hits_threshold)
@ -135,7 +133,7 @@ void FileCache::initialize()
is_initialized = true;
for (size_t i = 0; i < background_download_threads; ++i)
for (size_t i = 0; i < settings.background_download_threads; ++i)
download_threads.emplace_back([this] { metadata.downloadThreadFunc(); });
cleanup_thread = std::make_unique<ThreadFromGlobalPool>(std::function{ [this]{ metadata.cleanupThreadFunc(); }});
@ -152,7 +150,7 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment:
/// Given range = [left, right] and non-overlapping ordered set of file segments,
/// find list [segment1, ..., segmentN] of segments which intersect with given range.
if (bypass_cache_threshold && range.size() > bypass_cache_threshold)
if (settings.bypass_cache_threshold && range.size() > settings.bypass_cache_threshold)
{
auto file_segment = std::make_shared<FileSegment>(
locked_key.getKey(), range.left, range.size(), FileSegment::State::DETACHED);
@ -257,7 +255,7 @@ std::vector<FileSegment::Range> FileCache::splitRange(size_t offset, size_t size
FileSegments file_segments;
while (current_pos < end_pos_non_included)
{
auto current_file_segment_size = std::min(remaining_size, max_file_segment_size);
auto current_file_segment_size = std::min(remaining_size, settings.max_file_segment_size);
ranges.emplace_back(current_pos, current_pos + current_file_segment_size - 1);
remaining_size -= current_file_segment_size;
@ -273,7 +271,7 @@ FileSegments FileCache::splitRangeIntoFileSegments(
size_t size,
FileSegment::State state,
size_t file_segments_limit,
const CreateFileSegmentSettings & settings)
const CreateFileSegmentSettings & create_settings)
{
assert(size > 0);
@ -286,11 +284,11 @@ FileSegments FileCache::splitRangeIntoFileSegments(
FileSegments file_segments;
while (current_pos < end_pos_non_included && (!file_segments_limit || file_segments.size() < file_segments_limit))
{
current_file_segment_size = std::min(remaining_size, max_file_segment_size);
current_file_segment_size = std::min(remaining_size, settings.max_file_segment_size);
remaining_size -= current_file_segment_size;
auto file_segment_metadata_it = addFileSegment(
locked_key, current_pos, current_file_segment_size, state, settings, nullptr);
locked_key, current_pos, current_file_segment_size, state, create_settings, nullptr);
file_segments.push_back(file_segment_metadata_it->second->file_segment);
current_pos += current_file_segment_size;
@ -305,7 +303,7 @@ void FileCache::fillHolesWithEmptyFileSegments(
const FileSegment::Range & range,
size_t file_segments_limit,
bool fill_with_detached_file_segments,
const CreateFileSegmentSettings & settings)
const CreateFileSegmentSettings & create_settings)
{
/// There are segments [segment1, ..., segmentN]
/// (non-overlapping, non-empty, ascending-ordered) which (maybe partially)
@ -362,7 +360,7 @@ void FileCache::fillHolesWithEmptyFileSegments(
if (fill_with_detached_file_segments)
{
auto file_segment = std::make_shared<FileSegment>(
locked_key.getKey(), current_pos, hole_size, FileSegment::State::DETACHED, settings);
locked_key.getKey(), current_pos, hole_size, FileSegment::State::DETACHED, create_settings);
file_segments.insert(it, file_segment);
++processed_count;
@ -373,7 +371,7 @@ void FileCache::fillHolesWithEmptyFileSegments(
FileSegments hole;
for (const auto & r : ranges)
{
auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, settings, nullptr);
auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr);
hole.push_back(metadata_it->second->file_segment);
++processed_count;
@ -418,7 +416,7 @@ void FileCache::fillHolesWithEmptyFileSegments(
if (fill_with_detached_file_segments)
{
auto file_segment = std::make_shared<FileSegment>(
locked_key.getKey(), current_pos, hole_size, FileSegment::State::DETACHED, settings);
locked_key.getKey(), current_pos, hole_size, FileSegment::State::DETACHED, create_settings);
file_segments.insert(file_segments.end(), file_segment);
}
@ -428,7 +426,7 @@ void FileCache::fillHolesWithEmptyFileSegments(
FileSegments hole;
for (const auto & r : ranges)
{
auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, settings, nullptr);
auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr);
hole.push_back(metadata_it->second->file_segment);
++processed_count;
@ -447,7 +445,7 @@ FileSegmentsHolderPtr FileCache::set(
const Key & key,
size_t offset,
size_t size,
const CreateFileSegmentSettings & settings)
const CreateFileSegmentSettings & create_settings)
{
assertInitialized();
@ -458,17 +456,17 @@ FileSegmentsHolderPtr FileCache::set(
if (!file_segments.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Having intersection with already existing cache");
if (settings.unbounded)
if (create_settings.unbounded)
{
/// If the file is unbounded, we can create a single file_segment_metadata for it.
auto file_segment_metadata_it = addFileSegment(
*locked_key, offset, size, FileSegment::State::EMPTY, settings, nullptr);
*locked_key, offset, size, FileSegment::State::EMPTY, create_settings, nullptr);
file_segments = {file_segment_metadata_it->second->file_segment};
}
else
{
file_segments = splitRangeIntoFileSegments(
*locked_key, offset, size, FileSegment::State::EMPTY, /* file_segments_limit */0, settings);
*locked_key, offset, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings);
}
return std::make_unique<FileSegmentsHolder>(std::move(file_segments));
@ -480,7 +478,7 @@ FileCache::getOrSet(
size_t offset,
size_t size,
size_t file_size,
const CreateFileSegmentSettings & settings,
const CreateFileSegmentSettings & create_settings,
size_t file_segments_limit)
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheGetOrSetMicroseconds);
@ -489,8 +487,8 @@ FileCache::getOrSet(
FileSegment::Range range(offset, offset + size - 1);
const auto aligned_offset = roundDownToMultiple(range.left, boundary_alignment);
auto aligned_end_offset = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size) - 1;
const auto aligned_offset = roundDownToMultiple(range.left, settings.boundary_alignment);
auto aligned_end_offset = std::min(roundUpToMultiple(offset + size, settings.boundary_alignment), file_size) - 1;
chassert(aligned_offset <= range.left);
chassert(aligned_end_offset >= range.right);
@ -586,7 +584,7 @@ FileCache::getOrSet(
if (file_segments.empty())
{
file_segments = splitRangeIntoFileSegments(*locked_key, range.left, range.size(), FileSegment::State::EMPTY, file_segments_limit, settings);
file_segments = splitRangeIntoFileSegments(*locked_key, range.left, range.size(), FileSegment::State::EMPTY, file_segments_limit, create_settings);
}
else
{
@ -594,7 +592,7 @@ FileCache::getOrSet(
chassert(file_segments.back()->range().left <= range.right);
fillHolesWithEmptyFileSegments(
*locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */false, settings);
*locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */false, create_settings);
if (!file_segments.front()->range().contains(offset))
{
@ -649,7 +647,7 @@ KeyMetadata::iterator FileCache::addFileSegment(
size_t offset,
size_t size,
FileSegment::State state,
const CreateFileSegmentSettings & settings,
const CreateFileSegmentSettings & create_settings,
const CacheGuard::Lock * lock)
{
/// Create a file_segment_metadata and put it in `files` map by [key][offset].
@ -703,7 +701,7 @@ KeyMetadata::iterator FileCache::addFileSegment(
result_state = state;
}
auto file_segment = std::make_shared<FileSegment>(key, offset, size, result_state, settings, background_download_threads > 0, this, locked_key.getKeyMetadata());
auto file_segment = std::make_shared<FileSegment>(key, offset, size, result_state, create_settings, settings.background_download_threads > 0, this, locked_key.getKeyMetadata());
auto file_segment_metadata = std::make_shared<FileSegmentMetadata>(std::move(file_segment));
auto [file_segment_metadata_it, inserted] = locked_key.emplace(offset, file_segment_metadata);
@ -1051,9 +1049,9 @@ void FileCache::loadMetadataImpl()
std::mutex set_exception_mutex;
std::atomic<bool> stop_loading = false;
LOG_INFO(log, "Loading filesystem cache with {} threads", metadata_download_threads);
LOG_INFO(log, "Loading filesystem cache with {} threads", settings.load_metadata_threads);
for (size_t i = 0; i < metadata_download_threads; ++i)
for (size_t i = 0; i < settings.load_metadata_threads; ++i)
{
try
{
@ -1258,6 +1256,11 @@ FileCache::~FileCache()
void FileCache::deactivateBackgroundOperations()
{
{
auto lock = lockCache();
shutdown = true;
}
metadata.cancelDownload();
metadata.cancelCleanup();
@ -1349,6 +1352,36 @@ void FileCache::assertCacheCorrectness()
});
}
FileCacheSettings FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings)
{
if (!is_initialized)
return settings;
auto lock = lockCache();
if (shutdown)
return settings;
if (settings.background_download_queue_size_limit != new_settings.background_download_queue_size_limit)
{
LOG_DEBUG(log, "Changing background_download_queue_size_limit from {} to {}",
settings.background_download_queue_size_limit, new_settings.background_download_queue_size_limit);
metadata.setBackgroundDownloadQueueSizeLimit(new_settings.background_download_queue_size_limit);
settings.background_download_queue_size_limit = new_settings.background_download_queue_size_limit;
}
if (settings.background_download_threads < new_settings.background_download_threads)
{
LOG_DEBUG(log, "Changing background_download_threads from {} to {}", settings.background_download_threads, new_settings.background_download_threads);
size_t threads_to_add = new_settings.background_download_threads - settings.background_download_threads;
for (size_t i = 0; i < threads_to_add; ++i)
download_threads.emplace_back([this] { metadata.downloadThreadFunc(); });
settings.background_download_threads = new_settings.background_download_threads;
}
return settings;
}
FileCache::QueryContextHolder::QueryContextHolder(
const String & query_id_,
FileCache * cache_,
@ -1371,13 +1404,13 @@ FileCache::QueryContextHolder::~QueryContextHolder()
}
FileCache::QueryContextHolderPtr FileCache::getQueryContextHolder(
const String & query_id, const ReadSettings & settings)
const String & query_id, const ReadSettings & read_settings)
{
if (!query_limit || settings.filesystem_cache_max_download_size == 0)
if (!query_limit || read_settings.filesystem_cache_max_download_size == 0)
return {};
auto lock = lockCache();
auto context = query_limit->getOrSetQueryContext(query_id, settings, lock);
auto context = query_limit->getOrSetQueryContext(query_id, read_settings, lock);
return std::make_unique<QueryContextHolder>(query_id, this, std::move(context));
}

View File

@ -16,6 +16,7 @@
#include <Interpreters/Cache/Metadata.h>
#include <Interpreters/Cache/QueryLimit.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h>
#include <Interpreters/Cache/FileCacheSettings.h>
#include <filesystem>
@ -55,7 +56,7 @@ public:
using PriorityIterator = IFileCachePriority::Iterator;
using PriorityIterationResult = IFileCachePriority::IterationResult;
FileCache(const std::string & cache_name, const FileCacheSettings & settings);
FileCache(const std::string & cache_name, const FileCacheSettings & settings_);
~FileCache();
@ -122,7 +123,7 @@ public:
size_t getFileSegmentsNum() const;
size_t getMaxFileSegmentSize() const { return max_file_segment_size; }
size_t getMaxFileSegmentSize() const { return settings.max_file_segment_size; }
bool tryReserve(FileSegment & file_segment, size_t size, FileCacheReserveStat & stat);
@ -154,14 +155,12 @@ public:
FileSegments sync();
FileCacheSettings applySettingsIfPossible(const FileCacheSettings & settings);
private:
using KeyAndOffset = FileCacheKeyAndOffset;
const size_t max_file_segment_size;
const size_t bypass_cache_threshold = 0;
const size_t boundary_alignment;
const size_t background_download_threads; /// 0 means background download is disabled.
const size_t metadata_download_threads;
FileCacheSettings settings;
Poco::Logger * log;
@ -169,6 +168,7 @@ private:
std::atomic<bool> is_initialized = false;
mutable std::mutex init_mutex;
std::unique_ptr<StatusFile> status_file;
size_t shutdown = false;
CacheMetadata metadata;

View File

@ -22,7 +22,9 @@ FileCacheFactory::CacheByName FileCacheFactory::getAll()
}
FileCachePtr FileCacheFactory::getOrCreate(
const std::string & cache_name, const FileCacheSettings & file_cache_settings)
const std::string & cache_name,
const FileCacheSettings & file_cache_settings,
const std::string & config_path)
{
std::lock_guard lock(mutex);
@ -31,7 +33,7 @@ FileCachePtr FileCacheFactory::getOrCreate(
{
auto cache = std::make_shared<FileCache>(cache_name, file_cache_settings);
it = caches_by_name.emplace(
cache_name, std::make_unique<FileCacheData>(cache, file_cache_settings)).first;
cache_name, std::make_unique<FileCacheData>(cache, file_cache_settings, config_path)).first;
}
return it->second->cache;
@ -48,4 +50,27 @@ FileCacheFactory::FileCacheData FileCacheFactory::getByName(const std::string &
return *it->second;
}
void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfiguration & config)
{
CacheByName caches_by_name_copy;
{
std::lock_guard lock(mutex);
caches_by_name_copy = caches_by_name;
}
for (const auto & [_, cache_info] : caches_by_name_copy)
{
if (cache_info->config_path.empty())
continue;
FileCacheSettings settings;
settings.loadFromConfig(config, cache_info->config_path);
if (settings == cache_info->settings)
continue;
cache_info->settings = cache_info->cache->applySettingsIfPossible(settings);
}
}
}

View File

@ -6,7 +6,6 @@
#include <boost/noncopyable.hpp>
#include <unordered_map>
#include <mutex>
#include <list>
namespace DB
{
@ -21,21 +20,28 @@ public:
{
FileCachePtr cache;
FileCacheSettings settings;
std::string config_path;
FileCacheData() = default;
FileCacheData(FileCachePtr cache_, const FileCacheSettings & settings_) : cache(cache_), settings(settings_) {}
FileCacheData(FileCachePtr cache_, const FileCacheSettings & settings_, const std::string & config_path_)
: cache(cache_), settings(settings_), config_path(config_path_) {}
};
using FileCacheDataPtr = std::shared_ptr<FileCacheData>;
using CacheByName = std::unordered_map<std::string, FileCacheDataPtr>;
static FileCacheFactory & instance();
FileCachePtr getOrCreate(const std::string & cache_name, const FileCacheSettings & file_cache_settings);
FileCachePtr getOrCreate(
const std::string & cache_name,
const FileCacheSettings & file_cache_settings,
const std::string & config_path);
CacheByName getAll();
FileCacheData getByName(const std::string & cache_name);
void updateSettingsFromConfig(const Poco::Util::AbstractConfiguration & config);
private:
std::mutex mutex;
CacheByName caches_by_name;

View File

@ -35,6 +35,8 @@ struct FileCacheSettings
void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
void loadFromCollection(const NamedCollection & collection);
bool operator ==(const FileCacheSettings &) const = default;
private:
using FuncHas = std::function<bool(std::string_view)>;
using FuncGetUInt = std::function<size_t(std::string_view)>;

View File

@ -473,7 +473,7 @@ public:
{
{
std::lock_guard lock(mutex);
if (cancelled || (queue_size_limit && queue.size() == queue_size_limit))
if (cancelled || (queue_size_limit && queue.size() >= queue_size_limit))
return false;
queue.push(DownloadInfo{file_segment->key(), file_segment->offset(), file_segment});
}
@ -483,6 +483,12 @@ public:
return true;
}
void setQueueLimit(size_t size)
{
std::lock_guard lock(mutex);
queue_size_limit = size;
}
private:
void cancel()
{
@ -493,7 +499,7 @@ private:
cv.notify_all();
}
const size_t queue_size_limit;
size_t queue_size_limit;
std::mutex mutex;
std::condition_variable cv;
bool cancelled = false;
@ -607,6 +613,11 @@ void CacheMetadata::downloadThreadFunc()
}
}
void CacheMetadata::setBackgroundDownloadQueueSizeLimit(size_t size)
{
download_queue->setQueueLimit(size);
}
void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory)
{
LOG_TEST(

View File

@ -154,6 +154,8 @@ public:
void cancelDownload();
void setBackgroundDownloadQueueSizeLimit(size_t size);
private:
const std::string path; /// Cache base path
const CleanupQueuePtr cleanup_queue;

View File

@ -17,6 +17,15 @@
<cache_on_write_operations>1</cache_on_write_operations>
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
</s3_cache>
<s3_cache_02933>
<type>cache</type>
<disk>s3_disk</disk>
<path>s3_cache/</path>
<max_size>128Mi</max_size>
<cache_on_write_operations>1</cache_on_write_operations>
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
<background_download_threads>1</background_download_threads>
</s3_cache_02933>
<!-- local disks -->
<local_disk>
<type>local_blob_storage</type>

View File

@ -0,0 +1,2 @@
134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 0 0 1
134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 10 1000 0 1

View File

@ -0,0 +1,25 @@
#!/usr/bin/env bash
# Tags: no-fasttest
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
disk_name="s3_cache_02933"
$CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'"
config_path=/etc/clickhouse-server/config.d/storage_conf.xml
config_path_tmp=$config_path.tmp
cat $config_path \
| sed "s|<background_download_threads>0<\/background_download_threads>|<background_download_threads>10<\/background_download_threads>|" \
> $config_path_tmp
mv $config_path_tmp $config_path
cat $config_path \
| sed "s|<background_download_queue_size_limit>0<\/background_download_queue_size_limit>|<background_download_queue_size_limit>1000<\/background_download_queue_size_limit>|" \
> $config_path_tmp
mv $config_path_tmp $config_path
sleep 3
$CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'"