mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Allow to change some cache settings without server restart
This commit is contained in:
parent
8db3bac7bd
commit
f44f7c8c28
@ -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,
|
||||
|
@ -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));
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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)>;
|
||||
|
@ -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(
|
||||
|
@ -154,6 +154,8 @@ public:
|
||||
|
||||
void cancelDownload();
|
||||
|
||||
void setBackgroundDownloadQueueSizeLimit(size_t size);
|
||||
|
||||
private:
|
||||
const std::string path; /// Cache base path
|
||||
const CleanupQueuePtr cleanup_queue;
|
||||
|
@ -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>
|
||||
|
@ -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
|
25
tests/queries/0_stateless/02933_change_cache_setting_without_restart.sh
Executable file
25
tests/queries/0_stateless/02933_change_cache_setting_without_restart.sh
Executable 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}'"
|
Loading…
Reference in New Issue
Block a user