mirror of https://github.com/facebook/rocksdb.git
Remove prototype FastLRUCache (#10954)
Summary: This was just a stepping stone to what eventually became HyperClockCache, and is now just more code to maintain. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10954 Test Plan: tests updated Reviewed By: akankshamahajan15 Differential Revision: D41310123 Pulled By: pdillinger fbshipit-source-id: 618ee148a1a0a29ee756ba8fe28359617b7cd67c
This commit is contained in:
parent
b55e70357c
commit
32520df1d9
|
@ -653,7 +653,6 @@ set(SOURCES
|
|||
cache/charged_cache.cc
|
||||
cache/clock_cache.cc
|
||||
cache/compressed_secondary_cache.cc
|
||||
cache/fast_lru_cache.cc
|
||||
cache/lru_cache.cc
|
||||
cache/sharded_cache.cc
|
||||
db/arena_wrapped_db_iter.cc
|
||||
|
|
2
TARGETS
2
TARGETS
|
@ -16,7 +16,6 @@ cpp_library_wrapper(name="rocksdb_lib", srcs=[
|
|||
"cache/charged_cache.cc",
|
||||
"cache/clock_cache.cc",
|
||||
"cache/compressed_secondary_cache.cc",
|
||||
"cache/fast_lru_cache.cc",
|
||||
"cache/lru_cache.cc",
|
||||
"cache/sharded_cache.cc",
|
||||
"db/arena_wrapped_db_iter.cc",
|
||||
|
@ -356,7 +355,6 @@ cpp_library_wrapper(name="rocksdb_whole_archive_lib", srcs=[
|
|||
"cache/charged_cache.cc",
|
||||
"cache/clock_cache.cc",
|
||||
"cache/compressed_secondary_cache.cc",
|
||||
"cache/fast_lru_cache.cc",
|
||||
"cache/lru_cache.cc",
|
||||
"cache/sharded_cache.cc",
|
||||
"db/arena_wrapped_db_iter.cc",
|
||||
|
|
|
@ -13,7 +13,6 @@
|
|||
#include <set>
|
||||
#include <sstream>
|
||||
|
||||
#include "cache/fast_lru_cache.h"
|
||||
#include "db/db_impl/db_impl.h"
|
||||
#include "monitoring/histogram.h"
|
||||
#include "port/port.h"
|
||||
|
@ -297,10 +296,6 @@ class CacheBench {
|
|||
cache_ = HyperClockCacheOptions(FLAGS_cache_size, FLAGS_value_bytes,
|
||||
FLAGS_num_shard_bits)
|
||||
.MakeSharedCache();
|
||||
} else if (FLAGS_cache_type == "fast_lru_cache") {
|
||||
cache_ = NewFastLRUCache(
|
||||
FLAGS_cache_size, FLAGS_value_bytes, FLAGS_num_shard_bits,
|
||||
false /*strict_capacity_limit*/, kDefaultCacheMetadataChargePolicy);
|
||||
} else if (FLAGS_cache_type == "lru_cache") {
|
||||
LRUCacheOptions opts(FLAGS_cache_size, FLAGS_num_shard_bits,
|
||||
false /* strict_capacity_limit */,
|
||||
|
|
|
@ -15,15 +15,14 @@
|
|||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "cache/fast_lru_cache.h"
|
||||
#include "cache/lru_cache.h"
|
||||
#include "port/stack_trace.h"
|
||||
#include "test_util/testharness.h"
|
||||
#include "util/coding.h"
|
||||
#include "util/string_util.h"
|
||||
|
||||
// FastLRUCache and HyperClockCache only support 16-byte keys, so some of
|
||||
// the tests originally wrote for LRUCache do not work on the other caches.
|
||||
// HyperClockCache only supports 16-byte keys, so some of the tests
|
||||
// originally written for LRUCache do not work on the other caches.
|
||||
// Those tests were adapted to use 16-byte keys. We kept the original ones.
|
||||
// TODO: Remove the original tests if they ever become unused.
|
||||
|
||||
|
@ -76,7 +75,6 @@ void EraseDeleter2(const Slice& /*key*/, void* value) {
|
|||
|
||||
const std::string kLRU = "lru";
|
||||
const std::string kHyperClock = "hyper_clock";
|
||||
const std::string kFast = "fast";
|
||||
|
||||
} // anonymous namespace
|
||||
|
||||
|
@ -86,7 +84,7 @@ class CacheTest : public testing::TestWithParam<std::string> {
|
|||
static std::string type_;
|
||||
|
||||
static void Deleter(const Slice& key, void* v) {
|
||||
if (type_ == kFast || type_ == kHyperClock) {
|
||||
if (type_ == kHyperClock) {
|
||||
current_->deleted_keys_.push_back(DecodeKey16Bytes(key));
|
||||
} else {
|
||||
current_->deleted_keys_.push_back(DecodeKey32Bits(key));
|
||||
|
@ -126,11 +124,6 @@ class CacheTest : public testing::TestWithParam<std::string> {
|
|||
capacity, estimated_value_size_ /*estimated_value_size*/)
|
||||
.MakeSharedCache();
|
||||
}
|
||||
if (type == kFast) {
|
||||
return NewFastLRUCache(
|
||||
capacity, estimated_value_size_, -1 /*num_shard_bits*/,
|
||||
false /*strict_capacity_limit*/, kDefaultCacheMetadataChargePolicy);
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
@ -153,11 +146,6 @@ class CacheTest : public testing::TestWithParam<std::string> {
|
|||
nullptr /*allocator*/, charge_policy)
|
||||
.MakeSharedCache();
|
||||
}
|
||||
if (type == kFast) {
|
||||
return NewFastLRUCache(capacity, 1 /*estimated_value_size*/,
|
||||
num_shard_bits, strict_capacity_limit,
|
||||
charge_policy);
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
@ -167,7 +155,7 @@ class CacheTest : public testing::TestWithParam<std::string> {
|
|||
// LRUCache doesn't, so the encoding depends on the cache type.
|
||||
std::string EncodeKey(int k) {
|
||||
auto type = GetParam();
|
||||
if (type == kFast || type == kHyperClock) {
|
||||
if (type == kHyperClock) {
|
||||
return EncodeKey16Bytes(k);
|
||||
} else {
|
||||
return EncodeKey32Bits(k);
|
||||
|
@ -176,7 +164,7 @@ class CacheTest : public testing::TestWithParam<std::string> {
|
|||
|
||||
int DecodeKey(const Slice& k) {
|
||||
auto type = GetParam();
|
||||
if (type == kFast || type == kHyperClock) {
|
||||
if (type == kHyperClock) {
|
||||
return DecodeKey16Bytes(k);
|
||||
} else {
|
||||
return DecodeKey32Bits(k);
|
||||
|
@ -733,7 +721,7 @@ TEST_P(CacheTest, ReleaseWithoutErase) {
|
|||
|
||||
TEST_P(CacheTest, SetCapacity) {
|
||||
auto type = GetParam();
|
||||
if (type == kFast || type == kHyperClock) {
|
||||
if (type == kHyperClock) {
|
||||
ROCKSDB_GTEST_BYPASS(
|
||||
"FastLRUCache and HyperClockCache don't support arbitrary capacity "
|
||||
"adjustments.");
|
||||
|
@ -787,14 +775,6 @@ TEST_P(CacheTest, SetCapacity) {
|
|||
}
|
||||
|
||||
TEST_P(LRUCacheTest, SetStrictCapacityLimit) {
|
||||
auto type = GetParam();
|
||||
if (type == kFast) {
|
||||
ROCKSDB_GTEST_BYPASS(
|
||||
"FastLRUCache only supports a limited number of "
|
||||
"inserts beyond "
|
||||
"capacity.");
|
||||
return;
|
||||
}
|
||||
// test1: set the flag to false. Insert more keys than capacity. See if they
|
||||
// all go through.
|
||||
std::shared_ptr<Cache> cache = NewCache(5, 0, false);
|
||||
|
@ -1045,9 +1025,8 @@ TEST_P(CacheTest, GetChargeAndDeleter) {
|
|||
}
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(CacheTestInstance, CacheTest,
|
||||
testing::Values(kLRU, kHyperClock, kFast));
|
||||
INSTANTIATE_TEST_CASE_P(CacheTestInstance, LRUCacheTest,
|
||||
testing::Values(kLRU, kFast));
|
||||
testing::Values(kLRU, kHyperClock));
|
||||
INSTANTIATE_TEST_CASE_P(CacheTestInstance, LRUCacheTest, testing::Values(kLRU));
|
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
|
||||
|
|
|
@ -1,580 +0,0 @@
|
|||
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under both the GPLv2 (found in the
|
||||
// COPYING file in the root directory) and Apache 2.0 License
|
||||
// (found in the LICENSE.Apache file in the root directory).
|
||||
//
|
||||
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style license that can be
|
||||
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||
|
||||
#include "cache/fast_lru_cache.h"
|
||||
|
||||
#include <cassert>
|
||||
#include <cstdint>
|
||||
#include <cstdio>
|
||||
#include <functional>
|
||||
|
||||
#include "monitoring/perf_context_imp.h"
|
||||
#include "monitoring/statistics.h"
|
||||
#include "port/lang.h"
|
||||
#include "util/distributed_mutex.h"
|
||||
#include "util/hash.h"
|
||||
#include "util/math.h"
|
||||
#include "util/random.h"
|
||||
|
||||
namespace ROCKSDB_NAMESPACE {
|
||||
|
||||
namespace fast_lru_cache {
|
||||
|
||||
LRUHandleTable::LRUHandleTable(int hash_bits)
|
||||
: length_bits_(hash_bits),
|
||||
length_bits_mask_((uint32_t{1} << length_bits_) - 1),
|
||||
occupancy_(0),
|
||||
occupancy_limit_(static_cast<uint32_t>((uint32_t{1} << length_bits_) *
|
||||
kStrictLoadFactor)),
|
||||
array_(new LRUHandle[size_t{1} << length_bits_]) {
|
||||
assert(hash_bits <= 32);
|
||||
}
|
||||
|
||||
LRUHandleTable::~LRUHandleTable() {
|
||||
ApplyToEntriesRange([](LRUHandle* h) { h->FreeData(); }, 0, GetTableSize());
|
||||
}
|
||||
|
||||
LRUHandle* LRUHandleTable::Lookup(const Slice& key, uint32_t hash) {
|
||||
int probe = 0;
|
||||
int slot = FindVisibleElement(key, hash, probe, 0);
|
||||
return (slot == -1) ? nullptr : &array_[slot];
|
||||
}
|
||||
|
||||
LRUHandle* LRUHandleTable::Insert(LRUHandle* h, LRUHandle** old) {
|
||||
int probe = 0;
|
||||
int slot = FindVisibleElementOrAvailableSlot(h->key(), h->hash, probe,
|
||||
1 /*displacement*/);
|
||||
*old = nullptr;
|
||||
if (slot == -1) {
|
||||
// TODO(Guido) Don't we need to roll back displacements here?
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
if (array_[slot].IsEmpty() || array_[slot].IsTombstone()) {
|
||||
bool empty = array_[slot].IsEmpty();
|
||||
Assign(slot, h);
|
||||
LRUHandle* new_entry = &array_[slot];
|
||||
if (empty) {
|
||||
// This used to be an empty slot.
|
||||
return new_entry;
|
||||
}
|
||||
// It used to be a tombstone, so there may already be a copy of the
|
||||
// key in the table.
|
||||
slot = FindVisibleElement(h->key(), h->hash, probe, 0 /*displacement*/);
|
||||
if (slot == -1) {
|
||||
// No existing copy of the key.
|
||||
return new_entry;
|
||||
}
|
||||
*old = &array_[slot];
|
||||
return new_entry;
|
||||
} else {
|
||||
// There is an existing copy of the key.
|
||||
*old = &array_[slot];
|
||||
// Find an available slot for the new element.
|
||||
array_[slot].displacements++;
|
||||
slot = FindAvailableSlot(h->key(), probe, 1 /*displacement*/);
|
||||
if (slot == -1) {
|
||||
// No available slots. Roll back displacements.
|
||||
probe = 0;
|
||||
slot = FindVisibleElement(h->key(), h->hash, probe, -1);
|
||||
array_[slot].displacements--;
|
||||
FindAvailableSlot(h->key(), probe, -1);
|
||||
return nullptr;
|
||||
}
|
||||
Assign(slot, h);
|
||||
return &array_[slot];
|
||||
}
|
||||
}
|
||||
|
||||
void LRUHandleTable::Remove(LRUHandle* h) {
|
||||
assert(h->next == nullptr &&
|
||||
h->prev == nullptr); // Already off the LRU list.
|
||||
int probe = 0;
|
||||
FindSlot(
|
||||
h->key(), [&h](LRUHandle* e) { return e == h; }, probe,
|
||||
-1 /*displacement*/);
|
||||
h->SetIsVisible(false);
|
||||
h->SetIsElement(false);
|
||||
occupancy_--;
|
||||
}
|
||||
|
||||
void LRUHandleTable::Assign(int slot, LRUHandle* h) {
|
||||
LRUHandle* dst = &array_[slot];
|
||||
uint32_t disp = dst->displacements;
|
||||
*dst = *h;
|
||||
dst->displacements = disp;
|
||||
dst->SetIsVisible(true);
|
||||
dst->SetIsElement(true);
|
||||
occupancy_++;
|
||||
}
|
||||
|
||||
void LRUHandleTable::Exclude(LRUHandle* h) { h->SetIsVisible(false); }
|
||||
|
||||
int LRUHandleTable::FindVisibleElement(const Slice& key, uint32_t hash,
|
||||
int& probe, int displacement) {
|
||||
return FindSlot(
|
||||
key,
|
||||
[&](LRUHandle* h) { return h->Matches(key, hash) && h->IsVisible(); },
|
||||
probe, displacement);
|
||||
}
|
||||
|
||||
int LRUHandleTable::FindAvailableSlot(const Slice& key, int& probe,
|
||||
int displacement) {
|
||||
return FindSlot(
|
||||
key, [](LRUHandle* h) { return h->IsEmpty() || h->IsTombstone(); }, probe,
|
||||
displacement);
|
||||
}
|
||||
|
||||
int LRUHandleTable::FindVisibleElementOrAvailableSlot(const Slice& key,
|
||||
uint32_t hash, int& probe,
|
||||
int displacement) {
|
||||
return FindSlot(
|
||||
key,
|
||||
[&](LRUHandle* h) {
|
||||
return h->IsEmpty() || h->IsTombstone() ||
|
||||
(h->Matches(key, hash) && h->IsVisible());
|
||||
},
|
||||
probe, displacement);
|
||||
}
|
||||
|
||||
inline int LRUHandleTable::FindSlot(const Slice& key,
|
||||
std::function<bool(LRUHandle*)> cond,
|
||||
int& probe, int displacement) {
|
||||
uint32_t base = ModTableSize(Hash(key.data(), key.size(), kProbingSeed1));
|
||||
uint32_t increment =
|
||||
ModTableSize((Hash(key.data(), key.size(), kProbingSeed2) << 1) | 1);
|
||||
uint32_t current = ModTableSize(base + probe * increment);
|
||||
while (true) {
|
||||
LRUHandle* h = &array_[current];
|
||||
probe++;
|
||||
if (current == base && probe > 1) {
|
||||
// We looped back.
|
||||
return -1;
|
||||
}
|
||||
if (cond(h)) {
|
||||
return current;
|
||||
}
|
||||
if (h->IsEmpty()) {
|
||||
// We check emptyness after the condition, because
|
||||
// the condition may be emptyness.
|
||||
return -1;
|
||||
}
|
||||
h->displacements += displacement;
|
||||
current = ModTableSize(current + increment);
|
||||
}
|
||||
}
|
||||
|
||||
LRUCacheShard::LRUCacheShard(size_t capacity, size_t estimated_value_size,
|
||||
bool strict_capacity_limit,
|
||||
CacheMetadataChargePolicy metadata_charge_policy)
|
||||
: CacheShardBase(metadata_charge_policy),
|
||||
capacity_(capacity),
|
||||
strict_capacity_limit_(strict_capacity_limit),
|
||||
table_(
|
||||
CalcHashBits(capacity, estimated_value_size, metadata_charge_policy)),
|
||||
usage_(0),
|
||||
lru_usage_(0) {
|
||||
// Make empty circular linked list.
|
||||
lru_.next = &lru_;
|
||||
lru_.prev = &lru_;
|
||||
lru_low_pri_ = &lru_;
|
||||
}
|
||||
|
||||
void LRUCacheShard::EraseUnRefEntries() {
|
||||
autovector<LRUHandle> last_reference_list;
|
||||
{
|
||||
DMutexLock l(mutex_);
|
||||
while (lru_.next != &lru_) {
|
||||
LRUHandle* old = lru_.next;
|
||||
// LRU list contains only elements which can be evicted.
|
||||
assert(old->IsVisible() && !old->HasRefs());
|
||||
LRU_Remove(old);
|
||||
table_.Remove(old);
|
||||
assert(usage_ >= old->total_charge);
|
||||
usage_ -= old->total_charge;
|
||||
last_reference_list.push_back(*old);
|
||||
}
|
||||
}
|
||||
|
||||
// Free the entries here outside of mutex for performance reasons.
|
||||
for (auto& h : last_reference_list) {
|
||||
h.FreeData();
|
||||
}
|
||||
}
|
||||
|
||||
void LRUCacheShard::ApplyToSomeEntries(
|
||||
const std::function<void(const Slice& key, void* value, size_t charge,
|
||||
DeleterFn deleter)>& callback,
|
||||
size_t average_entries_per_lock, size_t* state) {
|
||||
// The state is essentially going to be the starting hash, which works
|
||||
// nicely even if we resize between calls because we use upper-most
|
||||
// hash bits for table indexes.
|
||||
DMutexLock l(mutex_);
|
||||
size_t length_bits = table_.GetLengthBits();
|
||||
size_t length = table_.GetTableSize();
|
||||
|
||||
assert(average_entries_per_lock > 0);
|
||||
// Assuming we are called with same average_entries_per_lock repeatedly,
|
||||
// this simplifies some logic (index_end will not overflow).
|
||||
assert(average_entries_per_lock < length || *state == 0);
|
||||
|
||||
size_t index_begin = *state >> (sizeof(size_t) * 8u - length_bits);
|
||||
size_t index_end = index_begin + average_entries_per_lock;
|
||||
if (index_end >= length) {
|
||||
// Going to end
|
||||
index_end = length;
|
||||
*state = SIZE_MAX;
|
||||
} else {
|
||||
*state = index_end << (sizeof(size_t) * 8u - length_bits);
|
||||
}
|
||||
|
||||
table_.ApplyToEntriesRange(
|
||||
[callback,
|
||||
metadata_charge_policy = metadata_charge_policy_](LRUHandle* h) {
|
||||
callback(h->key(), h->value, h->GetCharge(metadata_charge_policy),
|
||||
h->deleter);
|
||||
},
|
||||
index_begin, index_end);
|
||||
}
|
||||
|
||||
void LRUCacheShard::LRU_Remove(LRUHandle* h) {
|
||||
assert(h->next != nullptr);
|
||||
assert(h->prev != nullptr);
|
||||
h->next->prev = h->prev;
|
||||
h->prev->next = h->next;
|
||||
h->prev = h->next = nullptr;
|
||||
assert(lru_usage_ >= h->total_charge);
|
||||
lru_usage_ -= h->total_charge;
|
||||
}
|
||||
|
||||
void LRUCacheShard::LRU_Insert(LRUHandle* h) {
|
||||
assert(h->next == nullptr);
|
||||
assert(h->prev == nullptr);
|
||||
// Insert h to head of LRU list.
|
||||
h->next = &lru_;
|
||||
h->prev = lru_.prev;
|
||||
h->prev->next = h;
|
||||
h->next->prev = h;
|
||||
lru_usage_ += h->total_charge;
|
||||
}
|
||||
|
||||
void LRUCacheShard::EvictFromLRU(size_t charge,
|
||||
autovector<LRUHandle>* deleted) {
|
||||
while ((usage_ + charge) > capacity_ && lru_.next != &lru_) {
|
||||
LRUHandle* old = lru_.next;
|
||||
// LRU list contains only elements which can be evicted.
|
||||
assert(old->IsVisible() && !old->HasRefs());
|
||||
LRU_Remove(old);
|
||||
table_.Remove(old);
|
||||
assert(usage_ >= old->total_charge);
|
||||
usage_ -= old->total_charge;
|
||||
deleted->push_back(*old);
|
||||
}
|
||||
}
|
||||
|
||||
size_t LRUCacheShard::CalcEstimatedHandleCharge(
|
||||
size_t estimated_value_size,
|
||||
CacheMetadataChargePolicy metadata_charge_policy) {
|
||||
LRUHandle h;
|
||||
h.CalcTotalCharge(estimated_value_size, metadata_charge_policy);
|
||||
return h.total_charge;
|
||||
}
|
||||
|
||||
int LRUCacheShard::CalcHashBits(
|
||||
size_t capacity, size_t estimated_value_size,
|
||||
CacheMetadataChargePolicy metadata_charge_policy) {
|
||||
size_t handle_charge =
|
||||
CalcEstimatedHandleCharge(estimated_value_size, metadata_charge_policy);
|
||||
assert(handle_charge > 0);
|
||||
uint32_t num_entries =
|
||||
static_cast<uint32_t>(capacity / (kLoadFactor * handle_charge)) + 1;
|
||||
assert(num_entries <= uint32_t{1} << 31);
|
||||
return FloorLog2((num_entries << 1) - 1);
|
||||
}
|
||||
|
||||
void LRUCacheShard::SetCapacity(size_t capacity) {
|
||||
autovector<LRUHandle> last_reference_list;
|
||||
{
|
||||
DMutexLock l(mutex_);
|
||||
if (capacity > capacity_) {
|
||||
assert(false); // Not supported.
|
||||
}
|
||||
capacity_ = capacity;
|
||||
EvictFromLRU(0, &last_reference_list);
|
||||
}
|
||||
|
||||
// Free the entries here outside of mutex for performance reasons.
|
||||
for (auto& h : last_reference_list) {
|
||||
h.FreeData();
|
||||
}
|
||||
}
|
||||
|
||||
void LRUCacheShard::SetStrictCapacityLimit(bool strict_capacity_limit) {
|
||||
DMutexLock l(mutex_);
|
||||
strict_capacity_limit_ = strict_capacity_limit;
|
||||
}
|
||||
|
||||
Status LRUCacheShard::Insert(const Slice& key, uint32_t hash, void* value,
|
||||
size_t charge, Cache::DeleterFn deleter,
|
||||
LRUHandle** handle, Cache::Priority /*priority*/) {
|
||||
if (key.size() != kCacheKeySize) {
|
||||
return Status::NotSupported("FastLRUCache only supports key size " +
|
||||
std::to_string(kCacheKeySize) + "B");
|
||||
}
|
||||
|
||||
LRUHandle tmp;
|
||||
tmp.value = value;
|
||||
tmp.deleter = deleter;
|
||||
tmp.hash = hash;
|
||||
tmp.CalcTotalCharge(charge, metadata_charge_policy_);
|
||||
for (int i = 0; i < kCacheKeySize; i++) {
|
||||
tmp.key_data[i] = key.data()[i];
|
||||
}
|
||||
|
||||
Status s = Status::OK();
|
||||
autovector<LRUHandle> last_reference_list;
|
||||
{
|
||||
DMutexLock l(mutex_);
|
||||
assert(table_.GetOccupancy() <= table_.GetOccupancyLimit());
|
||||
|
||||
// Free the space following strict LRU policy until enough space
|
||||
// is freed or the lru list is empty.
|
||||
EvictFromLRU(tmp.total_charge, &last_reference_list);
|
||||
if ((usage_ + tmp.total_charge > capacity_ &&
|
||||
(strict_capacity_limit_ || handle == nullptr)) ||
|
||||
table_.GetOccupancy() == table_.GetOccupancyLimit()) {
|
||||
// There are two measures of capacity:
|
||||
// - Space (or charge) capacity: The maximum possible sum of the charges
|
||||
// of the elements.
|
||||
// - Table capacity: The number of slots in the hash table.
|
||||
// These are incomparable, in the sense that one doesn't imply the other.
|
||||
// Typically we will reach space capacity before table capacity---
|
||||
// if the user always inserts values with size equal to
|
||||
// estimated_value_size, then at most a kLoadFactor fraction of slots
|
||||
// will ever be occupied. But in some cases we may reach table capacity
|
||||
// before space capacity---if the user initially claims a very large
|
||||
// estimated_value_size but then inserts tiny values, more elements than
|
||||
// initially estimated will be inserted.
|
||||
|
||||
// TODO(Guido) Some tests (at least two from cache_test, as well as the
|
||||
// stress tests) currently assume the table capacity is unbounded.
|
||||
if (handle == nullptr) {
|
||||
// Don't insert the entry but still return ok, as if the entry inserted
|
||||
// into cache and get evicted immediately.
|
||||
last_reference_list.push_back(tmp);
|
||||
} else {
|
||||
if (table_.GetOccupancy() == table_.GetOccupancyLimit()) {
|
||||
// TODO: Consider using a distinct status for this case, but usually
|
||||
// it will be handled the same way as reaching charge capacity limit
|
||||
s = Status::MemoryLimit(
|
||||
"Insert failed because all slots in the hash table are full.");
|
||||
} else {
|
||||
s = Status::MemoryLimit(
|
||||
"Insert failed because the total charge has exceeded the "
|
||||
"capacity.");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Insert into the cache. Note that the cache might get larger than its
|
||||
// capacity if not enough space was freed up.
|
||||
LRUHandle* old;
|
||||
LRUHandle* h = table_.Insert(&tmp, &old);
|
||||
assert(h != nullptr); // We're below occupancy, so this insertion should
|
||||
// never fail.
|
||||
usage_ += h->total_charge;
|
||||
if (old != nullptr) {
|
||||
s = Status::OkOverwritten();
|
||||
assert(old->IsVisible());
|
||||
table_.Exclude(old);
|
||||
if (!old->HasRefs()) {
|
||||
// old is on LRU because it's in cache and its reference count is 0.
|
||||
LRU_Remove(old);
|
||||
table_.Remove(old);
|
||||
assert(usage_ >= old->total_charge);
|
||||
usage_ -= old->total_charge;
|
||||
last_reference_list.push_back(*old);
|
||||
}
|
||||
}
|
||||
if (handle == nullptr) {
|
||||
LRU_Insert(h);
|
||||
} else {
|
||||
// If caller already holds a ref, no need to take one here.
|
||||
if (!h->HasRefs()) {
|
||||
h->Ref();
|
||||
}
|
||||
*handle = h;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Free the entries here outside of mutex for performance reasons.
|
||||
for (auto& h : last_reference_list) {
|
||||
h.FreeData();
|
||||
}
|
||||
|
||||
return s;
|
||||
}
|
||||
|
||||
LRUHandle* LRUCacheShard::Lookup(const Slice& key, uint32_t hash) {
|
||||
LRUHandle* h = nullptr;
|
||||
{
|
||||
DMutexLock l(mutex_);
|
||||
h = table_.Lookup(key, hash);
|
||||
if (h != nullptr) {
|
||||
assert(h->IsVisible());
|
||||
if (!h->HasRefs()) {
|
||||
// The entry is in LRU since it's in hash and has no external
|
||||
// references.
|
||||
LRU_Remove(h);
|
||||
}
|
||||
h->Ref();
|
||||
}
|
||||
}
|
||||
return h;
|
||||
}
|
||||
|
||||
bool LRUCacheShard::Ref(LRUHandle* h) {
|
||||
DMutexLock l(mutex_);
|
||||
// To create another reference - entry must be already externally referenced.
|
||||
assert(h->HasRefs());
|
||||
h->Ref();
|
||||
return true;
|
||||
}
|
||||
|
||||
bool LRUCacheShard::Release(LRUHandle* h, bool erase_if_last_ref) {
|
||||
if (h == nullptr) {
|
||||
return false;
|
||||
}
|
||||
LRUHandle copy;
|
||||
bool last_reference = false;
|
||||
{
|
||||
DMutexLock l(mutex_);
|
||||
last_reference = h->Unref();
|
||||
if (last_reference && h->IsVisible()) {
|
||||
// The item is still in cache, and nobody else holds a reference to it.
|
||||
if (usage_ > capacity_ || erase_if_last_ref) {
|
||||
// The LRU list must be empty since the cache is full.
|
||||
assert(lru_.next == &lru_ || erase_if_last_ref);
|
||||
// Take this opportunity and remove the item.
|
||||
table_.Remove(h);
|
||||
} else {
|
||||
// Put the item back on the LRU list, and don't free it.
|
||||
LRU_Insert(h);
|
||||
last_reference = false;
|
||||
}
|
||||
}
|
||||
// If it was the last reference, then decrement the cache usage.
|
||||
if (last_reference) {
|
||||
assert(usage_ >= h->total_charge);
|
||||
usage_ -= h->total_charge;
|
||||
copy = *h;
|
||||
}
|
||||
}
|
||||
|
||||
// Free the entry here outside of mutex for performance reasons.
|
||||
if (last_reference) {
|
||||
copy.FreeData();
|
||||
}
|
||||
return last_reference;
|
||||
}
|
||||
|
||||
void LRUCacheShard::Erase(const Slice& key, uint32_t hash) {
|
||||
LRUHandle copy;
|
||||
bool last_reference = false;
|
||||
{
|
||||
DMutexLock l(mutex_);
|
||||
LRUHandle* h = table_.Lookup(key, hash);
|
||||
if (h != nullptr) {
|
||||
table_.Exclude(h);
|
||||
if (!h->HasRefs()) {
|
||||
// The entry is in LRU since it's in cache and has no external
|
||||
// references.
|
||||
LRU_Remove(h);
|
||||
table_.Remove(h);
|
||||
assert(usage_ >= h->total_charge);
|
||||
usage_ -= h->total_charge;
|
||||
last_reference = true;
|
||||
copy = *h;
|
||||
}
|
||||
}
|
||||
}
|
||||
// Free the entry here outside of mutex for performance reasons.
|
||||
// last_reference will only be true if e != nullptr.
|
||||
if (last_reference) {
|
||||
copy.FreeData();
|
||||
}
|
||||
}
|
||||
|
||||
size_t LRUCacheShard::GetUsage() const {
|
||||
DMutexLock l(mutex_);
|
||||
return usage_;
|
||||
}
|
||||
|
||||
size_t LRUCacheShard::GetPinnedUsage() const {
|
||||
DMutexLock l(mutex_);
|
||||
assert(usage_ >= lru_usage_);
|
||||
return usage_ - lru_usage_;
|
||||
}
|
||||
|
||||
size_t LRUCacheShard::GetOccupancyCount() const {
|
||||
DMutexLock l(mutex_);
|
||||
return table_.GetOccupancy();
|
||||
}
|
||||
|
||||
size_t LRUCacheShard::GetTableAddressCount() const {
|
||||
DMutexLock l(mutex_);
|
||||
return table_.GetTableSize();
|
||||
}
|
||||
|
||||
LRUCache::LRUCache(size_t capacity, size_t estimated_value_size,
|
||||
int num_shard_bits, bool strict_capacity_limit,
|
||||
CacheMetadataChargePolicy metadata_charge_policy)
|
||||
: ShardedCache(capacity, num_shard_bits, strict_capacity_limit,
|
||||
nullptr /*allocator*/) {
|
||||
assert(estimated_value_size > 0 ||
|
||||
metadata_charge_policy != kDontChargeCacheMetadata);
|
||||
size_t per_shard = GetPerShardCapacity();
|
||||
InitShards([=](LRUCacheShard* cs) {
|
||||
new (cs) LRUCacheShard(per_shard, estimated_value_size,
|
||||
strict_capacity_limit, metadata_charge_policy);
|
||||
});
|
||||
}
|
||||
|
||||
void* LRUCache::Value(Handle* handle) {
|
||||
return reinterpret_cast<const LRUHandle*>(handle)->value;
|
||||
}
|
||||
|
||||
size_t LRUCache::GetCharge(Handle* handle) const {
|
||||
return reinterpret_cast<const LRUHandle*>(handle)->GetCharge(
|
||||
GetShard(0).metadata_charge_policy_);
|
||||
}
|
||||
|
||||
Cache::DeleterFn LRUCache::GetDeleter(Handle* handle) const {
|
||||
auto h = reinterpret_cast<const LRUHandle*>(handle);
|
||||
return h->deleter;
|
||||
}
|
||||
|
||||
} // namespace fast_lru_cache
|
||||
|
||||
std::shared_ptr<Cache> NewFastLRUCache(
|
||||
size_t capacity, size_t estimated_value_size, int num_shard_bits,
|
||||
bool strict_capacity_limit,
|
||||
CacheMetadataChargePolicy metadata_charge_policy) {
|
||||
if (num_shard_bits >= 20) {
|
||||
return nullptr; // The cache cannot be sharded into too many fine pieces.
|
||||
}
|
||||
if (num_shard_bits < 0) {
|
||||
num_shard_bits = GetDefaultCacheShardBits(capacity);
|
||||
}
|
||||
return std::make_shared<fast_lru_cache::LRUCache>(
|
||||
capacity, estimated_value_size, num_shard_bits, strict_capacity_limit,
|
||||
metadata_charge_policy);
|
||||
}
|
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
|
@ -1,476 +0,0 @@
|
|||
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved
|
||||
// This source code is licensed under both the GPLv2 (found in the
|
||||
// COPYING file in the root directory) and Apache 2.0 License
|
||||
// (found in the LICENSE.Apache file in the root directory).
|
||||
//
|
||||
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style license that can be
|
||||
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||
#pragma once
|
||||
|
||||
#include <array>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
#include "cache/cache_key.h"
|
||||
#include "cache/sharded_cache.h"
|
||||
#include "port/lang.h"
|
||||
#include "port/malloc.h"
|
||||
#include "port/port.h"
|
||||
#include "rocksdb/secondary_cache.h"
|
||||
#include "util/autovector.h"
|
||||
#include "util/distributed_mutex.h"
|
||||
|
||||
namespace ROCKSDB_NAMESPACE {
|
||||
|
||||
namespace fast_lru_cache {
|
||||
|
||||
// Forward declaration of friend class.
|
||||
class FastLRUCacheTest;
|
||||
|
||||
// LRU cache implementation using an open-address hash table.
|
||||
//
|
||||
// Every slot in the hash table is an LRUHandle. Because handles can be
|
||||
// referenced externally, we can't discard them immediately once they are
|
||||
// deleted (via a delete or an LRU eviction) or replaced by a new version
|
||||
// (via an insert of the same key). The state of an element is defined by
|
||||
// the following two properties:
|
||||
// (R) Referenced: An element can be referenced externally (refs > 0), or not.
|
||||
// Importantly, an element can be evicted if and only if it's not
|
||||
// referenced. In particular, when an element becomes referenced, it's
|
||||
// temporarily taken out of the LRU list until all references to it
|
||||
// are dropped.
|
||||
// (V) Visible: An element can visible for lookups (IS_VISIBLE set), or not.
|
||||
// Initially, every element is visible. An element that is not visible is
|
||||
// called a ghost.
|
||||
// These properties induce 4 different states, with transitions defined as
|
||||
// follows:
|
||||
// - V --> not V: When a visible element is deleted or replaced by a new
|
||||
// version.
|
||||
// - Not V --> V: This cannot happen. A ghost remains in that state until it's
|
||||
// not referenced any more, at which point it's ready to be removed from the
|
||||
// hash table. (A ghost simply waits to transition to the afterlife---it will
|
||||
// never be visible again.)
|
||||
// - R --> not R: When all references to an element are dropped.
|
||||
// - Not R --> R: When an unreferenced element becomes referenced. This can only
|
||||
// happen if the element is V, since references to an element can only be
|
||||
// created when it's visible.
|
||||
//
|
||||
// Internally, the cache uses an open-addressed hash table to index the handles.
|
||||
// We use tombstone counters to keep track of displacements.
|
||||
// Because of the tombstones and the two possible visibility states of an
|
||||
// element, the table slots can be in 4 different states:
|
||||
// 1. Visible element (IS_ELEMENT set and IS_VISIBLE set): The slot contains a
|
||||
// key-value element.
|
||||
// 2. Ghost element (IS_ELEMENT set and IS_VISIBLE unset): The slot contains an
|
||||
// element that has been removed, but it's still referenced. It's invisible
|
||||
// to lookups.
|
||||
// 3. Tombstone (IS_ELEMENT unset and displacements > 0): The slot contains a
|
||||
// tombstone.
|
||||
// 4. Empty (IS_ELEMENT unset and displacements == 0): The slot is unused.
|
||||
// A slot that is an element can further have IS_VISIBLE set or not.
|
||||
// When a ghost is removed from the table, it can either transition to being a
|
||||
// tombstone or an empty slot, depending on the number of displacements of the
|
||||
// slot. In any case, the slot becomes available. When a handle is inserted
|
||||
// into that slot, it becomes a visible element again.
|
||||
|
||||
// The load factor p is a real number in (0, 1) such that at all
|
||||
// times at most a fraction p of all slots, without counting tombstones,
|
||||
// are occupied by elements. This means that the probability that a
|
||||
// random probe hits an empty slot is at most p, and thus at most 1/p probes
|
||||
// are required on average. For example, p = 70% implies that between 1 and 2
|
||||
// probes are needed on average (bear in mind that this reasoning doesn't
|
||||
// consider the effects of clustering over time).
|
||||
// Because the size of the hash table is always rounded up to the next
|
||||
// power of 2, p is really an upper bound on the actual load factor---the
|
||||
// actual load factor is anywhere between p/2 and p. This is a bit wasteful,
|
||||
// but bear in mind that slots only hold metadata, not actual values.
|
||||
// Since space cost is dominated by the values (the LSM blocks),
|
||||
// overprovisioning the table with metadata only increases the total cache space
|
||||
// usage by a tiny fraction.
|
||||
constexpr double kLoadFactor = 0.35;
|
||||
|
||||
// The user can exceed kLoadFactor if the sizes of the inserted values don't
|
||||
// match estimated_value_size, or if strict_capacity_limit == false. To
|
||||
// avoid performance to plunge, we set a strict upper bound on the load factor.
|
||||
constexpr double kStrictLoadFactor = 0.7;
|
||||
|
||||
// Arbitrary seeds.
|
||||
constexpr uint32_t kProbingSeed1 = 0xbc9f1d34;
|
||||
constexpr uint32_t kProbingSeed2 = 0x7a2bb9d5;
|
||||
|
||||
// An experimental (under development!) alternative to LRUCache
|
||||
|
||||
struct LRUHandle {
|
||||
void* value;
|
||||
Cache::DeleterFn deleter;
|
||||
LRUHandle* next;
|
||||
LRUHandle* prev;
|
||||
size_t total_charge; // TODO(opt): Only allow uint32_t?
|
||||
// The hash of key(). Used for fast sharding and comparisons.
|
||||
uint32_t hash;
|
||||
// The number of external refs to this entry.
|
||||
uint32_t refs;
|
||||
|
||||
enum Flags : uint8_t {
|
||||
// Whether the handle is visible to Lookups.
|
||||
IS_VISIBLE = (1 << 0),
|
||||
// Whether the slot is in use by an element.
|
||||
IS_ELEMENT = (1 << 1),
|
||||
};
|
||||
uint8_t flags;
|
||||
|
||||
// The number of elements that hash to this slot or a lower one,
|
||||
// but wind up in a higher slot.
|
||||
uint32_t displacements;
|
||||
|
||||
std::array<char, kCacheKeySize> key_data;
|
||||
|
||||
LRUHandle() {
|
||||
value = nullptr;
|
||||
deleter = nullptr;
|
||||
next = nullptr;
|
||||
prev = nullptr;
|
||||
total_charge = 0;
|
||||
hash = 0;
|
||||
refs = 0;
|
||||
flags = 0;
|
||||
displacements = 0;
|
||||
key_data.fill(0);
|
||||
}
|
||||
|
||||
Slice key() const { return Slice(key_data.data(), kCacheKeySize); }
|
||||
|
||||
// For HandleImpl concept
|
||||
uint32_t GetHash() const { return hash; }
|
||||
|
||||
// Increase the reference count by 1.
|
||||
void Ref() { refs++; }
|
||||
|
||||
// Just reduce the reference count by 1. Return true if it was last reference.
|
||||
bool Unref() {
|
||||
assert(refs > 0);
|
||||
refs--;
|
||||
return refs == 0;
|
||||
}
|
||||
|
||||
// Return true if there are external refs, false otherwise.
|
||||
bool HasRefs() const { return refs > 0; }
|
||||
|
||||
bool IsVisible() const { return flags & IS_VISIBLE; }
|
||||
|
||||
void SetIsVisible(bool is_visible) {
|
||||
if (is_visible) {
|
||||
flags |= IS_VISIBLE;
|
||||
} else {
|
||||
flags &= ~IS_VISIBLE;
|
||||
}
|
||||
}
|
||||
|
||||
bool IsElement() const { return flags & IS_ELEMENT; }
|
||||
|
||||
void SetIsElement(bool is_element) {
|
||||
if (is_element) {
|
||||
flags |= IS_ELEMENT;
|
||||
} else {
|
||||
flags &= ~IS_ELEMENT;
|
||||
}
|
||||
}
|
||||
|
||||
void FreeData() {
|
||||
assert(refs == 0);
|
||||
if (deleter) {
|
||||
(*deleter)(key(), value);
|
||||
}
|
||||
}
|
||||
|
||||
// Calculate the memory usage by metadata.
|
||||
inline size_t CalcMetaCharge(
|
||||
CacheMetadataChargePolicy metadata_charge_policy) const {
|
||||
if (metadata_charge_policy != kFullChargeCacheMetadata) {
|
||||
return 0;
|
||||
} else {
|
||||
// #ifdef ROCKSDB_MALLOC_USABLE_SIZE
|
||||
// return malloc_usable_size(
|
||||
// const_cast<void*>(static_cast<const void*>(this)));
|
||||
// #else
|
||||
// TODO(Guido) malloc_usable_size only works when we call it on
|
||||
// a pointer allocated with malloc. Because our handles are all
|
||||
// allocated in a single shot as an array, the user can't call
|
||||
// CalcMetaCharge (or CalcTotalCharge or GetCharge) on a handle
|
||||
// pointer returned by the cache. Moreover, malloc_usable_size
|
||||
// expects a heap-allocated handle, but sometimes in our code we
|
||||
// wish to pass a stack-allocated handle (this is only a performance
|
||||
// concern).
|
||||
// What is the right way to compute metadata charges with pre-allocated
|
||||
// handles?
|
||||
return sizeof(LRUHandle);
|
||||
// #endif
|
||||
}
|
||||
}
|
||||
|
||||
inline void CalcTotalCharge(
|
||||
size_t charge, CacheMetadataChargePolicy metadata_charge_policy) {
|
||||
total_charge = charge + CalcMetaCharge(metadata_charge_policy);
|
||||
}
|
||||
|
||||
inline size_t GetCharge(
|
||||
CacheMetadataChargePolicy metadata_charge_policy) const {
|
||||
size_t meta_charge = CalcMetaCharge(metadata_charge_policy);
|
||||
assert(total_charge >= meta_charge);
|
||||
return total_charge - meta_charge;
|
||||
}
|
||||
|
||||
inline bool IsEmpty() {
|
||||
return !this->IsElement() && this->displacements == 0;
|
||||
}
|
||||
|
||||
inline bool IsTombstone() {
|
||||
return !this->IsElement() && this->displacements > 0;
|
||||
}
|
||||
|
||||
inline bool Matches(const Slice& some_key, uint32_t some_hash) {
|
||||
return this->IsElement() && this->hash == some_hash &&
|
||||
this->key() == some_key;
|
||||
}
|
||||
};
|
||||
|
||||
class LRUHandleTable {
|
||||
public:
|
||||
explicit LRUHandleTable(int hash_bits);
|
||||
~LRUHandleTable();
|
||||
|
||||
// Returns a pointer to a visible element matching the key/hash, or
|
||||
// nullptr if not present.
|
||||
LRUHandle* Lookup(const Slice& key, uint32_t hash);
|
||||
|
||||
// Inserts a copy of h into the hash table.
|
||||
// Returns a pointer to the inserted handle, or nullptr if no slot
|
||||
// available was found. If an existing visible element matching the
|
||||
// key/hash is already present in the hash table, the argument old
|
||||
// is set to pointe to it; otherwise, it's set to nullptr.
|
||||
LRUHandle* Insert(LRUHandle* h, LRUHandle** old);
|
||||
|
||||
// Removes h from the hash table. The handle must already be off
|
||||
// the LRU list.
|
||||
void Remove(LRUHandle* h);
|
||||
|
||||
// Turns a visible element h into a ghost (i.e., not visible).
|
||||
void Exclude(LRUHandle* h);
|
||||
|
||||
// Assigns a copy of h to the given slot.
|
||||
void Assign(int slot, LRUHandle* h);
|
||||
|
||||
template <typename T>
|
||||
void ApplyToEntriesRange(T func, size_t index_begin, size_t index_end) {
|
||||
for (size_t i = index_begin; i < index_end; i++) {
|
||||
LRUHandle* h = &array_[i];
|
||||
if (h->IsVisible()) {
|
||||
func(h);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
uint32_t GetTableSize() const { return uint32_t{1} << length_bits_; }
|
||||
|
||||
int GetLengthBits() const { return length_bits_; }
|
||||
|
||||
uint32_t GetOccupancyLimit() const { return occupancy_limit_; }
|
||||
|
||||
uint32_t GetOccupancy() const { return occupancy_; }
|
||||
|
||||
// Returns x mod 2^{length_bits_}.
|
||||
uint32_t ModTableSize(uint32_t x) { return x & length_bits_mask_; }
|
||||
|
||||
private:
|
||||
int FindVisibleElement(const Slice& key, uint32_t hash, int& probe,
|
||||
int displacement);
|
||||
|
||||
int FindAvailableSlot(const Slice& key, int& probe, int displacement);
|
||||
|
||||
int FindVisibleElementOrAvailableSlot(const Slice& key, uint32_t hash,
|
||||
int& probe, int displacement);
|
||||
|
||||
// Returns the index of the first slot probed (hashing with
|
||||
// the given key) with a handle e such that cond(e) is true.
|
||||
// Otherwise, if no match is found, returns -1.
|
||||
// For every handle e probed except the final slot, updates
|
||||
// e->displacements += displacement.
|
||||
// The argument probe is modified such that consecutive calls
|
||||
// to FindSlot continue probing right after where the previous
|
||||
// call left.
|
||||
int FindSlot(const Slice& key, std::function<bool(LRUHandle*)> cond,
|
||||
int& probe, int displacement);
|
||||
|
||||
// Number of hash bits used for table index.
|
||||
// The size of the table is 1 << length_bits_.
|
||||
int length_bits_;
|
||||
|
||||
const uint32_t length_bits_mask_;
|
||||
|
||||
// Number of elements in the table.
|
||||
uint32_t occupancy_;
|
||||
|
||||
// Maximum number of elements the user can store in the table.
|
||||
uint32_t occupancy_limit_;
|
||||
|
||||
std::unique_ptr<LRUHandle[]> array_;
|
||||
};
|
||||
|
||||
// A single shard of sharded cache.
|
||||
class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShardBase {
|
||||
public:
|
||||
LRUCacheShard(size_t capacity, size_t estimated_value_size,
|
||||
bool strict_capacity_limit,
|
||||
CacheMetadataChargePolicy metadata_charge_policy);
|
||||
|
||||
// For CacheShard concept
|
||||
using HandleImpl = LRUHandle;
|
||||
|
||||
// Keep 32-bit hashing for now (FIXME: upgrade to 64-bit)
|
||||
using HashVal = uint32_t;
|
||||
using HashCref = uint32_t;
|
||||
static inline HashVal ComputeHash(const Slice& key) {
|
||||
return Lower32of64(GetSliceNPHash64(key));
|
||||
}
|
||||
static inline uint32_t HashPieceForSharding(HashCref hash) { return hash; }
|
||||
|
||||
// Separate from constructor so caller can easily make an array of LRUCache
|
||||
// if current usage is more than new capacity, the function will attempt to
|
||||
// free the needed space.
|
||||
void SetCapacity(size_t capacity);
|
||||
|
||||
// Set the flag to reject insertion if cache if full.
|
||||
void SetStrictCapacityLimit(bool strict_capacity_limit);
|
||||
|
||||
// Like Cache methods, but with an extra "hash" parameter.
|
||||
// Insert an item into the hash table and, if handle is null, insert into
|
||||
// the LRU list. Older items are evicted as necessary. If the cache is full
|
||||
// and free_handle_on_fail is true, the item is deleted and handle is set to
|
||||
// nullptr.
|
||||
Status Insert(const Slice& key, uint32_t hash, void* value, size_t charge,
|
||||
Cache::DeleterFn deleter, LRUHandle** handle,
|
||||
Cache::Priority priority);
|
||||
|
||||
Status Insert(const Slice& key, uint32_t hash, void* value,
|
||||
const Cache::CacheItemHelper* helper, size_t charge,
|
||||
LRUHandle** handle, Cache::Priority priority) {
|
||||
return Insert(key, hash, value, charge, helper->del_cb, handle, priority);
|
||||
}
|
||||
|
||||
LRUHandle* Lookup(const Slice& key, uint32_t hash,
|
||||
const Cache::CacheItemHelper* /*helper*/,
|
||||
const Cache::CreateCallback& /*create_cb*/,
|
||||
Cache::Priority /*priority*/, bool /*wait*/,
|
||||
Statistics* /*stats*/) {
|
||||
return Lookup(key, hash);
|
||||
}
|
||||
LRUHandle* Lookup(const Slice& key, uint32_t hash);
|
||||
|
||||
bool Release(LRUHandle* handle, bool /*useful*/, bool erase_if_last_ref) {
|
||||
return Release(handle, erase_if_last_ref);
|
||||
}
|
||||
bool IsReady(LRUHandle* /*handle*/) { return true; }
|
||||
void Wait(LRUHandle* /*handle*/) {}
|
||||
|
||||
bool Ref(LRUHandle* handle);
|
||||
bool Release(LRUHandle* handle, bool erase_if_last_ref = false);
|
||||
void Erase(const Slice& key, uint32_t hash);
|
||||
|
||||
size_t GetUsage() const;
|
||||
size_t GetPinnedUsage() const;
|
||||
size_t GetOccupancyCount() const;
|
||||
size_t GetTableAddressCount() const;
|
||||
|
||||
void ApplyToSomeEntries(
|
||||
const std::function<void(const Slice& key, void* value, size_t charge,
|
||||
DeleterFn deleter)>& callback,
|
||||
size_t average_entries_per_lock, size_t* state);
|
||||
|
||||
void EraseUnRefEntries();
|
||||
|
||||
private:
|
||||
friend class LRUCache;
|
||||
friend class FastLRUCacheTest;
|
||||
|
||||
void LRU_Remove(LRUHandle* e);
|
||||
void LRU_Insert(LRUHandle* e);
|
||||
|
||||
// Free some space following strict LRU policy until enough space
|
||||
// to hold (usage_ + charge) is freed or the LRU list is empty
|
||||
// This function is not thread safe - it needs to be executed while
|
||||
// holding the mutex_.
|
||||
void EvictFromLRU(size_t charge, autovector<LRUHandle>* deleted);
|
||||
|
||||
// Returns the charge of a single handle.
|
||||
static size_t CalcEstimatedHandleCharge(
|
||||
size_t estimated_value_size,
|
||||
CacheMetadataChargePolicy metadata_charge_policy);
|
||||
|
||||
// Returns the number of bits used to hash an element in the hash
|
||||
// table.
|
||||
static int CalcHashBits(size_t capacity, size_t estimated_value_size,
|
||||
CacheMetadataChargePolicy metadata_charge_policy);
|
||||
|
||||
// Initialized before use.
|
||||
size_t capacity_;
|
||||
|
||||
// Whether to reject insertion if cache reaches its full capacity.
|
||||
bool strict_capacity_limit_;
|
||||
|
||||
// Dummy head of LRU list.
|
||||
// lru.prev is newest entry, lru.next is oldest entry.
|
||||
// LRU contains items which can be evicted, ie reference only by cache
|
||||
LRUHandle lru_;
|
||||
|
||||
// Pointer to head of low-pri pool in LRU list.
|
||||
LRUHandle* lru_low_pri_;
|
||||
|
||||
// ------------^^^^^^^^^^^^^-----------
|
||||
// Not frequently modified data members
|
||||
// ------------------------------------
|
||||
//
|
||||
// We separate data members that are updated frequently from the ones that
|
||||
// are not frequently updated so that they don't share the same cache line
|
||||
// which will lead into false cache sharing
|
||||
//
|
||||
// ------------------------------------
|
||||
// Frequently modified data members
|
||||
// ------------vvvvvvvvvvvvv-----------
|
||||
LRUHandleTable table_;
|
||||
|
||||
// Memory size for entries residing in the cache.
|
||||
size_t usage_;
|
||||
|
||||
// Memory size for entries residing only in the LRU list.
|
||||
size_t lru_usage_;
|
||||
|
||||
// mutex_ protects the following state.
|
||||
// We don't count mutex_ as the cache's internal state so semantically we
|
||||
// don't mind mutex_ invoking the non-const actions.
|
||||
mutable DMutex mutex_;
|
||||
};
|
||||
|
||||
class LRUCache
|
||||
#ifdef NDEBUG
|
||||
final
|
||||
#endif
|
||||
: public ShardedCache<LRUCacheShard> {
|
||||
public:
|
||||
LRUCache(size_t capacity, size_t estimated_value_size, int num_shard_bits,
|
||||
bool strict_capacity_limit,
|
||||
CacheMetadataChargePolicy metadata_charge_policy =
|
||||
kDontChargeCacheMetadata);
|
||||
const char* Name() const override { return "LRUCache"; }
|
||||
void* Value(Handle* handle) override;
|
||||
size_t GetCharge(Handle* handle) const override;
|
||||
DeleterFn GetDeleter(Handle* handle) const override;
|
||||
};
|
||||
} // namespace fast_lru_cache
|
||||
|
||||
std::shared_ptr<Cache> NewFastLRUCache(
|
||||
size_t capacity, size_t estimated_value_size, int num_shard_bits,
|
||||
bool strict_capacity_limit,
|
||||
CacheMetadataChargePolicy metadata_charge_policy);
|
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
|
@ -10,7 +10,6 @@
|
|||
|
||||
#include "cache/cache_key.h"
|
||||
#include "cache/clock_cache.h"
|
||||
#include "cache/fast_lru_cache.h"
|
||||
#include "db/db_test_util.h"
|
||||
#include "file/sst_file_manager_impl.h"
|
||||
#include "port/port.h"
|
||||
|
@ -364,148 +363,6 @@ TEST_F(LRUCacheTest, EntriesWithPriority) {
|
|||
ValidateLRUList({"x", "y", "g", "z", "d", "m"}, 2, 2, 2);
|
||||
}
|
||||
|
||||
// TODO: FastLRUCache and ClockCache use the same tests. We can probably remove
|
||||
// them from FastLRUCache after ClockCache becomes productive, and we don't plan
|
||||
// to use or maintain FastLRUCache any more.
|
||||
namespace fast_lru_cache {
|
||||
|
||||
// TODO(guido) Replicate LRU policy tests from LRUCache here.
|
||||
class FastLRUCacheTest : public testing::Test {
|
||||
public:
|
||||
FastLRUCacheTest() {}
|
||||
~FastLRUCacheTest() override { DeleteCache(); }
|
||||
|
||||
void DeleteCache() {
|
||||
if (cache_ != nullptr) {
|
||||
cache_->~LRUCacheShard();
|
||||
port::cacheline_aligned_free(cache_);
|
||||
cache_ = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
void NewCache(size_t capacity) {
|
||||
DeleteCache();
|
||||
cache_ = reinterpret_cast<LRUCacheShard*>(
|
||||
port::cacheline_aligned_alloc(sizeof(LRUCacheShard)));
|
||||
new (cache_) LRUCacheShard(capacity, 1 /*estimated_value_size*/,
|
||||
false /*strict_capacity_limit*/,
|
||||
kDontChargeCacheMetadata);
|
||||
}
|
||||
|
||||
Status Insert(const std::string& key) {
|
||||
return cache_->Insert(key, 0 /*hash*/, nullptr /*value*/, 1 /*charge*/,
|
||||
nullptr /*deleter*/, nullptr /*handle*/,
|
||||
Cache::Priority::LOW);
|
||||
}
|
||||
|
||||
Status Insert(char key, size_t len) { return Insert(std::string(len, key)); }
|
||||
|
||||
size_t CalcEstimatedHandleChargeWrapper(
|
||||
size_t estimated_value_size,
|
||||
CacheMetadataChargePolicy metadata_charge_policy) {
|
||||
return LRUCacheShard::CalcEstimatedHandleCharge(estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
}
|
||||
|
||||
int CalcHashBitsWrapper(size_t capacity, size_t estimated_value_size,
|
||||
CacheMetadataChargePolicy metadata_charge_policy) {
|
||||
return LRUCacheShard::CalcHashBits(capacity, estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
}
|
||||
|
||||
// Maximum number of items that a shard can hold.
|
||||
double CalcMaxOccupancy(size_t capacity, size_t estimated_value_size,
|
||||
CacheMetadataChargePolicy metadata_charge_policy) {
|
||||
size_t handle_charge = LRUCacheShard::CalcEstimatedHandleCharge(
|
||||
estimated_value_size, metadata_charge_policy);
|
||||
return capacity / (kLoadFactor * handle_charge);
|
||||
}
|
||||
bool TableSizeIsAppropriate(int hash_bits, double max_occupancy) {
|
||||
if (hash_bits == 0) {
|
||||
return max_occupancy <= 1;
|
||||
} else {
|
||||
return (1 << hash_bits >= max_occupancy) &&
|
||||
(1 << (hash_bits - 1) <= max_occupancy);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
LRUCacheShard* cache_ = nullptr;
|
||||
};
|
||||
|
||||
TEST_F(FastLRUCacheTest, ValidateKeySize) {
|
||||
NewCache(3);
|
||||
EXPECT_OK(Insert('a', 16));
|
||||
EXPECT_NOK(Insert('b', 15));
|
||||
EXPECT_OK(Insert('b', 16));
|
||||
EXPECT_NOK(Insert('c', 17));
|
||||
EXPECT_NOK(Insert('d', 1000));
|
||||
EXPECT_NOK(Insert('e', 11));
|
||||
EXPECT_NOK(Insert('f', 0));
|
||||
}
|
||||
|
||||
TEST_F(FastLRUCacheTest, CalcHashBitsTest) {
|
||||
size_t capacity;
|
||||
size_t estimated_value_size;
|
||||
double max_occupancy;
|
||||
int hash_bits;
|
||||
CacheMetadataChargePolicy metadata_charge_policy;
|
||||
// Vary the cache capacity, fix the element charge.
|
||||
for (int i = 0; i < 2048; i++) {
|
||||
capacity = i;
|
||||
estimated_value_size = 0;
|
||||
metadata_charge_policy = kFullChargeCacheMetadata;
|
||||
max_occupancy = CalcMaxOccupancy(capacity, estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
hash_bits = CalcHashBitsWrapper(capacity, estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
EXPECT_TRUE(TableSizeIsAppropriate(hash_bits, max_occupancy));
|
||||
}
|
||||
// Fix the cache capacity, vary the element charge.
|
||||
for (int i = 0; i < 1024; i++) {
|
||||
capacity = 1024;
|
||||
estimated_value_size = i;
|
||||
metadata_charge_policy = kFullChargeCacheMetadata;
|
||||
max_occupancy = CalcMaxOccupancy(capacity, estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
hash_bits = CalcHashBitsWrapper(capacity, estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
EXPECT_TRUE(TableSizeIsAppropriate(hash_bits, max_occupancy));
|
||||
}
|
||||
// Zero-capacity cache, and only values have charge.
|
||||
capacity = 0;
|
||||
estimated_value_size = 1;
|
||||
metadata_charge_policy = kDontChargeCacheMetadata;
|
||||
hash_bits = CalcHashBitsWrapper(capacity, estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
EXPECT_TRUE(TableSizeIsAppropriate(hash_bits, 0 /* max_occupancy */));
|
||||
// Zero-capacity cache, and only metadata has charge.
|
||||
capacity = 0;
|
||||
estimated_value_size = 0;
|
||||
metadata_charge_policy = kFullChargeCacheMetadata;
|
||||
hash_bits = CalcHashBitsWrapper(capacity, estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
EXPECT_TRUE(TableSizeIsAppropriate(hash_bits, 0 /* max_occupancy */));
|
||||
// Small cache, large elements.
|
||||
capacity = 1024;
|
||||
estimated_value_size = 8192;
|
||||
metadata_charge_policy = kFullChargeCacheMetadata;
|
||||
hash_bits = CalcHashBitsWrapper(capacity, estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
EXPECT_TRUE(TableSizeIsAppropriate(hash_bits, 0 /* max_occupancy */));
|
||||
// Large capacity.
|
||||
capacity = 31924172;
|
||||
estimated_value_size = 8192;
|
||||
metadata_charge_policy = kFullChargeCacheMetadata;
|
||||
max_occupancy =
|
||||
CalcMaxOccupancy(capacity, estimated_value_size, metadata_charge_policy);
|
||||
hash_bits = CalcHashBitsWrapper(capacity, estimated_value_size,
|
||||
metadata_charge_policy);
|
||||
EXPECT_TRUE(TableSizeIsAppropriate(hash_bits, max_occupancy));
|
||||
}
|
||||
|
||||
} // namespace fast_lru_cache
|
||||
|
||||
namespace clock_cache {
|
||||
|
||||
class ClockCacheTest : public testing::Test {
|
||||
|
|
|
@ -13,7 +13,6 @@
|
|||
|
||||
#include "cache/cache_entry_roles.h"
|
||||
#include "cache/cache_key.h"
|
||||
#include "cache/fast_lru_cache.h"
|
||||
#include "cache/lru_cache.h"
|
||||
#include "db/column_family.h"
|
||||
#include "db/db_impl/db_impl.h"
|
||||
|
@ -944,10 +943,7 @@ TEST_F(DBBlockCacheTest, AddRedundantStats) {
|
|||
capacity,
|
||||
BlockBasedTableOptions().block_size /*estimated_value_size*/,
|
||||
num_shard_bits)
|
||||
.MakeSharedCache(),
|
||||
NewFastLRUCache(capacity, 1 /*estimated_value_size*/, num_shard_bits,
|
||||
false /*strict_capacity_limit*/,
|
||||
kDefaultCacheMetadataChargePolicy)}) {
|
||||
.MakeSharedCache()}) {
|
||||
if (!base_cache) {
|
||||
// Skip clock cache when not supported
|
||||
continue;
|
||||
|
|
|
@ -12,7 +12,6 @@
|
|||
|
||||
#include "util/compression.h"
|
||||
#ifdef GFLAGS
|
||||
#include "cache/fast_lru_cache.h"
|
||||
#include "db_stress_tool/db_stress_common.h"
|
||||
#include "db_stress_tool/db_stress_compaction_filter.h"
|
||||
#include "db_stress_tool/db_stress_driver.h"
|
||||
|
@ -124,10 +123,6 @@ std::shared_ptr<Cache> StressTest::NewCache(size_t capacity,
|
|||
FLAGS_block_size /*estimated_entry_charge*/,
|
||||
num_shard_bits)
|
||||
.MakeSharedCache();
|
||||
} else if (FLAGS_cache_type == "fast_lru_cache") {
|
||||
return NewFastLRUCache(static_cast<size_t>(capacity), FLAGS_block_size,
|
||||
num_shard_bits, false /*strict_capacity_limit*/,
|
||||
kDefaultCacheMetadataChargePolicy);
|
||||
} else if (FLAGS_cache_type == "lru_cache") {
|
||||
LRUCacheOptions opts;
|
||||
opts.capacity = capacity;
|
||||
|
|
1
src.mk
1
src.mk
|
@ -6,7 +6,6 @@ LIB_SOURCES = \
|
|||
cache/cache_reservation_manager.cc \
|
||||
cache/charged_cache.cc \
|
||||
cache/clock_cache.cc \
|
||||
cache/fast_lru_cache.cc \
|
||||
cache/lru_cache.cc \
|
||||
cache/compressed_secondary_cache.cc \
|
||||
cache/sharded_cache.cc \
|
||||
|
|
|
@ -37,7 +37,6 @@
|
|||
#include <thread>
|
||||
#include <unordered_map>
|
||||
|
||||
#include "cache/fast_lru_cache.h"
|
||||
#include "db/db_impl/db_impl.h"
|
||||
#include "db/malloc_stats.h"
|
||||
#include "db/version_set.h"
|
||||
|
@ -3055,11 +3054,6 @@ class Benchmark {
|
|||
FLAGS_block_size /*estimated_entry_charge*/,
|
||||
FLAGS_cache_numshardbits)
|
||||
.MakeSharedCache();
|
||||
} else if (FLAGS_cache_type == "fast_lru_cache") {
|
||||
return NewFastLRUCache(static_cast<size_t>(capacity), FLAGS_block_size,
|
||||
FLAGS_cache_numshardbits,
|
||||
false /*strict_capacity_limit*/,
|
||||
kDefaultCacheMetadataChargePolicy);
|
||||
} else if (FLAGS_cache_type == "lru_cache") {
|
||||
LRUCacheOptions opts(
|
||||
static_cast<size_t>(capacity), FLAGS_cache_numshardbits,
|
||||
|
|
|
@ -122,7 +122,6 @@ default_params = {
|
|||
"use_direct_io_for_flush_and_compaction": lambda: random.randint(0, 1),
|
||||
"mock_direct_io": False,
|
||||
"cache_type": lambda: random.choice(["lru_cache", "hyper_clock_cache"]),
|
||||
# fast_lru_cache is incompatible with stress tests, because it doesn't support strict_capacity_limit == false.
|
||||
"use_full_merge_v1": lambda: random.randint(0, 1),
|
||||
"use_merge": lambda: random.randint(0, 1),
|
||||
# use_put_entity_one_in has to be the same across invocations for verification to work, hence no lambda
|
||||
|
|
Loading…
Reference in New Issue