update LRUCache set()

This commit is contained in:
lgbo-ustc 2021-12-07 10:57:05 +08:00 committed by liangjiabiao
parent f083c5594c
commit 0c1733817b
3 changed files with 89 additions and 300 deletions

View File

@ -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.

View File

@ -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*/) {}
};
}

View File

@ -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