Resolve conflicts after merge, add system.filesystem_cache_settings

This commit is contained in:
kssenii 2024-03-28 16:28:59 +01:00
parent 15768ffce6
commit 19b90d8348
11 changed files with 237 additions and 47 deletions

View File

@ -57,13 +57,25 @@ void EvictionCandidates::add(
++candidates_size;
}
void EvictionCandidates::removeQueueEntries(const CachePriorityGuard::Lock & lock)
{
for (const auto & [key, key_candidates] : candidates)
{
for (const auto & candidate : key_candidates.candidates)
candidate->getQueueIterator()->remove(lock);
}
invalidated_queue_entries = true;
}
void EvictionCandidates::evict()
{
if (candidates.empty())
return;
auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds);
queue_entries_to_invalidate.reserve(candidates_size);
if (!invalidated_queue_entries)
queue_entries_to_invalidate.reserve(candidates_size);
for (auto & [key, key_candidates] : candidates)
{
@ -111,7 +123,9 @@ void EvictionCandidates::evict()
/// it was freed in favour of some reserver, so we can make it visibly
/// free only for that particular reserver.
queue_entries_to_invalidate.push_back(iterator);
if (!invalidated_queue_entries)
queue_entries_to_invalidate.push_back(iterator);
key_candidates.candidates.pop_back();
}
}
@ -165,4 +179,9 @@ void EvictionCandidates::setSpaceHolder(
hold_space = std::make_unique<IFileCachePriority::HoldSpace>(size, elements, priority, lock);
}
void EvictionCandidates::insert(EvictionCandidates && other, const CachePriorityGuard::Lock &)
{
candidates.insert(make_move_iterator(other.candidates.begin()), make_move_iterator(other.candidates.end()));
}
}

View File

@ -9,6 +9,13 @@ class EvictionCandidates
public:
using FinalizeEvictionFunc = std::function<void(const CachePriorityGuard::Lock & lk)>;
EvictionCandidates() = default;
EvictionCandidates(EvictionCandidates && other) noexcept
{
candidates = std::move(other.candidates);
candidates_size = std::move(other.candidates_size);
queue_entries_to_invalidate = std::move(other.queue_entries_to_invalidate);
}
~EvictionCandidates();
void add(
@ -16,8 +23,12 @@ public:
LockedKey & locked_key,
const CachePriorityGuard::Lock &);
void insert(EvictionCandidates && other, const CachePriorityGuard::Lock &);
void evict();
void removeQueueEntries(const CachePriorityGuard::Lock &);
void onFinalize(FinalizeEvictionFunc && func) { on_finalize.emplace_back(std::move(func)); }
void finalize(
@ -47,7 +58,10 @@ private:
size_t candidates_size = 0;
std::vector<FinalizeEvictionFunc> on_finalize;
std::vector<IFileCachePriority::IteratorPtr> queue_entries_to_invalidate;
bool invalidated_queue_entries = false;
IFileCachePriority::HoldSpacePtr hold_space;
};

View File

