This commit is contained in:
kssenii 2024-04-05 12:59:57 +02:00
parent c93510b7cf
commit 1cd73b907c
8 changed files with 116 additions and 72 deletions

View File

@ -10,15 +10,6 @@ public:
using FinalizeEvictionFunc = std::function<void(const CachePriorityGuard::Lock & lk)>; using FinalizeEvictionFunc = std::function<void(const CachePriorityGuard::Lock & lk)>;
EvictionCandidates() = default; EvictionCandidates() = default;
EvictionCandidates(EvictionCandidates && other) noexcept
{
candidates = std::move(other.candidates);
candidates_size = std::move(other.candidates_size);
on_finalize = std::move(other.on_finalize);
queue_entries_to_invalidate = std::move(other.queue_entries_to_invalidate);
hold_space = std::move(other.hold_space);
}
~EvictionCandidates(); ~EvictionCandidates();
void add( void add(
@ -26,8 +17,6 @@ public:
LockedKey & locked_key, LockedKey & locked_key,
const CachePriorityGuard::Lock &); const CachePriorityGuard::Lock &);
void add(const EvictionCandidates & other, const CachePriorityGuard::Lock &) { candidates.insert(other.candidates.begin(), other.candidates.end()); }
void evict(); void evict();
void onFinalize(FinalizeEvictionFunc && func) { on_finalize.emplace_back(std::move(func)); } void onFinalize(FinalizeEvictionFunc && func) { on_finalize.emplace_back(std::move(func)); }

View File

@ -953,49 +953,71 @@ void FileCache::freeSpaceRatioKeepingThreadFunc()
static constexpr auto space_ratio_satisfied_reschedule_ms = 5000; static constexpr auto space_ratio_satisfied_reschedule_ms = 5000;
static constexpr auto general_reschedule_ms = 5000; static constexpr auto general_reschedule_ms = 5000;
while (true) if (shutdown)
return;
Stopwatch watch;
auto lock = tryLockCache();
/// To avoid deteriorating contention on cache,
/// proceed only if cache is not heavily used.
if (!lock)
{ {
keep_up_free_space_ratio_task->scheduleAfter(lock_failed_reschedule_ms);
return;
}
const size_t size_limit = main_priority->getSizeLimit(lock);
const size_t elements_limit = main_priority->getElementsLimit(lock);
const size_t desired_size = std::lround(keep_current_size_to_max_ratio * size_limit);
const size_t desired_elements_num = std::lround(keep_current_elements_to_max_ratio * elements_limit);
if ((size_limit == 0 || main_priority->getSize(lock) <= desired_size)
&& (elements_limit == 0 || main_priority->getElementsCount(lock) <= desired_elements_num))
{
/// Nothing to free - all limits are satisfied.
keep_up_free_space_ratio_task->scheduleAfter(space_ratio_satisfied_reschedule_ms);
return;
}
FileCacheReserveStat stat;
EvictionCandidates eviction_candidates;
try
{
/// Collect at most `keep_up_free_space_remove_batch` elements to evict,
/// (we use batches to make sure we do not block cache for too long,
/// by default the batch size is quite small).
const bool limits_satisfied = main_priority->collectCandidatesForEviction(
desired_size, desired_elements_num, keep_up_free_space_remove_batch, stat, eviction_candidates, lock);
#ifdef ABORT_ON_LOGICAL_ERROR
/// Let's make sure that we correctly processed the limits.
if (limits_satisfied && eviction_candidates.size() < keep_up_free_space_remove_batch)
{
const auto current_size = main_priority->getSize(lock);
chassert(current_size >= stat.total_stat.releasable_size);
chassert(!size_limit
|| current_size <= desired_size
|| current_size - stat.total_stat.releasable_size <= desired_size);
const auto current_elements_count = main_priority->getElementsCount(lock);
chassert(current_elements_count >= stat.total_stat.releasable_count);
chassert(!elements_limit
|| current_elements_count <= desired_elements_num
|| current_elements_count - stat.total_stat.releasable_count <= desired_elements_num);
}
#else
UNUSED(limits_satisfied);
#endif
if (shutdown) if (shutdown)
return; return;
auto lock = tryLockCache(); if (eviction_candidates.size() > 0)
if (!lock)
{ {
keep_up_free_space_ratio_task->scheduleAfter(lock_failed_reschedule_ms);
return;
}
const size_t size_limit = main_priority->getSizeLimit(lock);
const size_t elements_limit = main_priority->getElementsLimit(lock);
const size_t desired_size = std::lround(keep_current_size_to_max_ratio * size_limit);
const size_t desired_elements_num = std::lround(keep_current_elements_to_max_ratio * elements_limit);
if ((size_limit == 0 || main_priority->getSize(lock) <= desired_size)
&& (elements_limit == 0 || main_priority->getElementsCount(lock) <= desired_elements_num))
{
/// Nothing to free - all limits are satisfied.
keep_up_free_space_ratio_task->scheduleAfter(space_ratio_satisfied_reschedule_ms);
return;
}
try
{
FileCacheReserveStat stat;
auto eviction_candidates = main_priority->collectCandidatesForEviction(
desired_size, desired_elements_num, keep_up_free_space_remove_batch, stat, lock);
if (shutdown)
return;
if (eviction_candidates.size() == 0)
{
/// This case is impossible in realistic cache setup,
/// e.g. we should always be able to evict something.
keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms);
return;
}
LOG_TRACE(log, "Current usage {}/{} in size, {}/{} in elements count " LOG_TRACE(log, "Current usage {}/{} in size, {}/{} in elements count "
"(trying to keep size ration at {} and elements ratio at {}). " "(trying to keep size ration at {} and elements ratio at {}). "
"Collected {} eviction candidates, " "Collected {} eviction candidates, "
@ -1006,20 +1028,35 @@ void FileCache::freeSpaceRatioKeepingThreadFunc()
eviction_candidates.size(), stat.total_stat.non_releasable_count); eviction_candidates.size(), stat.total_stat.non_releasable_count);
lock.unlock(); lock.unlock();
/// Remove files from filesystem.
eviction_candidates.evict(); eviction_candidates.evict();
/// Take lock again to finalize eviction,
/// e.g. to update the in-memory state.
lock.lock(); lock.lock();
eviction_candidates.finalize(nullptr, lock); eviction_candidates.finalize(nullptr, lock);
} }
catch (...) else
{ {
tryLogCurrentException(__PRETTY_FUNCTION__);
keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms);
/// Let's catch such cases in ci, in general there should not be exceptions.
chassert(false);
} }
} }
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
if (eviction_candidates.size() > 0)
eviction_candidates.finalize(nullptr, lockCache());
keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms);
/// Let's catch such cases in ci,
/// in general there should not be exceptions.
chassert(false);
}
LOG_TRACE(log, "Free space ratio keeping thread finished in {} ms", watch.elapsedMilliseconds());
} }
void FileCache::iterate(IterateFunc && func, const UserID & user_id) void FileCache::iterate(IterateFunc && func, const UserID & user_id)

