mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-16 11:22:12 +00:00
update LRUCache set()
This commit is contained in:
parent
f083c5594c
commit
0c1733817b
@ -22,13 +22,37 @@ struct TrivialWeightFunction
|
||||
}
|
||||
};
|
||||
|
||||
enum class LRUCacheEvictStatus
|
||||
{
|
||||
CAN_EVITCT, // a key can be evicted
|
||||
TERMINATE_EVICT, // stop the evicting process
|
||||
SKIP_EVICT, // skip current value and keep iterating
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct TrivialLRUCacheEvitPolicy
|
||||
{
|
||||
inline LRUCacheEvictStatus canRelease(const T &) const
|
||||
{
|
||||
return LRUCacheEvictStatus::CAN_EVITCT;
|
||||
}
|
||||
|
||||
inline void release(T & )
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Thread-safe cache that evicts entries which are not used for a long time.
|
||||
/// WeightFunction is a functor that takes Mapped as a parameter and returns "weight" (approximate size)
|
||||
/// of that value.
|
||||
/// Cache starts to evict entries when their total weight exceeds max_size.
|
||||
/// Value weight should not change after insertion.
|
||||
template <typename TKey, typename TMapped, typename HashFunction = std::hash<TKey>, typename WeightFunction = TrivialWeightFunction<TMapped>>
|
||||
template <typename TKey,
|
||||
typename TMapped,
|
||||
typename HashFunction = std::hash<TKey>,
|
||||
typename WeightFunction = TrivialWeightFunction<TMapped>,
|
||||
typename EvictPolicy = TrivialLRUCacheEvitPolicy<TMapped>>
|
||||
class LRUCache
|
||||
{
|
||||
public:
|
||||
@ -57,11 +81,15 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
void set(const Key & key, const MappedPtr & mapped)
|
||||
/**
|
||||
* set() will fail if there is no space left and no keys could be evicted.
|
||||
* In some cases, a key can be only evicted when it is not refered by anyone.
|
||||
*/
|
||||
bool set(const Key & key, const MappedPtr & mapped)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
setImpl(key, mapped, lock);
|
||||
return setImpl(key, mapped, lock);
|
||||
}
|
||||
|
||||
/// If the value for the key is in the cache, returns it. If it is not, calls load_func() to
|
||||
@ -117,6 +145,8 @@ public:
|
||||
auto token_it = insert_tokens.find(key);
|
||||
if (token_it != insert_tokens.end() && token_it->second.get() == token)
|
||||
{
|
||||
// setImpl() may fail, but the final behavior seems not be affected
|
||||
// next call of getOrSet() will still call load_func()
|
||||
setImpl(key, token->value, cache_lock);
|
||||
result = true;
|
||||
}
|
||||
@ -259,6 +289,7 @@ private:
|
||||
std::atomic<size_t> misses {0};
|
||||
|
||||
WeightFunction weight_function;
|
||||
EvictPolicy evict_policy;
|
||||
|
||||
MappedPtr getImpl(const Key & key, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
@ -276,7 +307,7 @@ private:
|
||||
return cell.value;
|
||||
}
|
||||
|
||||
void setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
|
||||
bool setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
auto [it, inserted] = cells.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(key),
|
||||
@ -286,6 +317,15 @@ private:
|
||||
|
||||
if (inserted)
|
||||
{
|
||||
auto weight = mapped ? weight_function(*mapped) : 0;
|
||||
// move removeOverflow() ahead here. In default, the final result is the same as the old implementation
|
||||
if (!removeOverflow(weight))
|
||||
{
|
||||
// cannot find enough space to put in the new value
|
||||
cells.erase(it);
|
||||
return false;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
cell.queue_iterator = queue.insert(queue.end(), key);
|
||||
@ -298,6 +338,13 @@ private:
|
||||
}
|
||||
else
|
||||
{
|
||||
if (evict_policy.canRelease(*cell.value) != LRUCacheEvictStatus::CAN_EVITCT)
|
||||
{
|
||||
// the old value is refered by someone, cannot release now
|
||||
// in default policy, it is always CAN_EVITCT.
|
||||
return false;
|
||||
}
|
||||
evict_policy.release(*cell.value); // release the old value. this action is empty in default policy.
|
||||
current_size -= cell.size;
|
||||
queue.splice(queue.end(), queue, cell.queue_iterator);
|
||||
}
|
||||
@ -307,16 +354,20 @@ private:
|
||||
current_size += cell.size;
|
||||
|
||||
removeOverflow();
|
||||
return true;
|
||||
}
|
||||
|
||||
void removeOverflow()
|
||||
bool removeOverflow(size_t more_size = 0)
|
||||
{
|
||||
size_t current_weight_lost = 0;
|
||||
size_t queue_size = cells.size();
|
||||
auto key_it = queue.begin();
|
||||
|
||||
while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1))
|
||||
while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size))
|
||||
&& (queue_size > 1)
|
||||
&& key_it != queue.end())
|
||||
{
|
||||
const Key & key = queue.front();
|
||||
const Key & key = *key_it;
|
||||
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
@ -326,13 +377,33 @@ private:
|
||||
}
|
||||
|
||||
const auto & cell = it->second;
|
||||
auto evict_status = evict_policy.canRelease(*cell.value);// in default, it is CAN_EVITCT
|
||||
if (evict_status == LRUCacheEvictStatus::CAN_EVITCT)
|
||||
{
|
||||
// always call release() before erasing an element
|
||||
// in default, it's an empty action
|
||||
evict_policy.release(*cell.value);
|
||||
|
||||
current_size -= cell.size;
|
||||
current_weight_lost += cell.size;
|
||||
current_size -= cell.size;
|
||||
current_weight_lost += cell.size;
|
||||
|
||||
cells.erase(it);
|
||||
queue.pop_front();
|
||||
--queue_size;
|
||||
cells.erase(it);
|
||||
queue.pop_front();
|
||||
--queue_size;
|
||||
}
|
||||
else if (evict_status == LRUCacheEvictStatus::SKIP_EVICT)
|
||||
{
|
||||
// skip this element and try to evict the remaining ones.
|
||||
key_it++;
|
||||
continue;
|
||||
}
|
||||
else if (evict_status == LRUCacheEvictStatus::TERMINATE_EVICT)
|
||||
{
|
||||
// maybe we want to stop this iteration once we meet the first unreleasable element
|
||||
break;
|
||||
}
|
||||
LOG_ERROR(&Poco::Logger::get("LRUCache"), "This condition branch should not be reached.");
|
||||
abort();
|
||||
}
|
||||
|
||||
onRemoveOverflowWeightLoss(current_weight_lost);
|
||||
@ -342,6 +413,7 @@ private:
|
||||
LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
return !(current_size + more_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size));
|
||||
}
|
||||
|
||||
/// Override this method if you want to track how much weight was lost in removeOverflow method.
|
||||
|
@ -1,283 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <list>
|
||||
#include <memory>
|
||||
#include <chrono>
|
||||
#include <mutex>
|
||||
#include <atomic>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
struct TrivialUnreleasableLRUCacheWeightFunction
|
||||
{
|
||||
size_t operator()(const T &) const
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
};
|
||||
|
||||
enum class CacheEvictStatus
|
||||
{
|
||||
CAN_EVITCT, // a key can be evicted
|
||||
TERMINATE_EVICT, // stop the evicting process
|
||||
SKIP_EVICT, // skip current value and keep iterating
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct TrivialUnreleasableLRUCacheEvitPolicy
|
||||
{
|
||||
CacheEvictStatus canRelease(const T &)
|
||||
{
|
||||
return CacheEvictStatus::CAN_EVITCT;
|
||||
}
|
||||
|
||||
void release(T & )
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
/*
|
||||
* Another version LRU Cache。
|
||||
* A value can only be evicted or be updated if it is releasable. If there is no releasable value,
|
||||
* insert or update will fail.
|
||||
*/
|
||||
template <typename TKey,
|
||||
typename TMapped,
|
||||
typename HashFunction = std::hash<TKey>,
|
||||
typename WeightFunction = TrivialUnreleasableLRUCacheWeightFunction<TMapped>,
|
||||
typename EvictPolicy = TrivialUnreleasableLRUCacheEvitPolicy<TMapped>>
|
||||
class UnreleasableLRUCache
|
||||
{
|
||||
public:
|
||||
using Key = TKey;
|
||||
using Mapped = TMapped;
|
||||
using MappedPtr = std::shared_ptr<Mapped>;
|
||||
|
||||
/** Initialize LRUCache with max_size and max_elements_size.
|
||||
* max_elements_size == 0 means no elements size restrictions.
|
||||
*/
|
||||
UnreleasableLRUCache(size_t max_size_, size_t max_elements_size_ = 0)
|
||||
: max_size(std::max(static_cast<size_t>(1), max_size_))
|
||||
, max_elements_size(max_elements_size_)
|
||||
{}
|
||||
|
||||
MappedPtr get(const Key & key)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto res = getImpl(key, lock);
|
||||
if (res)
|
||||
++hits;
|
||||
else
|
||||
++misses;
|
||||
return res;
|
||||
}
|
||||
|
||||
/*
|
||||
* Fail on two cases
|
||||
* 1) the key exists, but the old value is not releasable
|
||||
* 2) the key not exists, but there is not enough space for it after trying to evict some least recently used values.
|
||||
*/
|
||||
bool set(const Key & key, const MappedPtr & mapped)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
return setImpl(key, mapped, lock);
|
||||
}
|
||||
|
||||
void getStats(size_t & out_hits, size_t & out_misses) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
out_hits = hits;
|
||||
out_misses = misses;
|
||||
}
|
||||
|
||||
size_t weight() const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return current_size;
|
||||
}
|
||||
|
||||
size_t count() const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return cells.size();
|
||||
}
|
||||
|
||||
size_t maxSize() const
|
||||
{
|
||||
return max_size;
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
queue.clear();
|
||||
cells.clear();
|
||||
current_size = 0;
|
||||
hits = 0;
|
||||
misses = 0;
|
||||
}
|
||||
|
||||
virtual ~UnreleasableLRUCache() {}
|
||||
|
||||
protected:
|
||||
using LRUQueue = std::list<Key>;
|
||||
using LRUQueueIterator = typename LRUQueue::iterator;
|
||||
|
||||
struct Cell
|
||||
{
|
||||
MappedPtr value;
|
||||
size_t size;
|
||||
LRUQueueIterator queue_iterator;
|
||||
};
|
||||
|
||||
using Cells = std::unordered_map<Key, Cell, HashFunction>;
|
||||
|
||||
Cells cells;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
private:
|
||||
|
||||
LRUQueue queue;
|
||||
|
||||
/// Total weight of values.
|
||||
size_t current_size = 0;
|
||||
const size_t max_size;
|
||||
const size_t max_elements_size;
|
||||
|
||||
std::atomic<size_t> hits {0};
|
||||
std::atomic<size_t> misses {0};
|
||||
|
||||
WeightFunction weight_function;
|
||||
EvictPolicy evict_policy;
|
||||
|
||||
MappedPtr getImpl(const Key & key, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
{
|
||||
return MappedPtr();
|
||||
}
|
||||
|
||||
Cell & cell = it->second;
|
||||
|
||||
/// Move the key to the end of the queue. The iterator remains valid.
|
||||
queue.splice(queue.end(), queue, cell.queue_iterator);
|
||||
|
||||
return cell.value;
|
||||
}
|
||||
|
||||
bool setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
auto [it, inserted] = cells.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(key),
|
||||
std::forward_as_tuple());
|
||||
|
||||
Cell & cell = it->second;
|
||||
|
||||
if (inserted)
|
||||
{
|
||||
auto weight = mapped ? weight_function(*mapped) : 0;
|
||||
if (!removeOverflow(weight))
|
||||
{
|
||||
// cannot insert this new value
|
||||
cells.erase(it);
|
||||
return false;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
cell.queue_iterator = queue.insert(queue.end(), key);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
cells.erase(it);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (evict_policy.canRelease(*cell.value) != CacheEvictStatus::CAN_EVITCT)
|
||||
{
|
||||
// the old value is not releasable
|
||||
return false;
|
||||
}
|
||||
evict_policy.release(*cell.value);
|
||||
current_size -= cell.size;
|
||||
queue.splice(queue.end(), queue, cell.queue_iterator);
|
||||
}
|
||||
|
||||
cell.value = mapped;
|
||||
cell.size = cell.value ? weight_function(*cell.value) : 0;
|
||||
current_size += cell.size;
|
||||
|
||||
removeOverflow(0);
|
||||
return true;
|
||||
}
|
||||
|
||||
// Or make your own implementation
|
||||
virtual bool removeOverflow(size_t more_size)
|
||||
{
|
||||
size_t current_weight_lost = 0;
|
||||
size_t queue_size = cells.size();
|
||||
|
||||
auto key_it = queue.begin();
|
||||
while ((current_size + more_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1)
|
||||
&& key_it != queue.end())
|
||||
{
|
||||
const Key & key = *key_it;
|
||||
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("UnreleasableLRUCache"), "UnreleasableLRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
|
||||
const auto & cell = it->second;
|
||||
auto cache_evict_status = evict_policy.canRelease(*(cell.value));
|
||||
if (cache_evict_status == CacheEvictStatus::CAN_EVITCT)
|
||||
{
|
||||
evict_policy.release(*(cell.value));
|
||||
current_size -= cell.size;
|
||||
current_weight_lost += cell.size;
|
||||
|
||||
cells.erase(it);
|
||||
key_it = queue.erase(key_it);
|
||||
--queue_size;
|
||||
}
|
||||
else if (cache_evict_status == CacheEvictStatus::SKIP_EVICT)
|
||||
{
|
||||
key_it++;
|
||||
continue;
|
||||
}
|
||||
else if (cache_evict_status == CacheEvictStatus::TERMINATE_EVICT)
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
onRemoveOverflowWeightLoss(current_weight_lost);
|
||||
|
||||
if (current_size > (1ull << 63))
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("UnreleasableLRUCache"), "UnreleasableLRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
return !(current_size + more_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size));
|
||||
}
|
||||
|
||||
/// Override this method if you want to track how much weight was lost in removeOverflow method.
|
||||
virtual void onRemoveOverflowWeightLoss(size_t /*weight_loss*/) {}
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -7,7 +7,7 @@
|
||||
#include <filesystem>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Common/UnreleasableLRUCache.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
@ -162,11 +162,11 @@ struct RemoteFileCacheWeightFunction
|
||||
|
||||
struct RemoteFileCacheEvictPolicy
|
||||
{
|
||||
CacheEvictStatus canRelease(RemoteCacheController & cache) const
|
||||
LRUCacheEvictStatus canRelease(RemoteCacheController & cache) const
|
||||
{
|
||||
if (cache.closable())
|
||||
return CacheEvictStatus::CAN_EVITCT;
|
||||
return CacheEvictStatus::SKIP_EVICT;
|
||||
return LRUCacheEvictStatus::CAN_EVITCT;
|
||||
return LRUCacheEvictStatus::SKIP_EVICT;
|
||||
}
|
||||
void release(RemoteCacheController & cache)
|
||||
{
|
||||
@ -177,7 +177,7 @@ struct RemoteFileCacheEvictPolicy
|
||||
class RemoteReadBufferCache
|
||||
{
|
||||
public:
|
||||
using CacheType = UnreleasableLRUCache<String, RemoteCacheController, std::hash<String>,
|
||||
using CacheType = LRUCache<String, RemoteCacheController, std::hash<String>,
|
||||
RemoteFileCacheWeightFunction, RemoteFileCacheEvictPolicy>;
|
||||
~RemoteReadBufferCache();
|
||||
// global instance
|
||||
|
Loading…
Reference in New Issue
Block a user