mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
commit
a6cfc7bc3b
@ -558,14 +558,16 @@ void LocalServer::processConfig()
|
||||
global_context->getProcessList().setMaxSize(0);
|
||||
|
||||
/// Size of cache for uncompressed blocks. Zero means disabled.
|
||||
String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", "");
|
||||
size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", 0);
|
||||
if (uncompressed_cache_size)
|
||||
global_context->setUncompressedCache(uncompressed_cache_size);
|
||||
global_context->setUncompressedCache(uncompressed_cache_size, uncompressed_cache_policy);
|
||||
|
||||
/// Size of cache for marks (index of MergeTree family of tables).
|
||||
String mark_cache_policy = config().getString("mark_cache_policy", "");
|
||||
size_t mark_cache_size = config().getUInt64("mark_cache_size", 5368709120);
|
||||
if (mark_cache_size)
|
||||
global_context->setMarkCache(mark_cache_size);
|
||||
global_context->setMarkCache(mark_cache_size, mark_cache_policy);
|
||||
|
||||
/// Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled.
|
||||
size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", 0);
|
||||
|
@ -1362,6 +1362,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
size_t max_cache_size = memory_amount * cache_size_to_ram_max_ratio;
|
||||
|
||||
/// Size of cache for uncompressed blocks. Zero means disabled.
|
||||
String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", "");
|
||||
LOG_INFO(log, "Uncompressed cache policy name {}", uncompressed_cache_policy);
|
||||
size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", 0);
|
||||
if (uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
@ -1369,7 +1371,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory",
|
||||
formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
|
||||
}
|
||||
global_context->setUncompressedCache(uncompressed_cache_size);
|
||||
global_context->setUncompressedCache(uncompressed_cache_size, uncompressed_cache_policy);
|
||||
|
||||
/// Load global settings from default_profile and system_profile.
|
||||
global_context->setDefaultProfiles(config());
|
||||
@ -1388,6 +1390,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
|
||||
/// Size of cache for marks (index of MergeTree family of tables).
|
||||
size_t mark_cache_size = config().getUInt64("mark_cache_size", 5368709120);
|
||||
String mark_cache_policy = config().getString("mark_cache_policy", "");
|
||||
if (!mark_cache_size)
|
||||
LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation.");
|
||||
if (mark_cache_size > max_cache_size)
|
||||
@ -1396,7 +1399,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory",
|
||||
formatReadableSizeWithBinarySuffix(mark_cache_size));
|
||||
}
|
||||
global_context->setMarkCache(mark_cache_size);
|
||||
global_context->setMarkCache(mark_cache_size, mark_cache_policy);
|
||||
|
||||
/// Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled.
|
||||
size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", 0);
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <Access/IAccessStorage.h>
|
||||
#include <base/defines.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <mutex>
|
||||
|
||||
|
||||
@ -63,7 +63,7 @@ private:
|
||||
std::shared_ptr<const Storages> getStoragesInternal() const;
|
||||
|
||||
std::shared_ptr<const Storages> nested_storages TSA_GUARDED_BY(mutex);
|
||||
mutable LRUCache<UUID, Storage> ids_cache TSA_GUARDED_BY(mutex);
|
||||
mutable CacheBase<UUID, Storage> ids_cache TSA_GUARDED_BY(mutex);
|
||||
mutable std::mutex mutex;
|
||||
};
|
||||
|
||||
|
@ -1,11 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <list>
|
||||
#include <memory>
|
||||
#include <chrono>
|
||||
#include <mutex>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ICachePolicy.h>
|
||||
#include <Common/LRUCachePolicy.h>
|
||||
#include <Common/SLRUCachePolicy.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <cassert>
|
||||
#include <chrono>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <base/defines.h>
|
||||
@ -13,43 +18,54 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
struct TrivialWeightFunction
|
||||
namespace ErrorCodes
|
||||
{
|
||||
size_t operator()(const T &) const
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
};
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
|
||||
/// Thread-safe cache that evicts entries which are not used for a long time.
|
||||
/// Thread-safe cache that evicts entries using special cache policy
|
||||
/// (default policy 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>>
|
||||
class LRUCache
|
||||
class CacheBase
|
||||
{
|
||||
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.
|
||||
*/
|
||||
explicit LRUCache(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_)
|
||||
{}
|
||||
CacheBase(size_t max_size, size_t max_elements_size = 0, String cache_policy_name = "", double size_ratio = 0.5)
|
||||
{
|
||||
auto on_weight_loss_function = [&](size_t weight_loss) { onRemoveOverflowWeightLoss(weight_loss); };
|
||||
|
||||
if (cache_policy_name.empty())
|
||||
{
|
||||
cache_policy_name = default_cache_policy_name;
|
||||
}
|
||||
|
||||
if (cache_policy_name == "LRU")
|
||||
{
|
||||
using LRUPolicy = LRUCachePolicy<TKey, TMapped, HashFunction, WeightFunction>;
|
||||
cache_policy = std::make_unique<LRUPolicy>(max_size, max_elements_size, on_weight_loss_function);
|
||||
}
|
||||
else if (cache_policy_name == "SLRU")
|
||||
{
|
||||
using SLRUPolicy = SLRUCachePolicy<TKey, TMapped, HashFunction, WeightFunction>;
|
||||
cache_policy = std::make_unique<SLRUPolicy>(max_size, max_elements_size, size_ratio, on_weight_loss_function);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Undeclared cache policy name: " + cache_policy_name, ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
}
|
||||
|
||||
MappedPtr get(const Key & key)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto res = getImpl(key);
|
||||
auto res = cache_policy->get(key, lock);
|
||||
if (res)
|
||||
++hits;
|
||||
else
|
||||
@ -61,20 +77,7 @@ public:
|
||||
void set(const Key & key, const MappedPtr & mapped)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
setImpl(key, mapped);
|
||||
}
|
||||
|
||||
void remove(const Key & key)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
return;
|
||||
auto & cell = it->second;
|
||||
current_size -= cell.size;
|
||||
queue.erase(cell.queue_iterator);
|
||||
cells.erase(it);
|
||||
cache_policy->set(key, mapped, lock);
|
||||
}
|
||||
|
||||
/// If the value for the key is in the cache, returns it. If it is not, calls load_func() to
|
||||
@ -91,8 +94,7 @@ public:
|
||||
InsertTokenHolder token_holder;
|
||||
{
|
||||
std::lock_guard cache_lock(mutex);
|
||||
|
||||
auto val = getImpl(key);
|
||||
auto val = cache_policy->get(key, cache_lock);
|
||||
if (val)
|
||||
{
|
||||
++hits;
|
||||
@ -130,7 +132,7 @@ public:
|
||||
auto token_it = insert_tokens.find(key);
|
||||
if (token_it != insert_tokens.end() && token_it->second.get() == token)
|
||||
{
|
||||
setImpl(key, token->value);
|
||||
cache_policy->set(key, token->value, cache_lock);
|
||||
result = true;
|
||||
}
|
||||
|
||||
@ -147,64 +149,64 @@ public:
|
||||
out_misses = misses;
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
insert_tokens.clear();
|
||||
hits = 0;
|
||||
misses = 0;
|
||||
cache_policy->reset(lock);
|
||||
}
|
||||
|
||||
void remove(const Key & key)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
cache_policy->remove(key, lock);
|
||||
}
|
||||
|
||||
size_t weight() const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return current_size;
|
||||
return cache_policy->weight(lock);
|
||||
}
|
||||
|
||||
size_t count() const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return cells.size();
|
||||
return cache_policy->count(lock);
|
||||
}
|
||||
|
||||
size_t maxSize() const
|
||||
TSA_NO_THREAD_SAFETY_ANALYSIS // disabled because max_size of cache_policy is a constant parameter
|
||||
{
|
||||
return max_size;
|
||||
return cache_policy->maxSize();
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
queue.clear();
|
||||
cells.clear();
|
||||
insert_tokens.clear();
|
||||
current_size = 0;
|
||||
hits = 0;
|
||||
misses = 0;
|
||||
}
|
||||
|
||||
virtual ~LRUCache() = default;
|
||||
virtual ~CacheBase() = default;
|
||||
|
||||
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 TSA_GUARDED_BY(mutex);
|
||||
|
||||
mutable std::mutex mutex;
|
||||
|
||||
private:
|
||||
using CachePolicy = ICachePolicy<TKey, TMapped, HashFunction, WeightFunction>;
|
||||
|
||||
std::unique_ptr<CachePolicy> cache_policy TSA_GUARDED_BY(mutex);
|
||||
|
||||
inline static const String default_cache_policy_name = "SLRU";
|
||||
|
||||
std::atomic<size_t> hits{0};
|
||||
std::atomic<size_t> misses{0};
|
||||
|
||||
/// Represents pending insertion attempt.
|
||||
struct InsertToken
|
||||
{
|
||||
explicit InsertToken(LRUCache & cache_) : cache(cache_) {}
|
||||
explicit InsertToken(CacheBase & cache_) : cache(cache_) {}
|
||||
|
||||
std::mutex mutex;
|
||||
bool cleaned_up TSA_GUARDED_BY(mutex) = false;
|
||||
MappedPtr value TSA_GUARDED_BY(mutex);
|
||||
|
||||
LRUCache & cache;
|
||||
CacheBase & cache;
|
||||
size_t refcount = 0; /// Protected by the cache mutex
|
||||
};
|
||||
|
||||
@ -221,7 +223,7 @@ private:
|
||||
|
||||
InsertTokenHolder() = default;
|
||||
|
||||
void acquire(const Key * key_, const std::shared_ptr<InsertToken> & token_, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
|
||||
void acquire(const Key * key_, const std::shared_ptr<InsertToken> & token_, std::lock_guard<std::mutex> & /* cache_lock */)
|
||||
TSA_NO_THREAD_SAFETY_ANALYSIS // disabled only because we can't reference the parent-level cache mutex from here
|
||||
{
|
||||
key = key_;
|
||||
@ -229,7 +231,7 @@ private:
|
||||
++token->refcount;
|
||||
}
|
||||
|
||||
void cleanup([[maybe_unused]] std::lock_guard<std::mutex> & token_lock, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
|
||||
void cleanup(std::lock_guard<std::mutex> & /* token_lock */, std::lock_guard<std::mutex> & /* cache_lock */)
|
||||
TSA_NO_THREAD_SAFETY_ANALYSIS // disabled only because we can't reference the parent-level cache mutex from here
|
||||
{
|
||||
token->cache.insert_tokens.erase(*key);
|
||||
@ -260,105 +262,8 @@ private:
|
||||
|
||||
friend struct InsertTokenHolder;
|
||||
|
||||
|
||||
InsertTokenById insert_tokens TSA_GUARDED_BY(mutex);
|
||||
|
||||
LRUQueue queue TSA_GUARDED_BY(mutex);
|
||||
|
||||
/// Total weight of values.
|
||||
size_t current_size TSA_GUARDED_BY(mutex) = 0;
|
||||
const size_t max_size;
|
||||
const size_t max_elements_size;
|
||||
|
||||
std::atomic<size_t> hits {0};
|
||||
std::atomic<size_t> misses {0};
|
||||
|
||||
const WeightFunction weight_function;
|
||||
|
||||
MappedPtr getImpl(const Key & key) TSA_REQUIRES(mutex)
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
void setImpl(const Key & key, const MappedPtr & mapped) TSA_REQUIRES(mutex)
|
||||
{
|
||||
auto [it, inserted] = cells.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(key),
|
||||
std::forward_as_tuple());
|
||||
|
||||
Cell & cell = it->second;
|
||||
|
||||
if (inserted)
|
||||
{
|
||||
try
|
||||
{
|
||||
cell.queue_iterator = queue.insert(queue.end(), key);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
cells.erase(it);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
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();
|
||||
}
|
||||
|
||||
void removeOverflow() TSA_REQUIRES(mutex)
|
||||
{
|
||||
size_t current_weight_lost = 0;
|
||||
size_t queue_size = cells.size();
|
||||
|
||||
while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1))
|
||||
{
|
||||
const Key & key = queue.front();
|
||||
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
|
||||
const auto & cell = it->second;
|
||||
|
||||
current_size -= cell.size;
|
||||
current_weight_lost += cell.size;
|
||||
|
||||
cells.erase(it);
|
||||
queue.pop_front();
|
||||
--queue_size;
|
||||
}
|
||||
|
||||
onRemoveOverflowWeightLoss(current_weight_lost);
|
||||
|
||||
if (current_size > (1ull << 63))
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
}
|
||||
|
||||
/// Override this method if you want to track how much weight was lost in removeOverflow method.
|
||||
virtual void onRemoveOverflowWeightLoss(size_t /*weight_loss*/) {}
|
||||
};
|
@ -4,7 +4,7 @@
|
||||
#include <Common/HashTable/HashTableKeyHolder.h>
|
||||
#include <Common/ColumnsHashingImpl.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <base/unaligned.h>
|
||||
|
||||
@ -188,7 +188,7 @@ public:
|
||||
void set(const DictionaryKey & key, const CachedValuesPtr & mapped) { cache.set(key, mapped); }
|
||||
|
||||
private:
|
||||
using Cache = LRUCache<DictionaryKey, CachedValues, DictionaryKeyHash>;
|
||||
using Cache = CacheBase<DictionaryKey, CachedValues, DictionaryKeyHash>;
|
||||
Cache cache;
|
||||
};
|
||||
|
||||
|
42
src/Common/ICachePolicy.h
Normal file
42
src/Common/ICachePolicy.h
Normal file
@ -0,0 +1,42 @@
|
||||
#pragma once
|
||||
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
template <typename T>
|
||||
struct TrivialWeightFunction
|
||||
{
|
||||
size_t operator()(const T &) const
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
};
|
||||
|
||||
template <typename TKey, typename TMapped, typename HashFunction = std::hash<TKey>, typename WeightFunction = TrivialWeightFunction<TMapped>>
|
||||
class ICachePolicy
|
||||
{
|
||||
public:
|
||||
using Key = TKey;
|
||||
using Mapped = TMapped;
|
||||
using MappedPtr = std::shared_ptr<Mapped>;
|
||||
using OnWeightLossFunction = std::function<void(size_t)>;
|
||||
|
||||
virtual size_t weight(std::lock_guard<std::mutex> & /* cache_lock */) const = 0;
|
||||
virtual size_t count(std::lock_guard<std::mutex> & /* cache_lock */) const = 0;
|
||||
virtual size_t maxSize() const = 0;
|
||||
|
||||
virtual void reset(std::lock_guard<std::mutex> & /* cache_lock */) = 0;
|
||||
virtual void remove(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) = 0;
|
||||
virtual MappedPtr get(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) = 0;
|
||||
virtual void set(const Key & key, const MappedPtr & mapped, std::lock_guard<std::mutex> & /* cache_lock */) = 0;
|
||||
|
||||
virtual ~ICachePolicy() = default;
|
||||
|
||||
protected:
|
||||
OnWeightLossFunction on_weight_loss_function = [](size_t) {};
|
||||
};
|
||||
|
||||
}
|
181
src/Common/LRUCachePolicy.h
Normal file
181
src/Common/LRUCachePolicy.h
Normal file
@ -0,0 +1,181 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/ICachePolicy.h>
|
||||
|
||||
#include <list>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/// Cache policy LRU 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.
|
||||
/// To work with the thread-safe implementation of this class use a class "CacheBase" with first parameter "LRU"
|
||||
/// and next parameters in the same order as in the constructor of the current class.
|
||||
template <typename TKey, typename TMapped, typename HashFunction = std::hash<TKey>, typename WeightFunction = TrivialWeightFunction<TMapped>>
|
||||
class LRUCachePolicy : public ICachePolicy<TKey, TMapped, HashFunction, WeightFunction>
|
||||
{
|
||||
public:
|
||||
using Key = TKey;
|
||||
using Mapped = TMapped;
|
||||
using MappedPtr = std::shared_ptr<Mapped>;
|
||||
|
||||
using Base = ICachePolicy<TKey, TMapped, HashFunction, WeightFunction>;
|
||||
using typename Base::OnWeightLossFunction;
|
||||
|
||||
/** Initialize LRUCachePolicy with max_size and max_elements_size.
|
||||
* max_elements_size == 0 means no elements size restrictions.
|
||||
*/
|
||||
explicit LRUCachePolicy(size_t max_size_, size_t max_elements_size_ = 0, OnWeightLossFunction on_weight_loss_function_ = {})
|
||||
: max_size(std::max(static_cast<size_t>(1), max_size_)), max_elements_size(max_elements_size_)
|
||||
{
|
||||
Base::on_weight_loss_function = on_weight_loss_function_;
|
||||
}
|
||||
|
||||
size_t weight(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
{
|
||||
return current_size;
|
||||
}
|
||||
|
||||
size_t count(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
{
|
||||
return cells.size();
|
||||
}
|
||||
|
||||
size_t maxSize() const override
|
||||
{
|
||||
return max_size;
|
||||
}
|
||||
|
||||
void reset(std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
{
|
||||
queue.clear();
|
||||
cells.clear();
|
||||
current_size = 0;
|
||||
}
|
||||
|
||||
void remove(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
return;
|
||||
auto & cell = it->second;
|
||||
current_size -= cell.size;
|
||||
queue.erase(cell.queue_iterator);
|
||||
cells.erase(it);
|
||||
}
|
||||
|
||||
MappedPtr get(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
void set(const Key & key, const MappedPtr & mapped, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
{
|
||||
auto [it, inserted] = cells.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(key),
|
||||
std::forward_as_tuple());
|
||||
|
||||
Cell & cell = it->second;
|
||||
|
||||
if (inserted)
|
||||
{
|
||||
try
|
||||
{
|
||||
cell.queue_iterator = queue.insert(queue.end(), key);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
cells.erase(it);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
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();
|
||||
}
|
||||
|
||||
protected:
|
||||
using LRUQueue = std::list<Key>;
|
||||
using LRUQueueIterator = typename LRUQueue::iterator;
|
||||
|
||||
LRUQueue queue;
|
||||
|
||||
struct Cell
|
||||
{
|
||||
MappedPtr value;
|
||||
size_t size;
|
||||
LRUQueueIterator queue_iterator;
|
||||
};
|
||||
|
||||
using Cells = std::unordered_map<Key, Cell, HashFunction>;
|
||||
|
||||
Cells cells;
|
||||
|
||||
/// Total weight of values.
|
||||
size_t current_size = 0;
|
||||
const size_t max_size;
|
||||
const size_t max_elements_size;
|
||||
|
||||
WeightFunction weight_function;
|
||||
|
||||
void removeOverflow()
|
||||
{
|
||||
size_t current_weight_lost = 0;
|
||||
size_t queue_size = cells.size();
|
||||
|
||||
while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 0))
|
||||
{
|
||||
const Key & key = queue.front();
|
||||
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
|
||||
const auto & cell = it->second;
|
||||
|
||||
current_size -= cell.size;
|
||||
current_weight_lost += cell.size;
|
||||
|
||||
cells.erase(it);
|
||||
queue.pop_front();
|
||||
--queue_size;
|
||||
}
|
||||
|
||||
Base::on_weight_loss_function(current_weight_lost);
|
||||
|
||||
if (current_size > (1ull << 63))
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -24,7 +24,7 @@ struct TrivialLRUResourceCacheReleaseFunction
|
||||
};
|
||||
|
||||
/**
|
||||
* Similar to implementation in LRUCache.h, but with the difference that keys can
|
||||
* Similar to implementation in LRUCachePolicy.h, but with the difference that keys can
|
||||
* only be evicted when they are releasable. Release state is controlled by this implementation.
|
||||
* get() and getOrSet() methods return a Holder to actual value, which does release() in destructor.
|
||||
*
|
||||
|
251
src/Common/SLRUCachePolicy.h
Normal file
251
src/Common/SLRUCachePolicy.h
Normal file
@ -0,0 +1,251 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/ICachePolicy.h>
|
||||
|
||||
#include <list>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Cache policy SLRU evicts entries which were used only once and are not used for a long time,
|
||||
/// this policy protects entries which were used more then once from a sequential scan.
|
||||
/// 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.
|
||||
/// To work with the thread-safe implementation of this class use a class "CacheBase" with first parameter "SLRU"
|
||||
/// and next parameters in the same order as in the constructor of the current class.
|
||||
template <typename TKey, typename TMapped, typename HashFunction = std::hash<TKey>, typename WeightFunction = TrivialWeightFunction<TMapped>>
|
||||
class SLRUCachePolicy : public ICachePolicy<TKey, TMapped, HashFunction, WeightFunction>
|
||||
{
|
||||
public:
|
||||
using Key = TKey;
|
||||
using Mapped = TMapped;
|
||||
using MappedPtr = std::shared_ptr<Mapped>;
|
||||
|
||||
using Base = ICachePolicy<TKey, TMapped, HashFunction, WeightFunction>;
|
||||
using typename Base::OnWeightLossFunction;
|
||||
|
||||
/** Initialize SLRUCachePolicy with max_size and max_protected_size.
|
||||
* max_protected_size shows how many of the most frequently used entries will not be evicted after a sequential scan.
|
||||
* max_protected_size == 0 means that the default protected size is equal to half of the total max size.
|
||||
*/
|
||||
/// TODO: construct from special struct with cache policy parametrs (also with max_protected_size).
|
||||
SLRUCachePolicy(size_t max_size_, size_t max_elements_size_ = 0, double size_ratio = 0.5, OnWeightLossFunction on_weight_loss_function_ = {})
|
||||
: max_protected_size(max_size_ * std::min(1.0, size_ratio))
|
||||
, max_size(max_size_)
|
||||
, max_elements_size(max_elements_size_)
|
||||
{
|
||||
Base::on_weight_loss_function = on_weight_loss_function_;
|
||||
}
|
||||
|
||||
size_t weight(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
{
|
||||
return current_size;
|
||||
}
|
||||
|
||||
size_t count(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
{
|
||||
return cells.size();
|
||||
}
|
||||
|
||||
size_t maxSize() const override
|
||||
{
|
||||
return max_size;
|
||||
}
|
||||
|
||||
void reset(std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
{
|
||||
cells.clear();
|
||||
probationary_queue.clear();
|
||||
protected_queue.clear();
|
||||
current_size = 0;
|
||||
current_protected_size = 0;
|
||||
}
|
||||
|
||||
void remove(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
return;
|
||||
auto & cell = it->second;
|
||||
current_size -= cell.size;
|
||||
if (cell.is_protected)
|
||||
{
|
||||
current_protected_size -= cell.size;
|
||||
}
|
||||
auto & queue = cell.is_protected ? protected_queue : probationary_queue;
|
||||
queue.erase(cell.queue_iterator);
|
||||
cells.erase(it);
|
||||
}
|
||||
|
||||
MappedPtr get(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
{
|
||||
return MappedPtr();
|
||||
}
|
||||
|
||||
Cell & cell = it->second;
|
||||
|
||||
if (cell.is_protected)
|
||||
{
|
||||
protected_queue.splice(protected_queue.end(), protected_queue, cell.queue_iterator);
|
||||
}
|
||||
else
|
||||
{
|
||||
cell.is_protected = true;
|
||||
current_protected_size += cell.size;
|
||||
protected_queue.splice(protected_queue.end(), probationary_queue, cell.queue_iterator);
|
||||
removeOverflow(protected_queue, max_protected_size, current_protected_size, /*is_protected=*/true);
|
||||
}
|
||||
|
||||
return cell.value;
|
||||
}
|
||||
|
||||
void set(const Key & key, const MappedPtr & mapped, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
{
|
||||
auto [it, inserted] = cells.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(key),
|
||||
std::forward_as_tuple());
|
||||
|
||||
Cell & cell = it->second;
|
||||
|
||||
if (inserted)
|
||||
{
|
||||
try
|
||||
{
|
||||
cell.queue_iterator = probationary_queue.insert(probationary_queue.end(), key);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
cells.erase(it);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
current_size -= cell.size;
|
||||
if (cell.is_protected)
|
||||
{
|
||||
current_protected_size -= cell.size;
|
||||
protected_queue.splice(protected_queue.end(), protected_queue, cell.queue_iterator);
|
||||
}
|
||||
else
|
||||
{
|
||||
cell.is_protected = true;
|
||||
protected_queue.splice(protected_queue.end(), probationary_queue, cell.queue_iterator);
|
||||
}
|
||||
}
|
||||
|
||||
cell.value = mapped;
|
||||
cell.size = cell.value ? weight_function(*cell.value) : 0;
|
||||
current_size += cell.size;
|
||||
current_protected_size += cell.is_protected ? cell.size : 0;
|
||||
|
||||
removeOverflow(protected_queue, max_protected_size, current_protected_size, /*is_protected=*/true);
|
||||
removeOverflow(probationary_queue, max_size, current_size, /*is_protected=*/false);
|
||||
}
|
||||
|
||||
protected:
|
||||
using SLRUQueue = std::list<Key>;
|
||||
using SLRUQueueIterator = typename SLRUQueue::iterator;
|
||||
|
||||
SLRUQueue probationary_queue;
|
||||
SLRUQueue protected_queue;
|
||||
|
||||
struct Cell
|
||||
{
|
||||
bool is_protected = false;
|
||||
MappedPtr value;
|
||||
size_t size;
|
||||
SLRUQueueIterator queue_iterator;
|
||||
};
|
||||
|
||||
using Cells = std::unordered_map<Key, Cell, HashFunction>;
|
||||
|
||||
Cells cells;
|
||||
|
||||
size_t current_protected_size = 0;
|
||||
size_t current_size = 0;
|
||||
const size_t max_protected_size;
|
||||
const size_t max_size;
|
||||
const size_t max_elements_size;
|
||||
|
||||
WeightFunction weight_function;
|
||||
|
||||
void removeOverflow(SLRUQueue & queue, const size_t max_weight_size, size_t & current_weight_size, bool is_protected)
|
||||
{
|
||||
size_t current_weight_lost = 0;
|
||||
size_t queue_size = queue.size();
|
||||
|
||||
std::function<bool()> need_remove;
|
||||
if (is_protected)
|
||||
{
|
||||
/// Check if after remove all elements from probationary part there will be no more than max elements
|
||||
/// in protected queue and weight of all protected elements will be less then max protected weight.
|
||||
/// It's not possible to check only cells.size() > max_elements_size
|
||||
/// because protected elements move to probationary part and still remain in cache.
|
||||
need_remove = [&]()
|
||||
{
|
||||
return ((max_elements_size != 0 && cells.size() - probationary_queue.size() > max_elements_size)
|
||||
|| (current_weight_size > max_weight_size)) && (queue_size > 0);
|
||||
};
|
||||
}
|
||||
else
|
||||
{
|
||||
need_remove = [&]()
|
||||
{
|
||||
return ((max_elements_size != 0 && cells.size() > max_elements_size)
|
||||
|| (current_weight_size > max_weight_size)) && (queue_size > 0);
|
||||
};
|
||||
}
|
||||
|
||||
while (need_remove())
|
||||
{
|
||||
const Key & key = queue.front();
|
||||
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("SLRUCache"), "SLRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
|
||||
auto & cell = it->second;
|
||||
|
||||
current_weight_size -= cell.size;
|
||||
|
||||
if (cell.is_protected)
|
||||
{
|
||||
cell.is_protected = false;
|
||||
probationary_queue.splice(probationary_queue.end(), queue, cell.queue_iterator);
|
||||
}
|
||||
else
|
||||
{
|
||||
current_weight_lost += cell.size;
|
||||
cells.erase(it);
|
||||
queue.pop_front();
|
||||
}
|
||||
|
||||
--queue_size;
|
||||
}
|
||||
|
||||
if (!is_protected)
|
||||
{
|
||||
Base::on_weight_loss_function(current_weight_lost);
|
||||
}
|
||||
|
||||
if (current_size > (1ull << 63))
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("SLRUCache"), "SLRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -1,12 +1,12 @@
|
||||
#include <iomanip>
|
||||
#include <iostream>
|
||||
#include <gtest/gtest.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/CacheBase.h>
|
||||
|
||||
TEST(LRUCache, set)
|
||||
{
|
||||
using SimpleLRUCache = DB::LRUCache<int, int>;
|
||||
auto lru_cache = SimpleLRUCache(10, 10);
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU");
|
||||
lru_cache.set(1, std::make_shared<int>(2));
|
||||
lru_cache.set(2, std::make_shared<int>(3));
|
||||
|
||||
@ -18,8 +18,8 @@ TEST(LRUCache, set)
|
||||
|
||||
TEST(LRUCache, update)
|
||||
{
|
||||
using SimpleLRUCache = DB::LRUCache<int, int>;
|
||||
auto lru_cache = SimpleLRUCache(10, 10);
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU");
|
||||
lru_cache.set(1, std::make_shared<int>(2));
|
||||
lru_cache.set(1, std::make_shared<int>(3));
|
||||
auto val = lru_cache.get(1);
|
||||
@ -29,11 +29,11 @@ TEST(LRUCache, update)
|
||||
|
||||
TEST(LRUCache, get)
|
||||
{
|
||||
using SimpleLRUCache = DB::LRUCache<int, int>;
|
||||
auto lru_cache = SimpleLRUCache(10, 10);
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU");
|
||||
lru_cache.set(1, std::make_shared<int>(2));
|
||||
lru_cache.set(2, std::make_shared<int>(3));
|
||||
SimpleLRUCache::MappedPtr value = lru_cache.get(1);
|
||||
SimpleCacheBase::MappedPtr value = lru_cache.get(1);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_EQ(*value, 2);
|
||||
|
||||
@ -49,8 +49,8 @@ struct ValueWeight
|
||||
|
||||
TEST(LRUCache, evictOnSize)
|
||||
{
|
||||
using SimpleLRUCache = DB::LRUCache<int, size_t>;
|
||||
auto lru_cache = SimpleLRUCache(20, 3);
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t>;
|
||||
auto lru_cache = SimpleCacheBase(/*max_size*/ 20, /*max_elements_size*/ 3, "LRU");
|
||||
lru_cache.set(1, std::make_shared<size_t>(2));
|
||||
lru_cache.set(2, std::make_shared<size_t>(3));
|
||||
lru_cache.set(3, std::make_shared<size_t>(4));
|
||||
@ -65,8 +65,8 @@ TEST(LRUCache, evictOnSize)
|
||||
|
||||
TEST(LRUCache, evictOnWeight)
|
||||
{
|
||||
using SimpleLRUCache = DB::LRUCache<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto lru_cache = SimpleLRUCache(10, 10);
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU");
|
||||
lru_cache.set(1, std::make_shared<size_t>(2));
|
||||
lru_cache.set(2, std::make_shared<size_t>(3));
|
||||
lru_cache.set(3, std::make_shared<size_t>(4));
|
||||
@ -86,8 +86,8 @@ TEST(LRUCache, evictOnWeight)
|
||||
|
||||
TEST(LRUCache, getOrSet)
|
||||
{
|
||||
using SimpleLRUCache = DB::LRUCache<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto lru_cache = SimpleLRUCache(10, 10);
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto lru_cache = SimpleCacheBase(/*max_size*/ 10, /*max_elements_size*/ 10, "LRU");
|
||||
size_t x = 10;
|
||||
auto load_func = [&] { return std::make_shared<size_t>(x); };
|
||||
auto [value, loaded] = lru_cache.getOrSet(1, load_func);
|
||||
|
210
src/Common/tests/gtest_slru_cahce.cpp
Normal file
210
src/Common/tests/gtest_slru_cahce.cpp
Normal file
@ -0,0 +1,210 @@
|
||||
#include <iomanip>
|
||||
#include <iostream>
|
||||
#include <gtest/gtest.h>
|
||||
#include <Common/CacheBase.h>
|
||||
|
||||
TEST(SLRUCache, set)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<int>(2));
|
||||
slru_cache.set(2, std::make_shared<int>(3));
|
||||
|
||||
auto w = slru_cache.weight();
|
||||
auto n = slru_cache.count();
|
||||
ASSERT_EQ(w, 2);
|
||||
ASSERT_EQ(n, 2);
|
||||
}
|
||||
|
||||
TEST(SLRUCache, update)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<int>(2));
|
||||
slru_cache.set(1, std::make_shared<int>(3));
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_TRUE(*value == 3);
|
||||
}
|
||||
|
||||
TEST(SLRUCache, get)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<int>(2));
|
||||
slru_cache.set(2, std::make_shared<int>(3));
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_EQ(*value, 2);
|
||||
|
||||
value = slru_cache.get(2);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_EQ(*value, 3);
|
||||
}
|
||||
|
||||
TEST(SLRUCache, remove)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<int>(2));
|
||||
slru_cache.set(2, std::make_shared<int>(3));
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_EQ(*value, 2);
|
||||
|
||||
slru_cache.remove(2);
|
||||
value = slru_cache.get(2);
|
||||
ASSERT_TRUE(value == nullptr);
|
||||
}
|
||||
|
||||
TEST(SLRUCache, removeFromProtected)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/2, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<int>(2));
|
||||
slru_cache.set(1, std::make_shared<int>(3));
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_EQ(*value, 3);
|
||||
|
||||
slru_cache.remove(1);
|
||||
value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value == nullptr);
|
||||
|
||||
slru_cache.set(1, std::make_shared<int>(4));
|
||||
slru_cache.set(1, std::make_shared<int>(5));
|
||||
|
||||
slru_cache.set(2, std::make_shared<int>(6));
|
||||
slru_cache.set(3, std::make_shared<int>(7));
|
||||
|
||||
value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_EQ(*value, 5);
|
||||
|
||||
value = slru_cache.get(3);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_EQ(*value, 7);
|
||||
|
||||
value = slru_cache.get(2);
|
||||
ASSERT_TRUE(value == nullptr);
|
||||
}
|
||||
|
||||
TEST(SLRUCache, reset)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<int>(2));
|
||||
slru_cache.set(2, std::make_shared<int>(3));
|
||||
|
||||
slru_cache.set(2, std::make_shared<int>(4)); /// add to protected_queue
|
||||
|
||||
slru_cache.reset();
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value == nullptr);
|
||||
|
||||
value = slru_cache.get(2);
|
||||
ASSERT_TRUE(value == nullptr);
|
||||
}
|
||||
|
||||
struct ValueWeight
|
||||
{
|
||||
size_t operator()(const size_t & x) const { return x; }
|
||||
};
|
||||
|
||||
TEST(SLRUCache, evictOnElements)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/1, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<size_t>(2));
|
||||
slru_cache.set(2, std::make_shared<size_t>(3));
|
||||
|
||||
auto n = slru_cache.count();
|
||||
ASSERT_EQ(n, 1);
|
||||
|
||||
auto w = slru_cache.weight();
|
||||
ASSERT_EQ(w, 3);
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value == nullptr);
|
||||
value = slru_cache.get(2);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_TRUE(*value == 3);
|
||||
}
|
||||
|
||||
|
||||
TEST(SLRUCache, evictOnWeight)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<size_t>(2));
|
||||
slru_cache.set(2, std::make_shared<size_t>(3));
|
||||
slru_cache.set(3, std::make_shared<size_t>(4));
|
||||
slru_cache.set(4, std::make_shared<size_t>(5));
|
||||
|
||||
auto n = slru_cache.count();
|
||||
ASSERT_EQ(n, 2);
|
||||
|
||||
auto w = slru_cache.weight();
|
||||
ASSERT_EQ(w, 9);
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value == nullptr);
|
||||
value = slru_cache.get(2);
|
||||
ASSERT_TRUE(value == nullptr);
|
||||
}
|
||||
|
||||
TEST(SLRUCache, evictFromProtectedPart)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<size_t>(2));
|
||||
slru_cache.set(1, std::make_shared<size_t>(2));
|
||||
|
||||
slru_cache.set(2, std::make_shared<size_t>(5));
|
||||
slru_cache.set(2, std::make_shared<size_t>(5));
|
||||
|
||||
slru_cache.set(3, std::make_shared<size_t>(5));
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value == nullptr);
|
||||
}
|
||||
|
||||
TEST(SLRUCache, evictStreamProtected)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
slru_cache.set(1, std::make_shared<size_t>(2));
|
||||
slru_cache.set(1, std::make_shared<size_t>(2));
|
||||
|
||||
slru_cache.set(2, std::make_shared<size_t>(3));
|
||||
slru_cache.set(2, std::make_shared<size_t>(3));
|
||||
|
||||
for (int key = 3; key < 10; ++key)
|
||||
{
|
||||
slru_cache.set(key, std::make_shared<size_t>(1 + key % 5));
|
||||
}
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_EQ(*value, 2);
|
||||
|
||||
value = slru_cache.get(2);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_EQ(*value, 3);
|
||||
}
|
||||
|
||||
TEST(SLRUCache, getOrSet)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto slru_cache = SimpleCacheBase(/*max_size=*/10, /*max_elements_size=*/0, "SLRU", /*size_ratio*/0.5);
|
||||
size_t x = 5;
|
||||
auto load_func = [&] { return std::make_shared<size_t>(x); };
|
||||
auto [value, loaded] = slru_cache.getOrSet(1, load_func);
|
||||
ASSERT_TRUE(value != nullptr);
|
||||
ASSERT_TRUE(*value == 5);
|
||||
}
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <IO/MMappedFile.h>
|
||||
@ -21,10 +21,10 @@ namespace DB
|
||||
/** Cache of opened and mmapped files for reading.
|
||||
* mmap/munmap is heavy operation and better to keep mapped file to subsequent use than to map/unmap every time.
|
||||
*/
|
||||
class MMappedFileCache : public LRUCache<UInt128, MMappedFile, UInt128TrivialHash>
|
||||
class MMappedFileCache : public CacheBase<UInt128, MMappedFile, UInt128TrivialHash>
|
||||
{
|
||||
private:
|
||||
using Base = LRUCache<UInt128, MMappedFile, UInt128TrivialHash>;
|
||||
using Base = CacheBase<UInt128, MMappedFile, UInt128TrivialHash>;
|
||||
|
||||
public:
|
||||
explicit MMappedFileCache(size_t max_size_in_bytes)
|
||||
|
@ -1,10 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
#include <Common/CacheBase.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -36,14 +36,14 @@ struct UncompressedSizeWeightFunction
|
||||
|
||||
/** Cache of decompressed blocks for implementation of CachedCompressedReadBuffer. thread-safe.
|
||||
*/
|
||||
class UncompressedCache : public LRUCache<UInt128, UncompressedCacheCell, UInt128TrivialHash, UncompressedSizeWeightFunction>
|
||||
class UncompressedCache : public CacheBase<UInt128, UncompressedCacheCell, UInt128TrivialHash, UncompressedSizeWeightFunction>
|
||||
{
|
||||
private:
|
||||
using Base = LRUCache<UInt128, UncompressedCacheCell, UInt128TrivialHash, UncompressedSizeWeightFunction>;
|
||||
using Base = CacheBase<UInt128, UncompressedCacheCell, UInt128TrivialHash, UncompressedSizeWeightFunction>;
|
||||
|
||||
public:
|
||||
explicit UncompressedCache(size_t max_size_in_bytes)
|
||||
: Base(max_size_in_bytes) {}
|
||||
explicit UncompressedCache(size_t max_size_in_bytes, const String & uncompressed_cache_policy = "")
|
||||
: Base(max_size_in_bytes, 0, uncompressed_cache_policy) {}
|
||||
|
||||
/// Calculate key from path to file and offset.
|
||||
static UInt128 hash(const String & path_to_file, size_t offset)
|
||||
|
@ -17,7 +17,7 @@
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -54,7 +54,7 @@ public:
|
||||
size_t median_size; // roughly the size we're going to preallocate on each thread
|
||||
};
|
||||
|
||||
using Cache = DB::LRUCache<UInt64, Entry>;
|
||||
using Cache = DB::CacheBase<UInt64, Entry>;
|
||||
using CachePtr = std::shared_ptr<Cache>;
|
||||
using Params = DB::Aggregator::Params::StatsCollectingParams;
|
||||
|
||||
|
@ -1657,14 +1657,14 @@ ProcessList::Element * Context::getProcessListElement() const
|
||||
}
|
||||
|
||||
|
||||
void Context::setUncompressedCache(size_t max_size_in_bytes)
|
||||
void Context::setUncompressedCache(size_t max_size_in_bytes, const String & uncompressed_cache_policy)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (shared->uncompressed_cache)
|
||||
throw Exception("Uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
shared->uncompressed_cache = std::make_shared<UncompressedCache>(max_size_in_bytes);
|
||||
shared->uncompressed_cache = std::make_shared<UncompressedCache>(max_size_in_bytes, uncompressed_cache_policy);
|
||||
}
|
||||
|
||||
|
||||
@ -1683,14 +1683,14 @@ void Context::dropUncompressedCache() const
|
||||
}
|
||||
|
||||
|
||||
void Context::setMarkCache(size_t cache_size_in_bytes)
|
||||
void Context::setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (shared->mark_cache)
|
||||
throw Exception("Mark cache has been already created.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
shared->mark_cache = std::make_shared<MarkCache>(cache_size_in_bytes);
|
||||
shared->mark_cache = std::make_shared<MarkCache>(cache_size_in_bytes, mark_cache_policy);
|
||||
}
|
||||
|
||||
MarkCachePtr Context::getMarkCache() const
|
||||
|
@ -780,12 +780,12 @@ public:
|
||||
void setSystemZooKeeperLogAfterInitializationIfNeeded();
|
||||
|
||||
/// Create a cache of uncompressed blocks of specified size. This can be done only once.
|
||||
void setUncompressedCache(size_t max_size_in_bytes);
|
||||
void setUncompressedCache(size_t max_size_in_bytes, const String & uncompressed_cache_policy);
|
||||
std::shared_ptr<UncompressedCache> getUncompressedCache() const;
|
||||
void dropUncompressedCache() const;
|
||||
|
||||
/// Create a cache of marks of specified size. This can be done only once.
|
||||
void setMarkCache(size_t cache_size_in_bytes);
|
||||
void setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy);
|
||||
std::shared_ptr<MarkCache> getMarkCache() const;
|
||||
void dropMarkCache() const;
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <Common/LRUCache.h>
|
||||
# include <Common/CacheBase.h>
|
||||
# include <Common/HashTable/Hash.h>
|
||||
# include <Interpreters/JIT/CHJIT.h>
|
||||
|
||||
@ -35,10 +35,10 @@ struct CompiledFunctionWeightFunction
|
||||
}
|
||||
};
|
||||
|
||||
class CompiledExpressionCache : public LRUCache<UInt128, CompiledExpressionCacheEntry, UInt128Hash, CompiledFunctionWeightFunction>
|
||||
class CompiledExpressionCache : public CacheBase<UInt128, CompiledExpressionCacheEntry, UInt128Hash, CompiledFunctionWeightFunction>
|
||||
{
|
||||
public:
|
||||
using Base = LRUCache<UInt128, CompiledExpressionCacheEntry, UInt128Hash, CompiledFunctionWeightFunction>;
|
||||
using Base = CacheBase<UInt128, CompiledExpressionCacheEntry, UInt128Hash, CompiledFunctionWeightFunction>;
|
||||
using Base::Base;
|
||||
};
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <shared_mutex>
|
||||
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Interpreters/IJoin.h>
|
||||
@ -69,7 +69,7 @@ private:
|
||||
size_t operator()(const Block & block) const { return block.bytes(); }
|
||||
};
|
||||
|
||||
using Cache = LRUCache<size_t, Block, std::hash<size_t>, BlockByteWeight>;
|
||||
using Cache = CacheBase<size_t, Block, std::hash<size_t>, BlockByteWeight>;
|
||||
|
||||
mutable std::shared_mutex rwlock;
|
||||
std::shared_ptr<TableJoin> table_join;
|
||||
|
@ -6,7 +6,7 @@
|
||||
|
||||
#include <Core/Field.h>
|
||||
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/CacheBase.h>
|
||||
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
@ -777,13 +777,13 @@ private:
|
||||
}
|
||||
|
||||
Poco::URI base_url;
|
||||
LRUCache<uint32_t, avro::ValidSchema> schema_cache;
|
||||
CacheBase<uint32_t, avro::ValidSchema> schema_cache;
|
||||
};
|
||||
|
||||
using ConfluentSchemaRegistry = AvroConfluentRowInputFormat::SchemaRegistry;
|
||||
#define SCHEMA_REGISTRY_CACHE_MAX_SIZE 1000
|
||||
/// Cache of Schema Registry URL -> SchemaRegistry
|
||||
static LRUCache<std::string, ConfluentSchemaRegistry> schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE);
|
||||
static CacheBase<std::string, ConfluentSchemaRegistry> schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE);
|
||||
|
||||
static std::shared_ptr<ConfluentSchemaRegistry> getConfluentSchemaRegistry(const FormatSettings & format_settings)
|
||||
{
|
||||
|
@ -10,7 +10,7 @@
|
||||
#include <ThriftHiveMetastore.h>
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <Common/PoolBase.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
#include <Storages/Hive/HiveFile.h>
|
||||
@ -135,7 +135,7 @@ private:
|
||||
|
||||
void tryCallHiveClient(std::function<void(ThriftHiveMetastoreClientPool::Entry &)> func);
|
||||
|
||||
LRUCache<String, HiveTableMetadata> table_metadata_cache;
|
||||
CacheBase<String, HiveTableMetadata> table_metadata_cache;
|
||||
ThriftHiveMetastoreClientPool client_pool;
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient");
|
||||
|
@ -170,7 +170,7 @@ protected:
|
||||
|
||||
using HiveFilePtr = std::shared_ptr<IHiveFile>;
|
||||
using HiveFiles = std::vector<HiveFilePtr>;
|
||||
using HiveFilesCache = LRUCache<String, IHiveFile>;
|
||||
using HiveFilesCache = CacheBase<String, IHiveFile>;
|
||||
using HiveFilesCachePtr = std::shared_ptr<HiveFilesCache>;
|
||||
|
||||
class HiveTextFile : public IHiveFile
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
@ -34,14 +34,14 @@ struct MarksWeightFunction
|
||||
/** Cache of 'marks' for StorageMergeTree.
|
||||
* Marks is an index structure that addresses ranges in column file, corresponding to ranges of primary key.
|
||||
*/
|
||||
class MarkCache : public LRUCache<UInt128, MarksInCompressedFile, UInt128TrivialHash, MarksWeightFunction>
|
||||
class MarkCache : public CacheBase<UInt128, MarksInCompressedFile, UInt128TrivialHash, MarksWeightFunction>
|
||||
{
|
||||
private:
|
||||
using Base = LRUCache<UInt128, MarksInCompressedFile, UInt128TrivialHash, MarksWeightFunction>;
|
||||
using Base = CacheBase<UInt128, MarksInCompressedFile, UInt128TrivialHash, MarksWeightFunction>;
|
||||
|
||||
public:
|
||||
explicit MarkCache(size_t max_size_in_bytes)
|
||||
: Base(max_size_in_bytes) {}
|
||||
explicit MarkCache(size_t max_size_in_bytes, const String & mark_cache_policy = "")
|
||||
: Base(max_size_in_bytes, 0, mark_cache_policy) {}
|
||||
|
||||
/// Calculate key from path to file and offset.
|
||||
static UInt128 hash(const String & path_to_file)
|
||||
|
@ -11,3 +11,8 @@ offsett
|
||||
numer
|
||||
ue
|
||||
alse
|
||||
nodel
|
||||
ot
|
||||
te
|
||||
fo
|
||||
ba
|
||||
|
2
utils/trace-visualizer/js/d3-gantt.js
vendored
2
utils/trace-visualizer/js/d3-gantt.js
vendored
@ -307,7 +307,7 @@
|
||||
x = d3.scaleLinear()
|
||||
.domain([timeDomainStart, timeDomainEnd])
|
||||
.range([0, width])
|
||||
//.clamp(true); // dosn't work with zoom/pan
|
||||
//.clamp(true); // doesn't work with zoom/pan
|
||||
xZoomed = x;
|
||||
y = d3.scaleBand()
|
||||
.domain([...data.bands])
|
||||
|
@ -26,7 +26,7 @@
|
||||
root.d3.tip = factory(d3, d3)
|
||||
}
|
||||
}(this, function(d3Collection, d3Selection) {
|
||||
// Public - contructs a new tooltip
|
||||
// Public - constructs a new tooltip
|
||||
//
|
||||
// Returns a tip
|
||||
return function() {
|
||||
|
Loading…
Reference in New Issue
Block a user