View File

@ -148,11 +148,12 @@ public:
const CachePriorityGuard::Lock &) = 0; const CachePriorityGuard::Lock &) = 0;
/// Collect eviction `candidates_num` candidates for eviction. /// Collect eviction `candidates_num` candidates for eviction.
virtual EvictionCandidates collectCandidatesForEviction( virtual bool collectCandidatesForEviction(
size_t desired_size, size_t desired_size,
size_t desired_elements_count, size_t desired_elements_count,
size_t max_candidates_to_evict, size_t max_candidates_to_evict,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & candidates,
const CachePriorityGuard::Lock &) = 0; const CachePriorityGuard::Lock &) = 0;
virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) = 0; virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) = 0;

View File

@ -322,24 +322,24 @@ bool LRUFileCachePriority::collectCandidatesForEviction(
} }
} }
EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction( bool LRUFileCachePriority::collectCandidatesForEviction(
size_t desired_size, size_t desired_size,
size_t desired_elements_count, size_t desired_elements_count,
size_t max_candidates_to_evict, size_t max_candidates_to_evict,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & res,
const CachePriorityGuard::Lock & lock) const CachePriorityGuard::Lock & lock)
{ {
if (!max_candidates_to_evict) if (!max_candidates_to_evict)
return {}; return {};
EvictionCandidates res;
auto stop_condition = [&, this]() auto stop_condition = [&, this]()
{ {
return (getSize(lock) <= desired_size && getElementsCount(lock) <= desired_elements_count) return (getSize(lock) <= desired_size && getElementsCount(lock) <= desired_elements_count)
|| res.size() >= max_candidates_to_evict; || res.size() >= max_candidates_to_evict;
}; };
iterateForEviction(res, stat, stop_condition, lock); iterateForEviction(res, stat, stop_condition, lock);
return res; return stop_condition();
} }
void LRUFileCachePriority::iterateForEviction( void LRUFileCachePriority::iterateForEviction(

View File

@ -62,11 +62,12 @@ public:
const UserID & user_id, const UserID & user_id,
const CachePriorityGuard::Lock &) override; const CachePriorityGuard::Lock &) override;
EvictionCandidates collectCandidatesForEviction( bool collectCandidatesForEviction(
size_t desired_size, size_t desired_size,
size_t desired_elements_count, size_t desired_elements_count,
size_t max_candidates_to_evict, size_t max_candidates_to_evict,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & res,
const CachePriorityGuard::Lock &) override; const CachePriorityGuard::Lock &) override;
void shuffle(const CachePriorityGuard::Lock &) override; void shuffle(const CachePriorityGuard::Lock &) override;

