2014-02-11 11:05:02 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <unordered_map>
|
|
|
|
#include <list>
|
|
|
|
#include <memory>
|
2015-05-07 12:28:09 +00:00
|
|
|
#include <chrono>
|
2016-05-28 10:15:36 +00:00
|
|
|
#include <mutex>
|
2017-01-31 12:17:42 +00:00
|
|
|
|
2015-12-16 20:50:56 +00:00
|
|
|
#include <common/logger_useful.h>
|
|
|
|
|
2014-02-11 11:05:02 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
struct TrivialWeightFunction
|
|
|
|
{
|
|
|
|
size_t operator()(const T & x) const
|
|
|
|
{
|
|
|
|
return 1;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2017-01-31 12:27:13 +00:00
|
|
|
|
|
|
|
/// Thread-safe cache that evicts entries which are not used for a long time or are expired.
|
|
|
|
/// 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 and when expiration time of these
|
|
|
|
/// entries is due.
|
|
|
|
/// Value weight should not change after insertion.
|
2015-05-07 12:28:09 +00:00
|
|
|
template <typename TKey, typename TMapped, typename HashFunction = std::hash<TMapped>, typename WeightFunction = TrivialWeightFunction<TMapped> >
|
2014-02-11 11:05:02 +00:00
|
|
|
class LRUCache
|
|
|
|
{
|
|
|
|
public:
|
2015-05-07 10:31:50 +00:00
|
|
|
using Key = TKey;
|
|
|
|
using Mapped = TMapped;
|
|
|
|
using MappedPtr = std::shared_ptr<Mapped>;
|
2015-05-07 12:28:09 +00:00
|
|
|
using Delay = std::chrono::seconds;
|
2015-05-07 10:31:50 +00:00
|
|
|
|
|
|
|
private:
|
2015-05-07 12:28:09 +00:00
|
|
|
using Clock = std::chrono::steady_clock;
|
|
|
|
using Timestamp = Clock::time_point;
|
2015-05-07 10:31:50 +00:00
|
|
|
|
|
|
|
public:
|
2015-05-07 12:28:09 +00:00
|
|
|
LRUCache(size_t max_size_, const Delay & expiration_delay_ = Delay::zero())
|
2015-05-07 10:31:50 +00:00
|
|
|
: max_size(std::max(1ul, max_size_)), expiration_delay(expiration_delay_) {}
|
2014-02-11 11:05:02 +00:00
|
|
|
|
|
|
|
MappedPtr get(const Key & key)
|
|
|
|
{
|
2016-05-28 10:15:36 +00:00
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
2014-02-11 11:05:02 +00:00
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
auto res = getImpl(key, lock);
|
|
|
|
if (res)
|
|
|
|
++hits;
|
|
|
|
else
|
2014-02-11 11:05:02 +00:00
|
|
|
++misses;
|
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
return res;
|
|
|
|
}
|
2015-05-07 10:31:50 +00:00
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
void set(const Key & key, const MappedPtr & mapped)
|
|
|
|
{
|
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
2014-02-11 11:05:02 +00:00
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
setImpl(key, mapped, lock);
|
2014-02-11 11:05:02 +00:00
|
|
|
}
|
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
/// If the value for the key is in the cache, returns it. If it is not, calls load_func() to
|
|
|
|
/// produce it, saves the result in the cache and returns it.
|
|
|
|
/// Only one of several concurrent threads calling getOrSet() will call load_func(),
|
|
|
|
/// others will wait for that call to complete and will use its result (this helps prevent cache stampede).
|
|
|
|
/// Exceptions occuring in load_func will be propagated to the caller. Another thread from the
|
|
|
|
/// set of concurrent threads will then try to call its load_func etc.
|
2017-01-31 17:04:21 +00:00
|
|
|
///
|
|
|
|
/// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call.
|
2017-01-31 12:17:42 +00:00
|
|
|
template<typename LoadFunc>
|
|
|
|
std::pair<MappedPtr, bool> getOrSet(const Key & key, LoadFunc&& load_func)
|
2014-02-11 11:05:02 +00:00
|
|
|
{
|
2017-01-31 12:17:42 +00:00
|
|
|
InsertTokenHolder token_holder;
|
|
|
|
{
|
2017-01-31 17:04:21 +00:00
|
|
|
std::lock_guard<std::mutex> cache_lock(mutex);
|
2014-02-11 11:05:02 +00:00
|
|
|
|
2017-01-31 17:04:21 +00:00
|
|
|
auto val = getImpl(key, cache_lock);
|
2017-01-31 12:17:42 +00:00
|
|
|
if (val)
|
|
|
|
{
|
|
|
|
++hits;
|
|
|
|
return std::make_pair(val, false);
|
|
|
|
}
|
2014-02-20 18:08:23 +00:00
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
auto & token = insert_tokens[key];
|
|
|
|
if (!token)
|
|
|
|
token = std::make_shared<InsertToken>(*this);
|
2014-02-11 11:05:02 +00:00
|
|
|
|
2017-01-31 17:04:21 +00:00
|
|
|
token_holder.acquire(&key, token, cache_lock);
|
2014-02-11 11:05:02 +00:00
|
|
|
}
|
2017-01-31 12:17:42 +00:00
|
|
|
|
|
|
|
InsertToken * token = token_holder.token.get();
|
|
|
|
|
|
|
|
std::lock_guard<std::mutex> token_lock(token->mutex);
|
|
|
|
|
|
|
|
token_holder.cleaned_up = token->cleaned_up;
|
|
|
|
|
|
|
|
if (token->value)
|
2014-02-11 11:05:02 +00:00
|
|
|
{
|
2017-01-31 12:17:42 +00:00
|
|
|
/// Another thread already produced the value while we waited for token->mutex.
|
|
|
|
++hits;
|
|
|
|
return std::make_pair(token->value, false);
|
2014-02-11 11:05:02 +00:00
|
|
|
}
|
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
++misses;
|
|
|
|
token->value = load_func();
|
2014-02-11 11:05:02 +00:00
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
std::lock_guard<std::mutex> cache_lock(mutex);
|
|
|
|
|
2017-01-31 17:04:21 +00:00
|
|
|
/// Insert the new value only if the token is still in present in insert_tokens.
|
|
|
|
/// (The token may be absent because of a concurrent reset() call).
|
|
|
|
auto token_it = insert_tokens.find(key);
|
|
|
|
if (token_it != insert_tokens.end() && token_it->second.get() == token)
|
|
|
|
setImpl(key, token->value, cache_lock);
|
2017-01-31 12:17:42 +00:00
|
|
|
|
|
|
|
if (!token->cleaned_up)
|
|
|
|
token_holder.cleanup(token_lock, cache_lock);
|
|
|
|
|
|
|
|
return std::make_pair(token->value, true);
|
2014-02-11 11:05:02 +00:00
|
|
|
}
|
|
|
|
|
2014-03-10 04:17:17 +00:00
|
|
|
void getStats(size_t & out_hits, size_t & out_misses) const
|
2014-02-11 11:05:02 +00:00
|
|
|
{
|
2016-05-28 10:15:36 +00:00
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
2014-02-11 11:05:02 +00:00
|
|
|
out_hits = hits;
|
|
|
|
out_misses = misses;
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t weight() const
|
|
|
|
{
|
2016-05-28 10:15:36 +00:00
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
2014-02-11 11:05:02 +00:00
|
|
|
return current_size;
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t count() const
|
|
|
|
{
|
2016-05-28 10:15:36 +00:00
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
2014-02-20 18:08:23 +00:00
|
|
|
return cells.size();
|
2014-02-11 11:05:02 +00:00
|
|
|
}
|
2014-02-20 18:08:23 +00:00
|
|
|
|
2014-03-04 19:11:32 +00:00
|
|
|
void reset()
|
|
|
|
{
|
2016-05-28 10:15:36 +00:00
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
2014-03-04 19:11:32 +00:00
|
|
|
queue.clear();
|
|
|
|
cells.clear();
|
2017-01-31 12:17:42 +00:00
|
|
|
insert_tokens.clear();
|
2014-03-04 19:11:32 +00:00
|
|
|
current_size = 0;
|
|
|
|
hits = 0;
|
|
|
|
misses = 0;
|
2014-04-17 15:46:58 +00:00
|
|
|
current_weight_lost = 0;
|
2014-03-04 19:11:32 +00:00
|
|
|
}
|
|
|
|
|
2014-04-17 15:46:58 +00:00
|
|
|
protected:
|
2017-01-31 12:27:13 +00:00
|
|
|
/// Total weight of evicted values. This value is reset every time it is sent to profile events.
|
2014-04-17 15:46:58 +00:00
|
|
|
size_t current_weight_lost = 0;
|
2017-01-31 12:27:13 +00:00
|
|
|
|
2014-02-11 11:05:02 +00:00
|
|
|
private:
|
2017-01-31 12:17:42 +00:00
|
|
|
|
|
|
|
/// Represents pending insertion attempt.
|
|
|
|
struct InsertToken
|
|
|
|
{
|
|
|
|
InsertToken(LRUCache & cache_) : cache(cache_) {}
|
|
|
|
|
|
|
|
std::mutex mutex;
|
|
|
|
bool cleaned_up = false; /// Protected by the token mutex
|
|
|
|
MappedPtr value; /// Protected by the token mutex
|
|
|
|
|
|
|
|
LRUCache & cache;
|
|
|
|
size_t refcount = 0; /// Protected by the cache mutex
|
|
|
|
};
|
|
|
|
|
|
|
|
using InsertTokenById = std::unordered_map<Key, std::shared_ptr<InsertToken>, HashFunction>;
|
|
|
|
|
|
|
|
/// This class is responsible for removing used insert tokens from the insert_tokens map.
|
|
|
|
/// Among several concurrent threads the first successful one is responsible for removal. But if they all
|
|
|
|
/// fail, then the last one is responsible.
|
|
|
|
struct InsertTokenHolder
|
|
|
|
{
|
|
|
|
const Key * key = nullptr;
|
|
|
|
std::shared_ptr<InsertToken> token;
|
|
|
|
bool cleaned_up = false;
|
|
|
|
|
|
|
|
InsertTokenHolder() = default;
|
|
|
|
|
|
|
|
void acquire(const Key * key_, const std::shared_ptr<InsertToken> & token_, std::lock_guard<std::mutex> & cache_lock)
|
|
|
|
{
|
|
|
|
key = key_;
|
|
|
|
token = token_;
|
|
|
|
++token->refcount;
|
|
|
|
}
|
|
|
|
|
|
|
|
void cleanup(std::lock_guard<std::mutex> & token_lock, std::lock_guard<std::mutex> & cache_lock)
|
|
|
|
{
|
2017-01-31 17:04:21 +00:00
|
|
|
token->cache.insert_tokens.erase(*key);
|
2017-01-31 12:17:42 +00:00
|
|
|
token->cleaned_up = true;
|
|
|
|
cleaned_up = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
~InsertTokenHolder()
|
|
|
|
{
|
|
|
|
if (!token)
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (cleaned_up)
|
|
|
|
return;
|
|
|
|
|
|
|
|
std::lock_guard<std::mutex> token_lock(token->mutex);
|
|
|
|
|
|
|
|
if (token->cleaned_up)
|
|
|
|
return;
|
|
|
|
|
|
|
|
std::lock_guard<std::mutex> cache_lock(token->cache.mutex);
|
|
|
|
|
|
|
|
--token->refcount;
|
|
|
|
if (token->refcount == 0)
|
|
|
|
cleanup(token_lock, cache_lock);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
friend struct InsertTokenHolder;
|
|
|
|
|
2015-05-07 10:31:50 +00:00
|
|
|
using LRUQueue = std::list<Key>;
|
|
|
|
using LRUQueueIterator = typename LRUQueue::iterator;
|
2014-02-11 11:05:02 +00:00
|
|
|
|
|
|
|
struct Cell
|
|
|
|
{
|
2015-05-07 12:28:09 +00:00
|
|
|
bool expired(const Timestamp & last_timestamp, const Delay & expiration_delay) const
|
|
|
|
{
|
2015-05-07 12:28:41 +00:00
|
|
|
return (expiration_delay == Delay::zero()) ||
|
2015-05-07 12:28:09 +00:00
|
|
|
((last_timestamp > timestamp) && ((last_timestamp - timestamp) > expiration_delay));
|
|
|
|
}
|
|
|
|
|
2014-02-11 11:05:02 +00:00
|
|
|
MappedPtr value;
|
|
|
|
size_t size;
|
|
|
|
LRUQueueIterator queue_iterator;
|
2015-05-07 12:28:09 +00:00
|
|
|
Timestamp timestamp;
|
2014-02-11 11:05:02 +00:00
|
|
|
};
|
|
|
|
|
2015-05-07 10:31:50 +00:00
|
|
|
using Cells = std::unordered_map<Key, Cell, HashFunction>;
|
2014-02-11 11:05:02 +00:00
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
InsertTokenById insert_tokens;
|
|
|
|
|
2014-02-11 11:05:02 +00:00
|
|
|
LRUQueue queue;
|
|
|
|
Cells cells;
|
2014-02-20 18:08:23 +00:00
|
|
|
|
2017-01-31 12:27:13 +00:00
|
|
|
/// Total weight of values.
|
2014-02-20 18:08:23 +00:00
|
|
|
size_t current_size = 0;
|
|
|
|
const size_t max_size;
|
2015-05-07 10:31:50 +00:00
|
|
|
const Delay expiration_delay;
|
2014-02-11 11:05:02 +00:00
|
|
|
|
2016-05-28 10:15:36 +00:00
|
|
|
mutable std::mutex mutex;
|
2014-02-20 18:08:23 +00:00
|
|
|
size_t hits = 0;
|
|
|
|
size_t misses = 0;
|
2014-02-11 11:05:02 +00:00
|
|
|
|
|
|
|
WeightFunction weight_function;
|
|
|
|
|
2017-01-31 12:17:42 +00:00
|
|
|
MappedPtr getImpl(const Key & key, std::lock_guard<std::mutex> & cache_lock)
|
|
|
|
{
|
|
|
|
auto it = cells.find(key);
|
|
|
|
if (it == cells.end())
|
|
|
|
{
|
|
|
|
return MappedPtr();
|
|
|
|
}
|
|
|
|
|
|
|
|
Cell & cell = it->second;
|
|
|
|
updateCellTimestamp(cell);
|
|
|
|
|
2017-01-31 12:27:13 +00:00
|
|
|
/// Move the key to the end of the queue. The iterator remains valid.
|
2017-01-31 12:17:42 +00:00
|
|
|
queue.splice(queue.end(), queue, cell.queue_iterator);
|
|
|
|
|
|
|
|
return cell.value;
|
|
|
|
}
|
|
|
|
|
|
|
|
void setImpl(const Key & key, const MappedPtr & mapped, std::lock_guard<std::mutex> & cache_lock)
|
|
|
|
{
|
|
|
|
auto res = cells.emplace(std::piecewise_construct,
|
|
|
|
std::forward_as_tuple(key),
|
|
|
|
std::forward_as_tuple());
|
|
|
|
|
|
|
|
Cell & cell = res.first->second;
|
|
|
|
bool inserted = res.second;
|
|
|
|
|
|
|
|
if (inserted)
|
|
|
|
{
|
|
|
|
cell.queue_iterator = queue.insert(queue.end(), key);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
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;
|
|
|
|
updateCellTimestamp(cell);
|
|
|
|
|
|
|
|
removeOverflow(cell.timestamp);
|
|
|
|
}
|
|
|
|
|
2015-05-07 12:28:09 +00:00
|
|
|
void updateCellTimestamp(Cell & cell)
|
|
|
|
{
|
|
|
|
if (expiration_delay != Delay::zero())
|
|
|
|
cell.timestamp = Clock::now();
|
|
|
|
}
|
2015-05-07 10:31:50 +00:00
|
|
|
|
|
|
|
void removeOverflow(const Timestamp & last_timestamp)
|
2014-02-11 11:05:02 +00:00
|
|
|
{
|
2014-02-20 18:08:23 +00:00
|
|
|
size_t queue_size = cells.size();
|
2015-05-07 10:31:50 +00:00
|
|
|
while ((current_size > max_size) && (queue_size > 1))
|
2014-02-11 11:05:02 +00:00
|
|
|
{
|
|
|
|
const Key & key = queue.front();
|
2015-05-07 10:31:50 +00:00
|
|
|
|
2014-02-20 18:08:23 +00:00
|
|
|
auto it = cells.find(key);
|
2015-05-07 10:31:50 +00:00
|
|
|
if (it == cells.end())
|
2015-12-16 20:50:56 +00:00
|
|
|
{
|
|
|
|
LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
|
|
|
abort();
|
|
|
|
}
|
2015-05-07 10:31:50 +00:00
|
|
|
|
|
|
|
const auto & cell = it->second;
|
2015-05-07 12:28:09 +00:00
|
|
|
if (!cell.expired(last_timestamp, expiration_delay))
|
2015-05-07 10:31:50 +00:00
|
|
|
break;
|
|
|
|
|
|
|
|
current_size -= cell.size;
|
|
|
|
current_weight_lost += cell.size;
|
|
|
|
|
2014-02-11 11:05:02 +00:00
|
|
|
cells.erase(it);
|
|
|
|
queue.pop_front();
|
2014-02-20 18:08:23 +00:00
|
|
|
--queue_size;
|
2014-02-11 11:05:02 +00:00
|
|
|
}
|
|
|
|
|
2014-02-20 18:08:23 +00:00
|
|
|
if (current_size > (1ull << 63))
|
2014-02-11 11:05:02 +00:00
|
|
|
{
|
2015-12-16 20:50:56 +00:00
|
|
|
LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
|
|
|
abort();
|
2014-02-11 11:05:02 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
}
|