2014-02-11 11:05:02 +00:00
|
|
|
|
#pragma once
|
|
|
|
|
|
|
|
|
|
#include <unordered_map>
|
|
|
|
|
#include <list>
|
|
|
|
|
#include <memory>
|
|
|
|
|
#include <Poco/ScopedLock.h>
|
|
|
|
|
#include <Poco/Mutex.h>
|
|
|
|
|
#include <DB/Core/ErrorCodes.h>
|
|
|
|
|
#include <DB/Core/Exception.h>
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
|
struct TrivialWeightFunction
|
|
|
|
|
{
|
|
|
|
|
size_t operator()(const T & x) const
|
|
|
|
|
{
|
|
|
|
|
return 1;
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
/** Кеш, вытесняющий долго не использовавшиеся записи. thread-safe.
|
|
|
|
|
* WeightFunction - тип, оператор () которого принимает Mapped и возвращает "вес" (примерный размер) этого значения.
|
|
|
|
|
* Кеш начинает выбрасывать значения, когда их суммарный вес превышает max_size.
|
|
|
|
|
* После вставки значения его вес не должен меняться.
|
|
|
|
|
*/
|
|
|
|
|
template <typename TKey, typename TMapped, typename HashFunction = std::hash<TMapped>, typename WeightFunction = TrivialWeightFunction<TMapped> >
|
|
|
|
|
class LRUCache
|
|
|
|
|
{
|
|
|
|
|
public:
|
|
|
|
|
typedef TKey Key;
|
|
|
|
|
typedef TMapped Mapped;
|
|
|
|
|
typedef std::shared_ptr<Mapped> MappedPtr;
|
|
|
|
|
|
|
|
|
|
LRUCache(size_t max_size_)
|
2014-02-20 18:08:23 +00:00
|
|
|
|
: max_size(std::max(1ul, max_size_)) {}
|
2014-02-11 11:05:02 +00:00
|
|
|
|
|
|
|
|
|
MappedPtr get(const Key & key)
|
|
|
|
|
{
|
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
|
|
|
|
|
2014-02-20 18:08:23 +00:00
|
|
|
|
auto it = cells.find(key);
|
2014-02-11 11:05:02 +00:00
|
|
|
|
if (it == cells.end())
|
|
|
|
|
{
|
|
|
|
|
++misses;
|
|
|
|
|
return MappedPtr();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
++hits;
|
|
|
|
|
Cell & cell = it->second;
|
|
|
|
|
|
|
|
|
|
/// Переместим ключ в конец очереди. Итератор остается валидным.
|
|
|
|
|
queue.splice(queue.end(), queue, cell.queue_iterator);
|
|
|
|
|
|
|
|
|
|
return cell.value;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void set(const Key & key, MappedPtr mapped)
|
|
|
|
|
{
|
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
|
|
|
|
|
2014-02-20 18:08:23 +00:00
|
|
|
|
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;
|
2014-02-11 11:05:02 +00:00
|
|
|
|
|
|
|
|
|
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;
|
|
|
|
|
|
|
|
|
|
removeOverflow();
|
|
|
|
|
}
|
|
|
|
|
|
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
|
|
|
|
{
|
2014-03-09 13:20:19 +00:00
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
2014-02-11 11:05:02 +00:00
|
|
|
|
out_hits = hits;
|
|
|
|
|
out_misses = misses;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
size_t weight() const
|
|
|
|
|
{
|
2014-03-09 13:20:19 +00:00
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
2014-02-11 11:05:02 +00:00
|
|
|
|
return current_size;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
size_t count() const
|
|
|
|
|
{
|
2014-03-09 13:20:19 +00:00
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> 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()
|
|
|
|
|
{
|
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
|
|
|
|
queue.clear();
|
|
|
|
|
cells.clear();
|
|
|
|
|
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:
|
|
|
|
|
size_t current_weight_lost = 0;
|
|
|
|
|
/// Суммарный вес выброшенных из кеша элементов.
|
|
|
|
|
/// Обнуляется каждый раз, когда информация добавляется в Profile events
|
2014-02-11 11:05:02 +00:00
|
|
|
|
private:
|
|
|
|
|
typedef std::list<Key> LRUQueue;
|
|
|
|
|
typedef typename LRUQueue::iterator LRUQueueIterator;
|
|
|
|
|
|
|
|
|
|
struct Cell
|
|
|
|
|
{
|
|
|
|
|
MappedPtr value;
|
|
|
|
|
size_t size;
|
|
|
|
|
LRUQueueIterator queue_iterator;
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
typedef std::unordered_map<Key, Cell, HashFunction> Cells;
|
|
|
|
|
|
|
|
|
|
LRUQueue queue;
|
|
|
|
|
Cells cells;
|
2014-02-20 18:08:23 +00:00
|
|
|
|
|
|
|
|
|
/// Суммарный вес значений.
|
|
|
|
|
size_t current_size = 0;
|
|
|
|
|
const size_t max_size;
|
2014-02-11 11:05:02 +00:00
|
|
|
|
|
2014-03-10 00:46:41 +00:00
|
|
|
|
mutable Poco::FastMutex 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;
|
|
|
|
|
|
|
|
|
|
void removeOverflow()
|
|
|
|
|
{
|
2014-02-20 18:08:23 +00:00
|
|
|
|
size_t queue_size = cells.size();
|
|
|
|
|
while (current_size > max_size && queue_size > 1)
|
2014-02-11 11:05:02 +00:00
|
|
|
|
{
|
|
|
|
|
const Key & key = queue.front();
|
2014-02-20 18:08:23 +00:00
|
|
|
|
auto it = cells.find(key);
|
2014-02-11 11:05:02 +00:00
|
|
|
|
current_size -= it->second.size;
|
2014-04-17 15:46:58 +00:00
|
|
|
|
current_weight_lost += it->second.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
|
|
|
|
{
|
|
|
|
|
queue.clear();
|
|
|
|
|
cells.clear();
|
|
|
|
|
current_size = 0;
|
|
|
|
|
throw Exception("LRUCache became inconsistent. There must be a bug in it. Clearing it for now.",
|
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
}
|