View File

@ -251,11 +251,12 @@ bool SLRUFileCachePriority::collectCandidatesForEvictionInProtected(
return true; return true;
} }
EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( bool SLRUFileCachePriority::collectCandidatesForEviction(
size_t desired_size, size_t desired_size,
size_t desired_elements_count, size_t desired_elements_count,
size_t max_candidates_to_evict, size_t max_candidates_to_evict,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & res,
const CachePriorityGuard::Lock & lock) const CachePriorityGuard::Lock & lock)
{ {
if (!max_candidates_to_evict) if (!max_candidates_to_evict)
@ -264,22 +265,36 @@ EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction(
const auto desired_probationary_size = getRatio(desired_size, 1 - size_ratio); const auto desired_probationary_size = getRatio(desired_size, 1 - size_ratio);
const auto desired_probationary_elements_num = getRatio(desired_elements_count, 1 - size_ratio); const auto desired_probationary_elements_num = getRatio(desired_elements_count, 1 - size_ratio);
auto res = probationary_queue.collectCandidatesForEviction( FileCacheReserveStat probationary_stat;
desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, stat, lock); const bool probationary_limit_satisfied = probationary_queue.collectCandidatesForEviction(
desired_probationary_size, desired_probationary_elements_num,
max_candidates_to_evict, probationary_stat, res, lock);
stat += probationary_stat;
LOG_TEST(log, "Collected {} to evict from probationary queue. Total size: {}",
res.size(), probationary_stat.total_stat.releasable_size);
chassert(res.size() <= max_candidates_to_evict); chassert(res.size() <= max_candidates_to_evict);
chassert(res.size() == stat.total_stat.releasable_count); chassert(res.size() == stat.total_stat.releasable_count);
if (res.size() == max_candidates_to_evict) if (res.size() >= max_candidates_to_evict)
return res; return probationary_limit_satisfied;
const auto desired_protected_size = getRatio(max_size, size_ratio); const auto desired_protected_size = getRatio(max_size, size_ratio);
const auto desired_protected_elements_num = getRatio(max_elements, size_ratio); const auto desired_protected_elements_num = getRatio(max_elements, size_ratio);
auto res_add = protected_queue.collectCandidatesForEviction( FileCacheReserveStat protected_stat;
desired_protected_size, desired_protected_elements_num, max_candidates_to_evict - res.size(), stat, lock); const bool protected_limit_satisfied = protected_queue.collectCandidatesForEviction(
res.add(res_add, lock); desired_protected_size, desired_protected_elements_num,
return res; max_candidates_to_evict - res.size(), protected_stat, res, lock);
stat += protected_stat;
LOG_TEST(log, "Collected {} to evict from protected queue. Total size: {}",
res.size(), protected_stat.total_stat.releasable_size);
return probationary_limit_satisfied && protected_limit_satisfied;
} }
void SLRUFileCachePriority::downgrade(IteratorPtr iterator, const CachePriorityGuard::Lock & lock) void SLRUFileCachePriority::downgrade(IteratorPtr iterator, const CachePriorityGuard::Lock & lock)

View File

@ -58,11 +58,12 @@ public:
const UserID & user_id, const UserID & user_id,
const CachePriorityGuard::Lock &) override; const CachePriorityGuard::Lock &) override;
EvictionCandidates collectCandidatesForEviction( bool collectCandidatesForEviction(
size_t desired_size, size_t desired_size,
size_t desired_elements_count, size_t desired_elements_count,
size_t max_candidates_to_evict, size_t max_candidates_to_evict,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & res,
const CachePriorityGuard::Lock &) override; const CachePriorityGuard::Lock &) override;
void shuffle(const CachePriorityGuard::Lock &) override; void shuffle(const CachePriorityGuard::Lock &) override;

View File

@ -19,13 +19,13 @@
<type>cache</type> <type>cache</type>
<disk>s3_disk</disk> <disk>s3_disk</disk>
<path>s3_cache/</path> <path>s3_cache/</path>
<max_size>64Mi</max_size> <max_size>100Mi</max_size>
<cache_on_write_operations>1</cache_on_write_operations> <cache_on_write_operations>1</cache_on_write_operations>
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms> <delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
<cache_policy>LRU</cache_policy> <cache_policy>LRU</cache_policy>
<slru_size_ratio>0.3</slru_size_ratio> <slru_size_ratio>0.3</slru_size_ratio>
<keep_free_space_size_ratio>0.1</keep_free_space_size_ratio> <keep_free_space_size_ratio>0.15</keep_free_space_size_ratio>
<keep_free_space_elements_ratio>0.1</keep_free_space_elements_ratio> <keep_free_space_elements_ratio>0.15</keep_free_space_elements_ratio>
</s3_cache> </s3_cache>
<s3_cache_02933> <s3_cache_02933>
<type>cache</type> <type>cache</type>