Add hash_seed to Caches (#11391)

Summary:
See motivation and description in new ShardedCacheOptions::hash_seed option.

Updated db_bench so that its seed param is used for the cache hash seed.
Made its code more safe to ensure seed is set before use.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11391

Test Plan:
unit tests added / updated

**Performance** - no discernible difference seen running cache_bench repeatedly before & after. With lru_cache and hyper_clock_cache.

Reviewed By: hx235

Differential Revision: D45557797

Pulled By: pdillinger

fbshipit-source-id: 40bf4da6d66f9d41a8a0eb8e5cf4246a4aa07934
This commit is contained in:
Peter Dillinger 2023-05-09 22:24:26 -07:00 committed by Facebook GitHub Bot
parent 6ba4717f35
commit f4a02f2c52
18 changed files with 260 additions and 37 deletions

View File

@ -3,6 +3,7 @@
### New Features
* Introduced a new option `block_protection_bytes_per_key`, which can be used to enable per key-value integrity protection for in-memory blocks in block cache (#11287).
* Added `JemallocAllocatorOptions::num_arenas`. Setting `num_arenas > 1` may mitigate mutex contention in the allocator, particularly in scenarios where block allocations commonly bypass jemalloc tcache.
* Improve the operational safety of publishing a DB or SST files to many hosts by using different block cache hash seeds on different hosts. The exact behavior is controlled by new option `ShardedCacheOptions::hash_seed`, which also documents the solved problem in more detail.
### Public API Changes
* Add `MakeSharedCache()` construction functions to various cache Options objects, and deprecated the `NewWhateverCache()` functions with long parameter lists.

92
cache/cache_test.cc vendored
View File

@ -21,6 +21,7 @@
#include "test_util/secondary_cache_test_util.h"
#include "test_util/testharness.h"
#include "util/coding.h"
#include "util/hash_containers.h"
#include "util/string_util.h"
// HyperClockCache only supports 16-byte keys, so some of the tests
@ -955,6 +956,97 @@ TEST_P(CacheTest, GetChargeAndDeleter) {
cache_->Release(h1);
}
namespace {
bool AreTwoCacheKeysOrdered(Cache* cache) {
std::vector<std::string> keys;
const auto callback = [&](const Slice& key, Cache::ObjectPtr /*value*/,
size_t /*charge*/,
const Cache::CacheItemHelper* /*helper*/) {
keys.push_back(key.ToString());
};
cache->ApplyToAllEntries(callback, /*opts*/ {});
EXPECT_EQ(keys.size(), 2U);
EXPECT_NE(keys[0], keys[1]);
return keys[0] < keys[1];
}
} // namespace
TEST_P(CacheTest, CacheUniqueSeeds) {
// kQuasiRandomHashSeed should generate unique seeds (up to 2 billion before
// repeating)
UnorderedSet<uint32_t> seeds_seen;
// Roughly sqrt(number of possible values) for a decent chance at detecting
// a random collision if it's possible (shouldn't be)
uint16_t kSamples = 20000;
seeds_seen.reserve(kSamples);
// Hash seed should affect ordering of entries in the table, so we should
// have extremely high chance of seeing two entries ordered both ways.
bool seen_forward_order = false;
bool seen_reverse_order = false;
for (int i = 0; i < kSamples; ++i) {
auto cache = NewCache(2, [=](ShardedCacheOptions& opts) {
opts.hash_seed = LRUCacheOptions::kQuasiRandomHashSeed;
opts.num_shard_bits = 0;
opts.metadata_charge_policy = kDontChargeCacheMetadata;
});
auto val = cache->GetHashSeed();
ASSERT_TRUE(seeds_seen.insert(val).second);
ASSERT_OK(cache->Insert(EncodeKey(1), nullptr, &kHelper, /*charge*/ 1));
ASSERT_OK(cache->Insert(EncodeKey(2), nullptr, &kHelper, /*charge*/ 1));
if (AreTwoCacheKeysOrdered(cache.get())) {
seen_forward_order = true;
} else {
seen_reverse_order = true;
}
}
ASSERT_TRUE(seen_forward_order);
ASSERT_TRUE(seen_reverse_order);
}
TEST_P(CacheTest, CacheHostSeed) {
// kHostHashSeed should generate a consistent seed within this process
// (and other processes on the same host, but not unit testing that).
// And we should be able to use that chosen seed as an explicit option
// (for debugging).
// And we should verify consistent ordering of entries.
uint32_t expected_seed = 0;
bool expected_order = false;
// 10 iterations -> chance of a random seed falsely appearing consistent
// should be low, just 1 in 2^9.
for (int i = 0; i < 10; ++i) {
auto cache = NewCache(2, [=](ShardedCacheOptions& opts) {
if (i != 5) {
opts.hash_seed = LRUCacheOptions::kHostHashSeed;
} else {
// Can be used as explicit seed
opts.hash_seed = static_cast<int32_t>(expected_seed);
ASSERT_GE(opts.hash_seed, 0);
}
opts.num_shard_bits = 0;
opts.metadata_charge_policy = kDontChargeCacheMetadata;
});
ASSERT_OK(cache->Insert(EncodeKey(1), nullptr, &kHelper, /*charge*/ 1));
ASSERT_OK(cache->Insert(EncodeKey(2), nullptr, &kHelper, /*charge*/ 1));
uint32_t val = cache->GetHashSeed();
bool order = AreTwoCacheKeysOrdered(cache.get());
if (i != 0) {
ASSERT_EQ(val, expected_seed);
ASSERT_EQ(order, expected_order);
} else {
expected_seed = val;
expected_order = order;
}
}
// Printed for reference in case it's needed to reproduce other unit test
// failures on another host
fprintf(stderr, "kHostHashSeed -> %u\n", (unsigned)expected_seed);
}
INSTANTIATE_TEST_CASE_P(CacheTestInstance, CacheTest,
secondary_cache_test_util::GetTestingCacheTypes());
INSTANTIATE_TEST_CASE_P(CacheTestInstance, LRUCacheTest,

19
cache/clock_cache.cc vendored
View File

@ -130,7 +130,8 @@ HyperClockTable::HyperClockTable(
size_t capacity, bool /*strict_capacity_limit*/,
CacheMetadataChargePolicy metadata_charge_policy,
MemoryAllocator* allocator,
const Cache::EvictionCallback* eviction_callback, const Opts& opts)
const Cache::EvictionCallback* eviction_callback, const uint32_t* hash_seed,
const Opts& opts)
: length_bits_(CalcHashBits(capacity, opts.estimated_value_size,
metadata_charge_policy)),
length_bits_mask_((size_t{1} << length_bits_) - 1),
@ -138,7 +139,8 @@ HyperClockTable::HyperClockTable(
kStrictLoadFactor)),
array_(new HandleImpl[size_t{1} << length_bits_]),
allocator_(allocator),
eviction_callback_(*eviction_callback) {
eviction_callback_(*eviction_callback),
hash_seed_(*hash_seed) {
if (metadata_charge_policy ==
CacheMetadataChargePolicy::kFullChargeCacheMetadata) {
usage_ += size_t{GetTableSize()} * sizeof(HandleImpl);
@ -1010,7 +1012,7 @@ inline void HyperClockTable::Evict(size_t requested_charge,
if (eviction_callback_) {
took_ownership =
eviction_callback_(ClockCacheShard<HyperClockTable>::ReverseHash(
h.GetHash(), &unhashed),
h.GetHash(), &unhashed, hash_seed_),
reinterpret_cast<Cache::Handle*>(&h));
}
if (!took_ownership) {
@ -1039,11 +1041,11 @@ ClockCacheShard<Table>::ClockCacheShard(
size_t capacity, bool strict_capacity_limit,
CacheMetadataChargePolicy metadata_charge_policy,
MemoryAllocator* allocator,
const Cache::EvictionCallback* eviction_callback,
const Cache::EvictionCallback* eviction_callback, const uint32_t* hash_seed,
const typename Table::Opts& opts)
: CacheShardBase(metadata_charge_policy),
table_(capacity, strict_capacity_limit, metadata_charge_policy, allocator,
eviction_callback, opts),
eviction_callback, hash_seed, opts),
capacity_(capacity),
strict_capacity_limit_(strict_capacity_limit) {
// Initial charge metadata should not exceed capacity
@ -1082,10 +1084,11 @@ void ClockCacheShard<Table>::ApplyToSomeEntries(
*state = index_end << (sizeof(size_t) * 8u - length_bits);
}
auto hash_seed = table_.GetHashSeed();
table_.ConstApplyToEntriesRange(
[callback](const HandleImpl& h) {
[callback, hash_seed](const HandleImpl& h) {
UniqueId64x2 unhashed;
callback(ReverseHash(h.hashed_key, &unhashed), h.value,
callback(ReverseHash(h.hashed_key, &unhashed, hash_seed), h.value,
h.GetTotalCharge(), h.helper);
},
index_begin, index_end, false);
@ -1295,7 +1298,7 @@ HyperClockCache::HyperClockCache(const HyperClockCacheOptions& opts)
table_opts.estimated_value_size = opts.estimated_entry_charge;
new (cs) Shard(per_shard, opts.strict_capacity_limit,
opts.metadata_charge_policy, alloc, &eviction_callback_,
table_opts);
&hash_seed_, table_opts);
});
}

16
cache/clock_cache.h vendored
View File

@ -407,7 +407,7 @@ class HyperClockTable {
CacheMetadataChargePolicy metadata_charge_policy,
MemoryAllocator* allocator,
const Cache::EvictionCallback* eviction_callback,
const Opts& opts);
const uint32_t* hash_seed, const Opts& opts);
~HyperClockTable();
Status Insert(const ClockHandleBasicData& proto, HandleImpl** handle,
@ -448,6 +448,8 @@ class HyperClockTable {
return standalone_usage_.load(std::memory_order_relaxed);
}
uint32_t GetHashSeed() const { return hash_seed_; }
// Acquire/release N references
void TEST_RefN(HandleImpl& handle, size_t n);
void TEST_ReleaseN(HandleImpl* handle, size_t n);
@ -546,6 +548,9 @@ class HyperClockTable {
// A reference to Cache::eviction_callback_
const Cache::EvictionCallback& eviction_callback_;
// A reference to ShardedCacheBase::hash_seed_
const uint32_t& hash_seed_;
// We partition the following members into different cache lines
// to avoid false sharing among Lookup, Release, Erase and Insert
// operations in ClockCacheShard.
@ -573,7 +578,7 @@ class ALIGN_AS(CACHE_LINE_SIZE) ClockCacheShard final : public CacheShardBase {
CacheMetadataChargePolicy metadata_charge_policy,
MemoryAllocator* allocator,
const Cache::EvictionCallback* eviction_callback,
const typename Table::Opts& opts);
const uint32_t* hash_seed, const typename Table::Opts& opts);
// For CacheShard concept
using HandleImpl = typename Table::HandleImpl;
@ -583,22 +588,23 @@ class ALIGN_AS(CACHE_LINE_SIZE) ClockCacheShard final : public CacheShardBase {
static inline uint32_t HashPieceForSharding(HashCref hash) {
return Upper32of64(hash[0]);
}
static inline HashVal ComputeHash(const Slice& key) {
static inline HashVal ComputeHash(const Slice& key, uint32_t seed) {
assert(key.size() == kCacheKeySize);
HashVal in;
HashVal out;
// NOTE: endian dependence
// TODO: use GetUnaligned?
std::memcpy(&in, key.data(), kCacheKeySize);
BijectiveHash2x64(in[1], in[0], &out[1], &out[0]);
BijectiveHash2x64(in[1], in[0] ^ seed, &out[1], &out[0]);
return out;
}
// For reconstructing key from hashed_key. Requires the caller to provide
// backing storage for the Slice in `unhashed`
static inline Slice ReverseHash(const UniqueId64x2& hashed,
UniqueId64x2* unhashed) {
UniqueId64x2* unhashed, uint32_t seed) {
BijectiveUnhash2x64(hashed[1], hashed[0], &(*unhashed)[1], &(*unhashed)[0]);
(*unhashed)[0] ^= seed;
// NOTE: endian dependence
return Slice(reinterpret_cast<const char*>(unhashed), kCacheKeySize);
}

4
cache/lru_cache.h vendored
View File

@ -279,8 +279,8 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShardBase {
using HashCref = uint32_t;
public: // Function definitions expected as parameter to ShardedCache
static inline HashVal ComputeHash(const Slice& key) {
return Lower32of64(GetSliceNPHash64(key));
static inline HashVal ComputeHash(const Slice& key, uint32_t seed) {
return Lower32of64(GetSliceNPHash64(key, seed));
}
// Separate from constructor so caller can easily make an array of LRUCache

View File

@ -397,7 +397,7 @@ class ClockCacheTest : public testing::Test {
opts.estimated_value_size = 1;
new (shard_)
Shard(capacity, strict_capacity_limit, kDontChargeCacheMetadata,
/*allocator*/ nullptr, &eviction_callback_, opts);
/*allocator*/ nullptr, &eviction_callback_, &hash_seed_, opts);
}
Status Insert(const UniqueId64x2& hashed_key,
@ -455,6 +455,7 @@ class ClockCacheTest : public testing::Test {
private:
Cache::EvictionCallback eviction_callback_;
uint32_t hash_seed_ = 0;
};
TEST_F(ClockCacheTest, Misc) {

View File

@ -13,16 +13,55 @@
#include <cstdint>
#include <memory>
#include "env/unique_id_gen.h"
#include "rocksdb/env.h"
#include "util/hash.h"
#include "util/math.h"
#include "util/mutexlock.h"
namespace ROCKSDB_NAMESPACE {
namespace {
// The generated seeds must fit in 31 bits so that
// ShardedCacheOptions::hash_seed can be set to it explicitly, for
// diagnostic/debugging purposes.
constexpr uint32_t kSeedMask = 0x7fffffff;
uint32_t DetermineSeed(int32_t hash_seed_option) {
if (hash_seed_option >= 0) {
// User-specified exact seed
return static_cast<uint32_t>(hash_seed_option);
}
static SemiStructuredUniqueIdGen gen;
if (hash_seed_option == ShardedCacheOptions::kHostHashSeed) {
std::string hostname;
Status s = Env::Default()->GetHostNameString(&hostname);
if (s.ok()) {
return GetSliceHash(hostname) & kSeedMask;
} else {
// Fall back on something stable within the process.
return static_cast<uint32_t>(gen.GetBaseUpper()) & kSeedMask;
}
} else {
// for kQuasiRandomHashSeed and fallback
uint32_t val = gen.GenerateNext<uint32_t>() & kSeedMask;
// Perform some 31-bit bijective transformations so that we get
// quasirandom, not just incrementing. (An incrementing seed from a
// random starting point would be fine, but hard to describe in a name.)
// See https://en.wikipedia.org/wiki/Quasirandom and using a murmur-like
// transformation here for our bijection in the lower 31 bits.
// See https://en.wikipedia.org/wiki/MurmurHash
val *= /*31-bit prime*/ 1150630961;
val ^= (val & kSeedMask) >> 17;
val *= /*31-bit prime*/ 1320603883;
return val & kSeedMask;
}
}
} // namespace
ShardedCacheBase::ShardedCacheBase(const ShardedCacheOptions& opts)
: Cache(opts.memory_allocator),
last_id_(1),
shard_mask_((uint32_t{1} << opts.num_shard_bits) - 1),
hash_seed_(DetermineSeed(opts.hash_seed)),
strict_capacity_limit_(opts.strict_capacity_limit),
capacity_(opts.capacity) {}

15
cache/sharded_cache.h vendored
View File

@ -34,8 +34,8 @@ class CacheShardBase {
std::string GetPrintableOptions() const { return ""; }
using HashVal = uint64_t;
using HashCref = uint64_t;
static inline HashVal ComputeHash(const Slice& key) {
return GetSliceNPHash64(key);
static inline HashVal ComputeHash(const Slice& key, uint32_t seed) {
return GetSliceNPHash64(key, seed);
}
static inline uint32_t HashPieceForSharding(HashCref hash) {
return Lower32of64(hash);
@ -104,6 +104,8 @@ class ShardedCacheBase : public Cache {
size_t GetUsage(Handle* handle) const override;
std::string GetPrintableOptions() const override;
uint32_t GetHashSeed() const override { return hash_seed_; }
protected: // fns
virtual void AppendPrintableOptions(std::string& str) const = 0;
size_t GetPerShardCapacity() const;
@ -112,6 +114,7 @@ class ShardedCacheBase : public Cache {
protected: // data
std::atomic<uint64_t> last_id_; // For NewId
const uint32_t shard_mask_;
const uint32_t hash_seed_;
// Dynamic configuration parameters, guarded by config_mutex_
bool strict_capacity_limit_;
@ -171,7 +174,7 @@ class ShardedCache : public ShardedCacheBase {
size_t charge, Handle** handle = nullptr,
Priority priority = Priority::LOW) override {
assert(helper);
HashVal hash = CacheShard::ComputeHash(key);
HashVal hash = CacheShard::ComputeHash(key, hash_seed_);
auto h_out = reinterpret_cast<HandleImpl**>(handle);
return GetShard(hash).Insert(key, hash, obj, helper, charge, h_out,
priority);
@ -181,7 +184,7 @@ class ShardedCache : public ShardedCacheBase {
const CacheItemHelper* helper, size_t charge,
bool allow_uncharged) override {
assert(helper);
HashVal hash = CacheShard::ComputeHash(key);
HashVal hash = CacheShard::ComputeHash(key, hash_seed_);
HandleImpl* result = GetShard(hash).CreateStandalone(
key, hash, obj, helper, charge, allow_uncharged);
return reinterpret_cast<Handle*>(result);
@ -191,14 +194,14 @@ class ShardedCache : public ShardedCacheBase {
CreateContext* create_context = nullptr,
Priority priority = Priority::LOW,
Statistics* stats = nullptr) override {
HashVal hash = CacheShard::ComputeHash(key);
HashVal hash = CacheShard::ComputeHash(key, hash_seed_);
HandleImpl* result = GetShard(hash).Lookup(key, hash, helper,
create_context, priority, stats);
return reinterpret_cast<Handle*>(result);
}
void Erase(const Slice& key) override {
HashVal hash = CacheShard::ComputeHash(key);
HashVal hash = CacheShard::ComputeHash(key, hash_seed_);
GetShard(hash).Erase(key, hash);
}

View File

@ -979,12 +979,14 @@ TEST_F(DBBlockCacheTest, CacheEntryRoleStats) {
const size_t capacity = size_t{1} << 25;
int iterations_tested = 0;
for (bool partition : {false, true}) {
SCOPED_TRACE("Partition? " + std::to_string(partition));
for (std::shared_ptr<Cache> cache :
{NewLRUCache(capacity),
HyperClockCacheOptions(
capacity,
BlockBasedTableOptions().block_size /*estimated_value_size*/)
.MakeSharedCache()}) {
SCOPED_TRACE(std::string("Cache: ") + cache->Name());
++iterations_tested;
Options options = CurrentOptions();
@ -1278,6 +1280,7 @@ TEST_F(DBBlockCacheTest, HyperClockCacheReportProblems) {
HyperClockCacheOptions hcc_opts{capacity, value_size_est};
hcc_opts.num_shard_bits = 2; // 4 shards
hcc_opts.metadata_charge_policy = kDontChargeCacheMetadata;
hcc_opts.hash_seed = 0; // deterministic hashing
std::shared_ptr<Cache> cache = hcc_opts.MakeSharedCache();
std::shared_ptr<CountingLogger> logger = std::make_shared<CountingLogger>();

View File

@ -258,6 +258,9 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname,
co.capacity = table_cache_size;
co.num_shard_bits = immutable_db_options_.table_cache_numshardbits;
co.metadata_charge_policy = kDontChargeCacheMetadata;
// TODO: Consider a non-fixed seed once test fallout (prefetch_test) is
// dealt with
co.hash_seed = 0;
table_cache_ = NewLRUCache(co);
SetDbSessionId();
assert(!db_session_id_.empty());

View File

@ -700,6 +700,7 @@ void InternalStats::CacheEntryRoleStats::BeginCollection(
cache_usage = cache->GetUsage();
table_size = cache->GetTableAddressCount();
occupancy = cache->GetOccupancyCount();
hash_seed = cache->GetHashSeed();
}
void InternalStats::CacheEntryRoleStats::EndCollection(
@ -724,7 +725,7 @@ std::string InternalStats::CacheEntryRoleStats::ToString(
std::ostringstream str;
str << "Block cache " << cache_id
<< " capacity: " << BytesToHumanString(cache_capacity)
<< " usage: " << BytesToHumanString(cache_usage)
<< " seed: " << hash_seed << " usage: " << BytesToHumanString(cache_usage)
<< " table_size: " << table_size << " occupancy: " << occupancy
<< " collections: " << collection_count
<< " last_copies: " << copies_of_last_collection

View File

@ -478,6 +478,7 @@ class InternalStats {
uint32_t copies_of_last_collection = 0;
uint64_t last_start_time_micros_ = 0;
uint64_t last_end_time_micros_ = 0;
uint32_t hash_seed = 0;
void Clear() {
// Wipe everything except collection_count

14
env/env_test.cc vendored
View File

@ -3134,6 +3134,20 @@ TEST_F(EnvTest, SemiStructuredUniqueIdGenTest) {
t.Run();
}
TEST_F(EnvTest, SemiStructuredUniqueIdGenTestSmaller) {
// For small generated types, will cycle through all the possible values.
SemiStructuredUniqueIdGen gen;
std::vector<bool> hit(256);
for (int i = 0; i < 256; ++i) {
auto val = gen.GenerateNext<uint8_t>();
ASSERT_FALSE(hit[val]);
hit[val] = true;
}
for (int i = 0; i < 256; ++i) {
ASSERT_TRUE(hit[i]);
}
}
TEST_F(EnvTest, FailureToCreateLockFile) {
auto env = Env::Default();
auto fs = env->GetFileSystem();

14
env/unique_id_gen.h vendored
View File

@ -14,6 +14,7 @@
#include <atomic>
#include <cstdint>
#include <type_traits>
#include "rocksdb/rocksdb_namespace.h"
@ -61,6 +62,19 @@ class SemiStructuredUniqueIdGen {
// to the next (thread safe).
void GenerateNext(uint64_t* upper, uint64_t* lower);
// For generating smaller values. Will cycle through all the possibilities
// before repeating.
template <typename T>
T GenerateNext() {
static_assert(sizeof(T) <= sizeof(uint64_t));
static_assert(std::is_integral_v<T>);
uint64_t ignore, val;
GenerateNext(&ignore, &val);
return static_cast<T>(val);
}
uint64_t GetBaseUpper() const { return base_upper_; }
private:
uint64_t base_upper_;
uint64_t base_lower_;

View File

@ -404,6 +404,9 @@ class Cache {
MemoryAllocator* memory_allocator() const { return memory_allocator_.get(); }
// See ShardedCacheOptions::hash_seed
virtual uint32_t GetHashSeed() const { return 0; }
// EXPERIMENTAL
// The following APIs are experimental and might change in the future.

View File

@ -138,6 +138,35 @@ struct ShardedCacheOptions {
// A SecondaryCache instance to use the non-volatile tier.
std::shared_ptr<SecondaryCache> secondary_cache;
// See hash_seed comments below
static constexpr int32_t kQuasiRandomHashSeed = -1;
static constexpr int32_t kHostHashSeed = -2;
// EXPERT OPTION: Specifies how a hash seed should be determined for the
// cache, or specifies a specific seed (only recommended for diagnostics or
// testing).
//
// Background: it could be dangerous to have different cache instances
// access the same SST files with the same hash seed, as correlated unlucky
// hashing across hosts or restarts could cause a widespread issue, rather
// than an isolated one. For example, with smaller block caches, it is
// possible for large full Bloom filters in a set of SST files to be randomly
// clustered into one cache shard, causing mutex contention or a thrashing
// condition as there's little or no space left for other entries assigned to
// the shard. If a set of SST files is broadcast and used on many hosts, we
// should ensure all have an independent chance of balanced shards.
//
// Values >= 0 will be treated as fixed hash seeds. Values < 0 are reserved
// for methods of dynamically choosing a seed, currently:
// * kQuasiRandomHashSeed - Each cache created chooses a seed mostly randomly,
// except that within a process, no seed is repeated until all have been
// issued.
// * kHostHashSeed - The seed is determined based on hashing the host name.
// Although this is arguably slightly worse for production reliability, it
// solves the essential problem of cross-host correlation while ensuring
// repeatable behavior on a host, for diagnostic purposes.
int32_t hash_seed = kHostHashSeed;
ShardedCacheOptions() {}
ShardedCacheOptions(
size_t _capacity, int _num_shard_bits, bool _strict_capacity_limit,

View File

@ -56,13 +56,15 @@ class WithCacheType : public TestCreateContext {
if (type == kLRU) {
LRUCacheOptions lru_opts;
lru_opts.capacity = capacity;
lru_opts.hash_seed = 0; // deterministic tests
if (modify_opts_fn) {
modify_opts_fn(lru_opts);
}
return NewLRUCache(lru_opts);
return lru_opts.MakeSharedCache();
}
if (type == kHyperClock) {
HyperClockCacheOptions hc_opts{capacity, estimated_value_size_};
hc_opts.hash_seed = 0; // deterministic tests
if (modify_opts_fn) {
modify_opts_fn(hc_opts);
}

View File

@ -33,6 +33,7 @@
#include <iostream>
#include <memory>
#include <mutex>
#include <optional>
#include <queue>
#include <thread>
#include <unordered_map>
@ -287,7 +288,7 @@ DEFINE_int32(bloom_locality, 0, "Control bloom filter probes locality");
DEFINE_int64(seed, 0,
"Seed base for random number generators. "
"When 0 it is derived from the current time.");
static int64_t seed_base;
static std::optional<int64_t> seed_base;
DEFINE_int32(threads, 1, "Number of concurrent threads to run.");
@ -2606,7 +2607,7 @@ struct ThreadState {
SharedState* shared;
explicit ThreadState(int index, int my_seed)
: tid(index), rand(seed_base + my_seed) {}
: tid(index), rand(*seed_base + my_seed) {}
};
class Duration {
@ -3002,6 +3003,11 @@ class Benchmark {
return allocator;
}
static int32_t GetCacheHashSeed() {
// For a fixed Cache seed, need a non-negative int32
return static_cast<int32_t>(*seed_base) & 0x7fffffff;
}
static std::shared_ptr<Cache> NewCache(int64_t capacity) {
if (capacity <= 0) {
return nullptr;
@ -3010,17 +3016,19 @@ class Benchmark {
fprintf(stderr, "Old clock cache implementation has been removed.\n");
exit(1);
} else if (FLAGS_cache_type == "hyper_clock_cache") {
return HyperClockCacheOptions(static_cast<size_t>(capacity),
FLAGS_block_size /*estimated_entry_charge*/,
FLAGS_cache_numshardbits)
.MakeSharedCache();
HyperClockCacheOptions hcco{
static_cast<size_t>(capacity),
static_cast<size_t>(FLAGS_block_size) /*estimated_entry_charge*/,
FLAGS_cache_numshardbits};
hcco.hash_seed = GetCacheHashSeed();
return hcco.MakeSharedCache();
} else if (FLAGS_cache_type == "lru_cache") {
LRUCacheOptions opts(
static_cast<size_t>(capacity), FLAGS_cache_numshardbits,
false /*strict_capacity_limit*/, FLAGS_cache_high_pri_pool_ratio,
GetCacheAllocator(), kDefaultToAdaptiveMutex,
kDefaultCacheMetadataChargePolicy, FLAGS_cache_low_pri_pool_ratio);
opts.hash_seed = GetCacheHashSeed();
if (!FLAGS_secondary_cache_uri.empty()) {
Status s = SecondaryCache::CreateFromString(
ConfigOptions(), FLAGS_secondary_cache_uri, &secondary_cache);
@ -3051,7 +3059,7 @@ class Benchmark {
NewCompressedSecondaryCache(secondary_cache_opts);
}
return NewLRUCache(opts);
return opts.MakeSharedCache();
} else {
fprintf(stderr, "Cache type not supported.");
exit(1);
@ -4891,7 +4899,7 @@ class Benchmark {
values_[i] = i;
}
RandomShuffle(values_.begin(), values_.end(),
static_cast<uint32_t>(seed_base));
static_cast<uint32_t>(*seed_base));
}
}
@ -5003,7 +5011,7 @@ class Benchmark {
// Default_random_engine provides slightly
// improved throughput over mt19937.
std::default_random_engine overwrite_gen{
static_cast<unsigned int>(seed_base)};
static_cast<unsigned int>(*seed_base)};
std::bernoulli_distribution overwrite_decider(p);
// Inserted key window is filled with the last N
@ -5013,7 +5021,7 @@ class Benchmark {
// - random access is O(1)
// - insertion/removal at beginning/end is also O(1).
std::deque<int64_t> inserted_key_window;
Random64 reservoir_id_gen(seed_base);
Random64 reservoir_id_gen(*seed_base);
// --- Variables used in disposable/persistent keys simulation:
// The following variables are used when
@ -5050,7 +5058,7 @@ class Benchmark {
ErrorExit();
}
}
Random rnd_disposable_entry(static_cast<uint32_t>(seed_base));
Random rnd_disposable_entry(static_cast<uint32_t>(*seed_base));
std::string random_value;
// Queue that stores scheduled timestamp of disposable entries deletes,
// along with starting index of disposable entry keys to delete.
@ -8516,7 +8524,7 @@ int db_bench_tool(int argc, char** argv) {
uint64_t now = FLAGS_env->GetSystemClock()->NowMicros();
seed_base = static_cast<int64_t>(now);
fprintf(stdout, "Set seed to %" PRIu64 " because --seed was 0\n",
seed_base);
*seed_base);
} else {
seed_base = FLAGS_seed;
}