@ -1383,7 +1383,7 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings,
if (new_settings.max_size != actual_settings.max_size
|| new_settings.max_elements != actual_settings.max_elements)
{
std::vector<std::string> evicted_paths;
std::optional<EvictionCandidates> eviction_candidates;
{
cache_is_being_resized.store(true, std::memory_order_relaxed);
SCOPE_EXIT({
@ -1391,18 +1391,27 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings,
});
auto cache_lock = lockCache();
FileCacheReserveStat stat;
auto eviction_candidates = main_priority->collectCandidatesForEviction(
new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, stat, cache_lock);
evicted_paths = eviction_candidates.evictFromMemory(nullptr, cache_lock);
FileCacheReserveStat stat;
eviction_candidates.emplace(main_priority->collectCandidatesForEviction(
new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, stat, cache_lock));
eviction_candidates->removeQueueEntries(cache_lock);
main_priority->modifySizeLimits(
new_settings.max_size, new_settings.max_elements, new_settings.slru_size_ratio, cache_lock);
}
for (const auto & path : evicted_paths)
fs::remove(path);
try
{
eviction_candidates->evict();
}
catch (...)
{
auto cache_lock = lockCache();
eviction_candidates->finalize(nullptr, cache_lock);
throw;
}
LOG_INFO(log, "Changed max_size from {} to {}, max_elements from {} to {}",
actual_settings.max_size, new_settings.max_size,

View File

@ -279,10 +279,42 @@ bool LRUFileCachePriority::collectCandidatesForEviction(
auto can_fit = [&]
{
return canFit(size, 1, stat.stat.releasable_size, stat.stat.releasable_count, lock);
return canFit(size, 1, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock);
};
iterateForEviction(res, stat, can_fit, lock);
return can_fit();
if (can_fit())
{
/// As eviction is done without a cache priority lock,
/// then if some space was partially available and some needed
/// to be freed via eviction, we need to make sure that this
/// partially available space is still available
/// after we finish with eviction for non-available space.
/// So we create a space holder for the currently available part
/// of the required space for the duration of eviction of the other
/// currently non-available part of the space.
const size_t hold_size = size > stat.total_stat.releasable_size
? size - stat.total_stat.releasable_size
: 0;
const size_t hold_elements = elements > stat.total_stat.releasable_count
? elements - stat.total_stat.releasable_count
: 0;
if (hold_size || hold_elements)
res.setSpaceHolder(hold_size, hold_elements, *this, lock);
// LOG_TEST(log, "Collected {} candidates for eviction (total size: {}). "
// "Took hold of size {} and elements {}",
// res.size(), stat.total_stat.releasable_size, hold_size, hold_elements);
return true;
}
else
{
return false;
}
}
EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction(
@ -295,7 +327,7 @@ EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction(
EvictionCandidates res;
auto stop_condition = [&, this]()
{
return canFit(0, 0, stat.stat.releasable_size, stat.stat.releasable_count,
return canFit(0, 0, stat.total_stat.releasable_size, stat.total_stat.releasable_count,
lock, &desired_size, &desired_elements_count)
|| (max_candidates_to_evict && res.size() >= max_candidates_to_evict);
};
@ -334,39 +366,6 @@ void LRUFileCachePriority::iterateForEviction(
{
return stop_condition() ? IterationResult::BREAK : iterate_func(locked_key, segment_metadata);
}, lock);
if (can_fit())
{
/// As eviction is done without a cache priority lock,
/// then if some space was partially available and some needed
/// to be freed via eviction, we need to make sure that this
/// partially available space is still available
/// after we finish with eviction for non-available space.
/// So we create a space holder for the currently available part
/// of the required space for the duration of eviction of the other
/// currently non-available part of the space.
const size_t hold_size = size > stat.total_stat.releasable_size
? size - stat.total_stat.releasable_size
: 0;
const size_t hold_elements = elements > stat.total_stat.releasable_count
? elements - stat.total_stat.releasable_count
: 0;
if (hold_size || hold_elements)
res.setSpaceHolder(hold_size, hold_elements, *this, lock);
// LOG_TEST(log, "Collected {} candidates for eviction (total size: {}). "
// "Took hold of size {} and elements {}",
// res.size(), stat.total_stat.releasable_size, hold_size, hold_elements);
return true;
}
else
{
return false;
}
}
LRUFileCachePriority::LRUIterator LRUFileCachePriority::move(

View File

@ -941,7 +941,7 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl(
file_segment->detach(segment_lock, *this);
if (!remove_only_metadata)
// if (!remove_only_metadata)
{
try
{

View File

@ -263,7 +263,7 @@ EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction(
desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, stat, lock);
chassert(!max_candidates_to_evict || res.size() <= max_candidates_to_evict);
chassert(res.size() == stat.stat.releasable_count);
chassert(res.size() == stat.total_stat.releasable_count);
if (max_candidates_to_evict && res.size() == max_candidates_to_evict)
return res;

View File

@ -0,0 +1,72 @@
#include "StorageSystemFilesystemCacheSettings.h"
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Access/Common/AccessFlags.h>
#include <Interpreters/Context.h>
#include <Disks/IDisk.h>
namespace DB
{
ColumnsDescription StorageSystemFilesystemCacheSettings::getColumnsDescription()
{
return ColumnsDescription
{
{"cache_name", std::make_shared<DataTypeString>(), "Name of the cache object"},
{"path", std::make_shared<DataTypeString>(), "Cache directory"},
{"max_size", std::make_shared<DataTypeUInt64>(), "Cache size limit by the number of bytes"},
{"max_elements", std::make_shared<DataTypeUInt64>(), "Cache size limit by the number of elements"},
{"current_size", std::make_shared<DataTypeUInt64>(), "Current cache size by the number of bytes"},
{"current_elements", std::make_shared<DataTypeUInt64>(), "Current cache size by the number of elements"},
{"max_file_segment_size", std::make_shared<DataTypeUInt64>(), "Maximum allowed file segment size"},
{"boundary_alignment", std::make_shared<DataTypeUInt64>(), "Boundary alignment of file segments"},
{"cache_on_write_operations", std::make_shared<DataTypeUInt8>(), "Write-through cache enablemenet setting"},
{"cache_hits_threshold", std::make_shared<DataTypeUInt8>(), "Cache hits threshold enablemenet setting"},
{"background_download_threads", std::make_shared<DataTypeUInt64>(), "Number of background download threads"},
{"background_download_queue_size_limit", std::make_shared<DataTypeUInt64>(), "Queue size limit for background download"},
{"load_metadata_threads", std::make_shared<DataTypeUInt64>(), "Number of load metadata threads"},
{"enable_bypass_cache_threshold", std::make_shared<DataTypeUInt64>(), "Bypass cache threshold limit enablement setting"},
};
}
StorageSystemFilesystemCacheSettings::StorageSystemFilesystemCacheSettings(const StorageID & table_id_)
: IStorageSystemOneBlock(table_id_, getColumnsDescription())
{
}
void StorageSystemFilesystemCacheSettings::fillData(
MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector<UInt8>) const
{
context->checkAccess(AccessType::SHOW_FILESYSTEM_CACHES);
auto caches = FileCacheFactory::instance().getAll();
for (const auto & [cache_name, cache_data] : caches)
{
const auto & settings = cache_data->getSettings();
const auto & cache = cache_data->cache;
size_t i = 0;
res_columns[i++]->insert(cache_name);
res_columns[i++]->insert(settings.base_path);
res_columns[i++]->insert(settings.max_size);
res_columns[i++]->insert(settings.max_elements);
res_columns[i++]->insert(cache->getUsedCacheSize());
res_columns[i++]->insert(cache->getFileSegmentsNum());
res_columns[i++]->insert(settings.max_file_segment_size);
res_columns[i++]->insert(settings.boundary_alignment);
res_columns[i++]->insert(settings.cache_on_write_operations);
res_columns[i++]->insert(settings.cache_hits_threshold);
res_columns[i++]->insert(settings.background_download_threads);
res_columns[i++]->insert(settings.background_download_queue_size_limit);
res_columns[i++]->insert(settings.load_metadata_threads);
res_columns[i++]->insert(settings.enable_bypass_cache_with_threshold);
}
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Storages/System/IStorageSystemOneBlock.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h>
namespace DB
{
class StorageSystemFilesystemCacheSettings final : public IStorageSystemOneBlock
{
public:
explicit StorageSystemFilesystemCacheSettings(const StorageID & table_id_);
std::string getName() const override { return "SystemFilesystemCacheSettings"; }
static ColumnsDescription getColumnsDescription();
protected:
void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector<UInt8>) const override;
};
}

View File

@ -78,6 +78,7 @@
#include <Storages/System/StorageSystemAsynchronousInserts.h>
#include <Storages/System/StorageSystemTransactions.h>
#include <Storages/System/StorageSystemFilesystemCache.h>
#include <Storages/System/StorageSystemFilesystemCacheSettings.h>
#include <Storages/System/StorageSystemQueryCache.h>
#include <Storages/System/StorageSystemNamedCollections.h>
#include <Storages/System/StorageSystemRemoteDataPaths.h>
@ -213,6 +214,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b
attach<StorageSystemPartMovesBetweenShards>(context, system_database, "part_moves_between_shards", "Contains information about parts which are currently in a process of moving between shards and their progress.");
attach<StorageSystemAsynchronousInserts>(context, system_database, "asynchronous_inserts", "Contains information about pending asynchronous inserts in queue in server's memory.");
attachNoDescription<StorageSystemFilesystemCache>(context, system_database, "filesystem_cache", "Contains information about all entries inside filesystem cache for remote objects.");
attachNoDescription<StorageSystemFilesystemCacheSettings>(context, system_database, "filesystem_cache_settings", "Contains information about all filesystem cache settings");
attachNoDescription<StorageSystemQueryCache>(context, system_database, "query_cache", "Contains information about all entries inside query cache in server's memory.");
attachNoDescription<StorageSystemRemoteDataPaths>(context, system_database, "remote_data_paths", "Contains a mapping from a filename on local filesystem to a blob name inside object storage.");
attach<StorageSystemCertificates>(context, system_database, "certificates", "Contains information about available certificates and their sources.");

View File

@ -0,0 +1,46 @@
#!/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"
$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(1000);
"
$CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null"
prev_max_size=$($CLICKHOUSE_CLIENT --query "SELECT max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name'")
$CLICKHOUSE_CLIENT --query "SELECT current_size > 0 FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated"
config_path=/etc/clickhouse-server/config.d/storage_conf.xml
config_path_tmp=$config_path.tmp
new_max_size=$($CLICKHOUSE_CLIENT --query "SELECT divide(max_size, 2) FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name'")
sed -i "s|<max_size>$prev_max_size<\/max_size>|<max_size>$new_max_size<\/max_size>|" $config_path
# echo $prev_max_size
# echo $new_max_size
$CLICKHOUSE_CLIENT -nm --query "
set send_logs_level='fatal';
SYSTEM RELOAD CONFIG"
$CLICKHOUSE_CLIENT --query "SELECT max_size == $new_max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated"
$CLICKHOUSE_CLIENT --query "SELECT current_size > 0 FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated"
$CLICKHOUSE_CLIENT --query "SELECT current_size <= max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated"
sed -i "s|<max_size>$new_max_size<\/max_size>|<max_size>$prev_max_size<\/max_size>|" $config_path
$CLICKHOUSE_CLIENT -nm --query "
set send_logs_level='fatal';
SYSTEM RELOAD CONFIG"
$CLICKHOUSE_CLIENT --query "SELECT max_size == $prev_max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated"
$CLICKHOUSE_CLIENT --query "SELECT current_size > 0 FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated"
$CLICKHOUSE_CLIENT --query "SELECT current_size <= max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated"