mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Better
This commit is contained in:
parent
c93510b7cf
commit
1cd73b907c
@ -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)); }
|
||||||
|
@ -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)
|
||||||
|
@ -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;
|
||||||
|
@ -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(
|
||||||
|
@ -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;
|
||||||
|
@ -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)
|
||||||
|
@ -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;
|
||||||
|
@ -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>
|
||||||
|
Loading…
Reference in New Issue
Block a user