ClickHouse/dbms/include/DB/Common/LRUCache.h

209 lines
4.9 KiB
C++
Raw Normal View History

#pragma once
#include <unordered_map>
#include <list>
#include <memory>
2015-05-07 12:28:09 +00:00
#include <chrono>
#include <Poco/ScopedLock.h>
#include <mutex>
2015-10-05 01:35:28 +00:00
#include <DB/Common/Exception.h>
2015-12-16 20:50:56 +00:00
#include <common/logger_useful.h>
namespace DB
{
template <typename T>
struct TrivialWeightFunction
{
size_t operator()(const T & x) const
{
return 1;
}
};
2015-05-07 10:31:50 +00:00
/** Кеш, вытесняющий долго не использовавшиеся и устаревшие записи. thread-safe.
* WeightFunction - тип, оператор () которого принимает Mapped и возвращает "вес" (примерный размер) этого значения.
2015-05-07 10:31:50 +00:00
* Кеш начинает выбрасывать значения, когда их суммарный вес превышает max_size и срок годности этих значений истёк.
* После вставки значения его вес не должен меняться.
*/
2015-05-07 12:28:09 +00:00
template <typename TKey, typename TMapped, typename HashFunction = std::hash<TMapped>, typename WeightFunction = TrivialWeightFunction<TMapped> >
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_) {}
MappedPtr get(const Key & key)
{
std::lock_guard<std::mutex> lock(mutex);
auto it = cells.find(key);
if (it == cells.end())
{
++misses;
return MappedPtr();
}
++hits;
Cell & cell = it->second;
2015-05-07 12:28:09 +00:00
updateCellTimestamp(cell);
2015-05-07 10:31:50 +00:00
/// Переместим ключ в конец очереди. Итератор остается валидным.
queue.splice(queue.end(), queue, cell.queue_iterator);
return cell.value;
}
void set(const Key & key, MappedPtr mapped)
{
std::lock_guard<std::mutex> lock(mutex);
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;
2015-05-07 12:28:09 +00:00
updateCellTimestamp(cell);
2015-05-07 12:28:09 +00:00
removeOverflow(cell.timestamp);
}
2014-03-10 04:17:17 +00:00
void getStats(size_t & out_hits, size_t & out_misses) const
{
std::lock_guard<std::mutex> lock(mutex);
out_hits = hits;
out_misses = misses;
}
size_t weight() const
{
std::lock_guard<std::mutex> lock(mutex);
return current_size;
}
size_t count() const
{
std::lock_guard<std::mutex> lock(mutex);
return cells.size();
}
void reset()
{
std::lock_guard<std::mutex> lock(mutex);
queue.clear();
cells.clear();
current_size = 0;
hits = 0;
misses = 0;
current_weight_lost = 0;
}
protected:
size_t current_weight_lost = 0;
/// Суммарный вес выброшенных из кеша элементов.
/// Обнуляется каждый раз, когда информация добавляется в Profile events
private:
2015-05-07 10:31:50 +00:00
using LRUQueue = std::list<Key>;
using LRUQueueIterator = typename LRUQueue::iterator;
struct Cell
{
2015-05-07 12:28:09 +00:00
public:
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));
}
public:
MappedPtr value;
size_t size;
LRUQueueIterator queue_iterator;
2015-05-07 12:28:09 +00:00
Timestamp timestamp;
};
2015-05-07 10:31:50 +00:00
using Cells = std::unordered_map<Key, Cell, HashFunction>;
LRUQueue queue;
Cells cells;
/// Суммарный вес значений.
size_t current_size = 0;
const size_t max_size;
2015-05-07 10:31:50 +00:00
const Delay expiration_delay;
mutable std::mutex mutex;
size_t hits = 0;
size_t misses = 0;
WeightFunction weight_function;
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)
{
size_t queue_size = cells.size();
2015-05-07 10:31:50 +00:00
while ((current_size > max_size) && (queue_size > 1))
{
const Key & key = queue.front();
2015-05-07 10:31:50 +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;
cells.erase(it);
queue.pop_front();
--queue_size;
}
if (current_size > (1ull << 63))
{
2015-12-16 20:50:56 +00:00
LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
abort();
}
}
};
}