You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/db/db_test_util.h

1276 lines
41 KiB

// 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 <fcntl.h>
#include <algorithm>
#include <cinttypes>
#include <map>
#include <set>
#include <string>
#include <thread>
#include <unordered_set>
#include <utility>
#include <vector>
#include "db/db_impl/db_impl.h"
#include "db/dbformat.h"
#include "file/filename.h"
#include "memtable/hash_linklist_rep.h"
#include "rocksdb/cache.h"
#include "rocksdb/compaction_filter.h"
#include "rocksdb/convenience.h"
#include "rocksdb/db.h"
#include "rocksdb/env.h"
#include "rocksdb/filter_policy.h"
#include "rocksdb/options.h"
#include "rocksdb/slice.h"
#include "rocksdb/sst_file_writer.h"
#include "rocksdb/statistics.h"
#include "rocksdb/table.h"
#include "rocksdb/utilities/checkpoint.h"
#include "table/mock_table.h"
#include "table/scoped_arena_iterator.h"
#include "test_util/sync_point.h"
#include "test_util/testharness.h"
#include "util/cast_util.h"
#include "util/compression.h"
#include "util/mutexlock.h"
#include "util/string_util.h"
#include "utilities/merge_operators.h"
namespace ROCKSDB_NAMESPACE {
class MockEnv;
namespace anon {
class AtomicCounter {
public:
explicit AtomicCounter(Env* env = NULL)
: env_(env), cond_count_(&mu_), count_(0) {}
void Increment() {
MutexLock l(&mu_);
count_++;
cond_count_.SignalAll();
}
int Read() {
MutexLock l(&mu_);
return count_;
}
bool WaitFor(int count) {
MutexLock l(&mu_);
uint64_t start = env_->NowMicros();
while (count_ < count) {
uint64_t now = env_->NowMicros();
cond_count_.TimedWait(now + /*1s*/ 1 * 1000 * 1000);
if (env_->NowMicros() - start > /*10s*/ 10 * 1000 * 1000) {
return false;
}
if (count_ < count) {
GTEST_LOG_(WARNING) << "WaitFor is taking more time than usual";
}
}
return true;
}
void Reset() {
MutexLock l(&mu_);
count_ = 0;
cond_count_.SignalAll();
}
private:
Env* env_;
port::Mutex mu_;
port::CondVar cond_count_;
int count_;
};
struct OptionsOverride {
std::shared_ptr<const FilterPolicy> filter_policy = nullptr;
// These will be used only if filter_policy is set
bool partition_filters = false;
uint64_t metadata_block_size = 1024;
// Used as a bit mask of individual enums in which to skip an XF test point
int skip_policy = 0;
};
} // namespace anon
enum SkipPolicy { kSkipNone = 0, kSkipNoSnapshot = 1, kSkipNoPrefix = 2 };
// A hacky skip list mem table that triggers flush after number of entries.
class SpecialMemTableRep : public MemTableRep {
public:
explicit SpecialMemTableRep(Allocator* allocator, MemTableRep* memtable,
int num_entries_flush)
: MemTableRep(allocator),
memtable_(memtable),
num_entries_flush_(num_entries_flush),
num_entries_(0) {}
virtual KeyHandle Allocate(const size_t len, char** buf) override {
return memtable_->Allocate(len, buf);
}
// Insert key into the list.
// REQUIRES: nothing that compares equal to key is currently in the list.
virtual void Insert(KeyHandle handle) override {
num_entries_++;
memtable_->Insert(handle);
}
Unordered Writes (#5218) Summary: Performing unordered writes in rocksdb when unordered_write option is set to true. When enabled the writes to memtable are done without joining any write thread. This offers much higher write throughput since the upcoming writes would not have to wait for the slowest memtable write to finish. The tradeoff is that the writes visible to a snapshot might change over time. If the application cannot tolerate that, it should implement its own mechanisms to work around that. Using TransactionDB with WRITE_PREPARED write policy is one way to achieve that. Doing so increases the max throughput by 2.2x without however compromising the snapshot guarantees. The patch is prepared based on an original by siying Existing unit tests are extended to include unordered_write option. Benchmark Results: ``` TEST_TMPDIR=/dev/shm/ ./db_bench_unordered --benchmarks=fillrandom --threads=32 --num=10000000 -max_write_buffer_number=16 --max_background_jobs=64 --batch_size=8 --writes=3000000 -level0_file_num_compaction_trigger=99999 --level0_slowdown_writes_trigger=99999 --level0_stop_writes_trigger=99999 -enable_pipelined_write=false -disable_auto_compactions --unordered_write=1 ``` With WAL - Vanilla RocksDB: 78.6 MB/s - WRITER_PREPARED with unordered_write: 177.8 MB/s (2.2x) - unordered_write: 368.9 MB/s (4.7x with relaxed snapshot guarantees) Without WAL - Vanilla RocksDB: 111.3 MB/s - WRITER_PREPARED with unordered_write: 259.3 MB/s MB/s (2.3x) - unordered_write: 645.6 MB/s (5.8x with relaxed snapshot guarantees) - WRITER_PREPARED with unordered_write disable concurrency control: 185.3 MB/s MB/s (2.35x) Limitations: - The feature is not yet extended to `max_successive_merges` > 0. The feature is also incompatible with `enable_pipelined_write` = true as well as with `allow_concurrent_memtable_write` = false. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5218 Differential Revision: D15219029 Pulled By: maysamyabandeh fbshipit-source-id: 38f2abc4af8780148c6128acdba2b3227bc81759
5 years ago
void InsertConcurrently(KeyHandle handle) override {
num_entries_++;
memtable_->Insert(handle);
}
// Returns true iff an entry that compares equal to key is in the list.
virtual bool Contains(const char* key) const override {
return memtable_->Contains(key);
}
virtual size_t ApproximateMemoryUsage() override {
// Return a high memory usage when number of entries exceeds the threshold
// to trigger a flush.
return (num_entries_ < num_entries_flush_) ? 0 : 1024 * 1024 * 1024;
}
virtual void Get(const LookupKey& k, void* callback_args,
bool (*callback_func)(void* arg,
const char* entry)) override {
memtable_->Get(k, callback_args, callback_func);
}
uint64_t ApproximateNumEntries(const Slice& start_ikey,
const Slice& end_ikey) override {
return memtable_->ApproximateNumEntries(start_ikey, end_ikey);
}
virtual MemTableRep::Iterator* GetIterator(Arena* arena = nullptr) override {
return memtable_->GetIterator(arena);
}
virtual ~SpecialMemTableRep() override {}
private:
std::unique_ptr<MemTableRep> memtable_;
int num_entries_flush_;
int num_entries_;
};
// The factory for the hacky skip list mem table that triggers flush after
// number of entries exceeds a threshold.
class SpecialSkipListFactory : public MemTableRepFactory {
public:
// After number of inserts exceeds `num_entries_flush` in a mem table, trigger
// flush.
explicit SpecialSkipListFactory(int num_entries_flush)
: num_entries_flush_(num_entries_flush) {}
using MemTableRepFactory::CreateMemTableRep;
virtual MemTableRep* CreateMemTableRep(
const MemTableRep::KeyComparator& compare, Allocator* allocator,
const SliceTransform* transform, Logger* /*logger*/) override {
return new SpecialMemTableRep(
allocator, factory_.CreateMemTableRep(compare, allocator, transform, 0),
num_entries_flush_);
}
virtual const char* Name() const override { return "SkipListFactory"; }
bool IsInsertConcurrentlySupported() const override {
return factory_.IsInsertConcurrentlySupported();
}
private:
SkipListFactory factory_;
int num_entries_flush_;
};
// Special Env used to delay background operations
class SpecialEnv : public EnvWrapper {
public:
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
4 years ago
explicit SpecialEnv(Env* base, bool time_elapse_only_sleep = false);
Status NewWritableFile(const std::string& f, std::unique_ptr<WritableFile>* r,
const EnvOptions& soptions) override {
class SSTableFile : public WritableFile {
private:
SpecialEnv* env_;
std::unique_ptr<WritableFile> base_;
public:
SSTableFile(SpecialEnv* env, std::unique_ptr<WritableFile>&& base)
: env_(env), base_(std::move(base)) {}
Status Append(const Slice& data) override {
if (env_->table_write_callback_) {
(*env_->table_write_callback_)();
}
if (env_->drop_writes_.load(std::memory_order_acquire)) {
// Drop writes on the floor
return Status::OK();
} else if (env_->no_space_.load(std::memory_order_acquire)) {
return Status::NoSpace("No space left on device");
} else {
env_->bytes_written_ += data.size();
return base_->Append(data);
}
}
Status Append(
const Slice& data,
const DataVerificationInfo& /* verification_info */) override {
return Append(data);
}
Status PositionedAppend(const Slice& data, uint64_t offset) override {
if (env_->table_write_callback_) {
(*env_->table_write_callback_)();
}
if (env_->drop_writes_.load(std::memory_order_acquire)) {
// Drop writes on the floor
return Status::OK();
} else if (env_->no_space_.load(std::memory_order_acquire)) {
return Status::NoSpace("No space left on device");
} else {
env_->bytes_written_ += data.size();
return base_->PositionedAppend(data, offset);
}
}
Status PositionedAppend(
const Slice& data, uint64_t offset,
const DataVerificationInfo& /* verification_info */) override {
return PositionedAppend(data, offset);
}
Status Truncate(uint64_t size) override { return base_->Truncate(size); }
Status RangeSync(uint64_t offset, uint64_t nbytes) override {
Status s = base_->RangeSync(offset, nbytes);
#if !(defined NDEBUG) || !defined(OS_WIN)
TEST_SYNC_POINT_CALLBACK("SpecialEnv::SStableFile::RangeSync", &s);
#endif // !(defined NDEBUG) || !defined(OS_WIN)
return s;
}
Status Close() override {
// SyncPoint is not supported in Released Windows Mode.
#if !(defined NDEBUG) || !defined(OS_WIN)
// Check preallocation size
// preallocation size is never passed to base file.
size_t preallocation_size = preallocation_block_size();
TEST_SYNC_POINT_CALLBACK("DBTestWritableFile.GetPreallocationStatus",
&preallocation_size);
#endif // !(defined NDEBUG) || !defined(OS_WIN)
Status s = base_->Close();
#if !(defined NDEBUG) || !defined(OS_WIN)
TEST_SYNC_POINT_CALLBACK("SpecialEnv::SStableFile::Close", &s);
#endif // !(defined NDEBUG) || !defined(OS_WIN)
return s;
}
Status Flush() override { return base_->Flush(); }
Status Sync() override {
++env_->sync_counter_;
while (env_->delay_sstable_sync_.load(std::memory_order_acquire)) {
env_->SleepForMicroseconds(100000);
}
Status s;
if (!env_->skip_fsync_) {
s = base_->Sync();
}
#if !(defined NDEBUG) || !defined(OS_WIN)
TEST_SYNC_POINT_CALLBACK("SpecialEnv::SStableFile::Sync", &s);
#endif // !(defined NDEBUG) || !defined(OS_WIN)
return s;
}
void SetIOPriority(Env::IOPriority pri) override {
base_->SetIOPriority(pri);
}
Env::IOPriority GetIOPriority() override {
return base_->GetIOPriority();
}
bool use_direct_io() const override {
return base_->use_direct_io();
}
Status Allocate(uint64_t offset, uint64_t len) override {
return base_->Allocate(offset, len);
}
size_t GetUniqueId(char* id, size_t max_size) const override {
return base_->GetUniqueId(id, max_size);
}
};
class ManifestFile : public WritableFile {
public:
ManifestFile(SpecialEnv* env, std::unique_ptr<WritableFile>&& b)
: env_(env), base_(std::move(b)) {}
Status Append(const Slice& data) override {
if (env_->manifest_write_error_.load(std::memory_order_acquire)) {
return Status::IOError("simulated writer error");
} else {
return base_->Append(data);
}
}
Status Append(
const Slice& data,
const DataVerificationInfo& /*verification_info*/) override {
return Append(data);
}
Refactor to support file_reader_writer on Windows. Summary. A change https://reviews.facebook.net/differential/diff/224721/ Has attempted to move common functionality out of platform dependent code to a new facility called file_reader_writer. This includes: - perf counters - Buffering - RateLimiting However, the change did not attempt to refactor Windows code. To mitigate, we introduce new quering interfaces such as UseOSBuffer(), GetRequiredBufferAlignment() and ReaderWriterForward() for pure forwarding where required. Introduce WritableFile got a new method Truncate(). This is to communicate to the file as to how much data it has on close. - When space is pre-allocated on Linux it is filled with zeros implicitly, no such thing exist on Windows so we must truncate file on close. - When operating in unbuffered mode the last page is filled with zeros but we still want to truncate. Previously, Close() would take care of it but now buffer management is shifted to the wrappers and the file has no idea about the file true size. This means that Close() on the wrapper level must always include Truncate() as well as wrapper __dtor should call Close() and against double Close(). Move buffered/unbuffered write logic to the wrapper. Utilize Aligned buffer class. Adjust tests and implement Truncate() where necessary. Come up with reasonable defaults for new virtual interfaces. Forward calls for RandomAccessReadAhead class to avoid double buffering and locking (double locking in unbuffered mode on WIndows).
9 years ago
Status Truncate(uint64_t size) override { return base_->Truncate(size); }
Status Close() override { return base_->Close(); }
Status Flush() override { return base_->Flush(); }
Status Sync() override {
++env_->sync_counter_;
if (env_->manifest_sync_error_.load(std::memory_order_acquire)) {
return Status::IOError("simulated sync error");
} else {
if (env_->skip_fsync_) {
return Status::OK();
} else {
return base_->Sync();
}
}
}
uint64_t GetFileSize() override { return base_->GetFileSize(); }
Status Allocate(uint64_t offset, uint64_t len) override {
return base_->Allocate(offset, len);
}
private:
SpecialEnv* env_;
std::unique_ptr<WritableFile> base_;
};
class WalFile : public WritableFile {
public:
WalFile(SpecialEnv* env, std::unique_ptr<WritableFile>&& b)
: env_(env), base_(std::move(b)) {
env_->num_open_wal_file_.fetch_add(1);
}
virtual ~WalFile() { env_->num_open_wal_file_.fetch_add(-1); }
Status Append(const Slice& data) override {
#if !(defined NDEBUG) || !defined(OS_WIN)
TEST_SYNC_POINT("SpecialEnv::WalFile::Append:1");
#endif
Status s;
if (env_->log_write_error_.load(std::memory_order_acquire)) {
s = Status::IOError("simulated writer error");
} else {
int slowdown =
env_->log_write_slowdown_.load(std::memory_order_acquire);
if (slowdown > 0) {
env_->SleepForMicroseconds(slowdown);
}
s = base_->Append(data);
}
#if !(defined NDEBUG) || !defined(OS_WIN)
TEST_SYNC_POINT("SpecialEnv::WalFile::Append:2");
#endif
return s;
}
Status Append(
const Slice& data,
const DataVerificationInfo& /* verification_info */) override {
return Append(data);
}
Refactor to support file_reader_writer on Windows. Summary. A change https://reviews.facebook.net/differential/diff/224721/ Has attempted to move common functionality out of platform dependent code to a new facility called file_reader_writer. This includes: - perf counters - Buffering - RateLimiting However, the change did not attempt to refactor Windows code. To mitigate, we introduce new quering interfaces such as UseOSBuffer(), GetRequiredBufferAlignment() and ReaderWriterForward() for pure forwarding where required. Introduce WritableFile got a new method Truncate(). This is to communicate to the file as to how much data it has on close. - When space is pre-allocated on Linux it is filled with zeros implicitly, no such thing exist on Windows so we must truncate file on close. - When operating in unbuffered mode the last page is filled with zeros but we still want to truncate. Previously, Close() would take care of it but now buffer management is shifted to the wrappers and the file has no idea about the file true size. This means that Close() on the wrapper level must always include Truncate() as well as wrapper __dtor should call Close() and against double Close(). Move buffered/unbuffered write logic to the wrapper. Utilize Aligned buffer class. Adjust tests and implement Truncate() where necessary. Come up with reasonable defaults for new virtual interfaces. Forward calls for RandomAccessReadAhead class to avoid double buffering and locking (double locking in unbuffered mode on WIndows).
9 years ago
Status Truncate(uint64_t size) override { return base_->Truncate(size); }
void PrepareWrite(size_t offset, size_t len) override {
base_->PrepareWrite(offset, len);
}
void SetPreallocationBlockSize(size_t size) override {
base_->SetPreallocationBlockSize(size);
}
Status Close() override {
// SyncPoint is not supported in Released Windows Mode.
#if !(defined NDEBUG) || !defined(OS_WIN)
// Check preallocation size
size_t block_size, last_allocated_block;
base_->GetPreallocationStatus(&block_size, &last_allocated_block);
TEST_SYNC_POINT_CALLBACK("DBTestWalFile.GetPreallocationStatus",
&block_size);
#endif // !(defined NDEBUG) || !defined(OS_WIN)
return base_->Close();
}
Status Flush() override { return base_->Flush(); }
Status Sync() override {
++env_->sync_counter_;
if (env_->corrupt_in_sync_) {
Append(std::string(33000, ' '));
return Status::IOError("Ingested Sync Failure");
}
if (env_->skip_fsync_) {
return Status::OK();
} else {
return base_->Sync();
}
}
bool IsSyncThreadSafe() const override {
return env_->is_wal_sync_thread_safe_.load();
}
Status Allocate(uint64_t offset, uint64_t len) override {
return base_->Allocate(offset, len);
}
private:
SpecialEnv* env_;
std::unique_ptr<WritableFile> base_;
};
class OtherFile : public WritableFile {
public:
OtherFile(SpecialEnv* env, std::unique_ptr<WritableFile>&& b)
: env_(env), base_(std::move(b)) {}
Status Append(const Slice& data) override { return base_->Append(data); }
Status Append(
const Slice& data,
const DataVerificationInfo& /*verification_info*/) override {
return Append(data);
}
Status Truncate(uint64_t size) override { return base_->Truncate(size); }
Status Close() override { return base_->Close(); }
Status Flush() override { return base_->Flush(); }
Status Sync() override {
if (env_->skip_fsync_) {
return Status::OK();
} else {
return base_->Sync();
}
}
uint64_t GetFileSize() override { return base_->GetFileSize(); }
Status Allocate(uint64_t offset, uint64_t len) override {
return base_->Allocate(offset, len);
}
private:
SpecialEnv* env_;
std::unique_ptr<WritableFile> base_;
};
Handle rename() failure in non-local FS (#8192) Summary: In a distributed environment, a file `rename()` operation can succeed on server (remote) side, but the client can somehow return non-ok status to RocksDB. Possible reasons include network partition, connection issue, etc. This happens in `rocksdb::SetCurrentFile()`, which can be called in `LogAndApply() -> ProcessManifestWrites()` if RocksDB tries to switch to a new MANIFEST. We currently always delete the new MANIFEST if an error occurs. This is problematic in distributed world. If the server-side successfully updates the CURRENT file via renaming, then a subsequent `DB::Open()` will try to look for the new MANIFEST and fail. As a fix, we can track the execution result of IO operations on the new MANIFEST. - If IO operations on the new MANIFEST fail, then we know the CURRENT must point to the original MANIFEST. Therefore, it is safe to remove the new MANIFEST. - If IO operations on the new MANIFEST all succeed, but somehow we end up in the clean up code block, then we do not know whether CURRENT points to the new or old MANIFEST. (For local POSIX-compliant FS, it should still point to old MANIFEST, but it does not matter if we keep the new MANIFEST.) Therefore, we keep the new MANIFEST. - Any future `LogAndApply()` will switch to a new MANIFEST and update CURRENT. - If process reopens the db immediately after the failure, then the CURRENT file can point to either the new MANIFEST or the old one, both of which exist. Therefore, recovery can succeed and ignore the other. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8192 Test Plan: make check Reviewed By: zhichao-cao Differential Revision: D27804648 Pulled By: riversand963 fbshipit-source-id: 9c16f2a5ce41bc6aadf085e48449b19ede8423e4
3 years ago
if (no_file_overwrite_.load(std::memory_order_acquire) &&
target()->FileExists(f).ok()) {
return Status::NotSupported("SpecialEnv::no_file_overwrite_ is true.");
}
if (non_writeable_rate_.load(std::memory_order_acquire) > 0) {
uint32_t random_number;
{
MutexLock l(&rnd_mutex_);
random_number = rnd_.Uniform(100);
}
if (random_number < non_writeable_rate_.load()) {
return Status::IOError("simulated random write error");
}
}
new_writable_count_++;
if (non_writable_count_.load() > 0) {
non_writable_count_--;
return Status::IOError("simulated write error");
}
EnvOptions optimized = soptions;
if (strstr(f.c_str(), "MANIFEST") != nullptr ||
strstr(f.c_str(), "log") != nullptr) {
optimized.use_mmap_writes = false;
optimized.use_direct_writes = false;
}
Status s = target()->NewWritableFile(f, r, optimized);
if (s.ok()) {
if (strstr(f.c_str(), ".sst") != nullptr) {
r->reset(new SSTableFile(this, std::move(*r)));
} else if (strstr(f.c_str(), "MANIFEST") != nullptr) {
r->reset(new ManifestFile(this, std::move(*r)));
} else if (strstr(f.c_str(), "log") != nullptr) {
r->reset(new WalFile(this, std::move(*r)));
} else {
r->reset(new OtherFile(this, std::move(*r)));
}
}
return s;
}
Status NewRandomAccessFile(const std::string& f,
std::unique_ptr<RandomAccessFile>* r,
const EnvOptions& soptions) override {
class CountingFile : public RandomAccessFile {
public:
CountingFile(std::unique_ptr<RandomAccessFile>&& target,
anon::AtomicCounter* counter,
std::atomic<size_t>* bytes_read)
: target_(std::move(target)),
counter_(counter),
bytes_read_(bytes_read) {}
virtual Status Read(uint64_t offset, size_t n, Slice* result,
char* scratch) const override {
counter_->Increment();
Status s = target_->Read(offset, n, result, scratch);
*bytes_read_ += result->size();
return s;
}
virtual Status Prefetch(uint64_t offset, size_t n) override {
Status s = target_->Prefetch(offset, n);
*bytes_read_ += n;
return s;
}
private:
std::unique_ptr<RandomAccessFile> target_;
anon::AtomicCounter* counter_;
std::atomic<size_t>* bytes_read_;
};
class RandomFailureFile : public RandomAccessFile {
public:
RandomFailureFile(std::unique_ptr<RandomAccessFile>&& target,
std::atomic<uint64_t>* failure_cnt, uint32_t fail_odd)
: target_(std::move(target)),
fail_cnt_(failure_cnt),
fail_odd_(fail_odd) {}
virtual Status Read(uint64_t offset, size_t n, Slice* result,
char* scratch) const override {
if (Random::GetTLSInstance()->OneIn(fail_odd_)) {
fail_cnt_->fetch_add(1);
return Status::IOError("random error");
}
return target_->Read(offset, n, result, scratch);
}
virtual Status Prefetch(uint64_t offset, size_t n) override {
return target_->Prefetch(offset, n);
}
private:
std::unique_ptr<RandomAccessFile> target_;
std::atomic<uint64_t>* fail_cnt_;
uint32_t fail_odd_;
};
Status s = target()->NewRandomAccessFile(f, r, soptions);
random_file_open_counter_++;
if (s.ok()) {
if (count_random_reads_) {
r->reset(new CountingFile(std::move(*r), &random_read_counter_,
&random_read_bytes_counter_));
} else if (rand_reads_fail_odd_ > 0) {
r->reset(new RandomFailureFile(std::move(*r), &num_reads_fails_,
rand_reads_fail_odd_));
}
}
if (s.ok() && soptions.compaction_readahead_size > 0) {
compaction_readahead_size_ = soptions.compaction_readahead_size;
}
return s;
}
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
6 years ago
virtual Status NewSequentialFile(const std::string& f,
std::unique_ptr<SequentialFile>* r,
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
6 years ago
const EnvOptions& soptions) override {
class CountingFile : public SequentialFile {
public:
CountingFile(std::unique_ptr<SequentialFile>&& target,
anon::AtomicCounter* counter)
: target_(std::move(target)), counter_(counter) {}
virtual Status Read(size_t n, Slice* result, char* scratch) override {
counter_->Increment();
return target_->Read(n, result, scratch);
}
virtual Status Skip(uint64_t n) override { return target_->Skip(n); }
private:
std::unique_ptr<SequentialFile> target_;
anon::AtomicCounter* counter_;
};
Status s = target()->NewSequentialFile(f, r, soptions);
if (s.ok() && count_sequential_reads_) {
r->reset(new CountingFile(std::move(*r), &sequential_read_counter_));
}
return s;
}
virtual void SleepForMicroseconds(int micros) override {
sleep_counter_.Increment();
if (no_slowdown_ || time_elapse_only_sleep_) {
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
4 years ago
addon_microseconds_.fetch_add(micros);
}
if (!no_slowdown_) {
target()->SleepForMicroseconds(micros);
}
}
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
4 years ago
void MockSleepForMicroseconds(int64_t micros) {
sleep_counter_.Increment();
assert(no_slowdown_);
addon_microseconds_.fetch_add(micros);
}
void MockSleepForSeconds(int64_t seconds) {
sleep_counter_.Increment();
assert(no_slowdown_);
addon_microseconds_.fetch_add(seconds * 1000000);
}
virtual Status GetCurrentTime(int64_t* unix_time) override {
Status s;
if (time_elapse_only_sleep_) {
*unix_time = maybe_starting_time_;
} else {
s = target()->GetCurrentTime(unix_time);
}
if (s.ok()) {
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
4 years ago
// mock microseconds elapsed to seconds of time
*unix_time += addon_microseconds_.load() / 1000000;
}
return s;
}
virtual uint64_t NowCPUNanos() override {
now_cpu_count_.fetch_add(1);
return target()->NowCPUNanos();
}
virtual uint64_t NowNanos() override {
return (time_elapse_only_sleep_ ? 0 : target()->NowNanos()) +
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
4 years ago
addon_microseconds_.load() * 1000;
}
virtual uint64_t NowMicros() override {
return (time_elapse_only_sleep_ ? 0 : target()->NowMicros()) +
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
4 years ago
addon_microseconds_.load();
}
virtual Status DeleteFile(const std::string& fname) override {
delete_count_.fetch_add(1);
return target()->DeleteFile(fname);
}
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
4 years ago
void SetMockSleep(bool enabled = true) { no_slowdown_ = enabled; }
Status NewDirectory(const std::string& name,
std::unique_ptr<Directory>* result) override {
if (!skip_fsync_) {
return target()->NewDirectory(name, result);
} else {
class NoopDirectory : public Directory {
public:
NoopDirectory() {}
~NoopDirectory() {}
Status Fsync() override { return Status::OK(); }
};
result->reset(new NoopDirectory());
return Status::OK();
}
}
// Something to return when mocking current time
const int64_t maybe_starting_time_;
Random rnd_;
port::Mutex rnd_mutex_; // Lock to pretect rnd_
// sstable Sync() calls are blocked while this pointer is non-nullptr.
std::atomic<bool> delay_sstable_sync_;
// Drop writes on the floor while this pointer is non-nullptr.
std::atomic<bool> drop_writes_;
// Simulate no-space errors while this pointer is non-nullptr.
std::atomic<bool> no_space_;
// Simulate non-writable file system while this pointer is non-nullptr
std::atomic<bool> non_writable_;
// Force sync of manifest files to fail while this pointer is non-nullptr
std::atomic<bool> manifest_sync_error_;
// Force write to manifest files to fail while this pointer is non-nullptr
std::atomic<bool> manifest_write_error_;
// Force write to log files to fail while this pointer is non-nullptr
std::atomic<bool> log_write_error_;
// Slow down every log write, in micro-seconds.
std::atomic<int> log_write_slowdown_;
Handle rename() failure in non-local FS (#8192) Summary: In a distributed environment, a file `rename()` operation can succeed on server (remote) side, but the client can somehow return non-ok status to RocksDB. Possible reasons include network partition, connection issue, etc. This happens in `rocksdb::SetCurrentFile()`, which can be called in `LogAndApply() -> ProcessManifestWrites()` if RocksDB tries to switch to a new MANIFEST. We currently always delete the new MANIFEST if an error occurs. This is problematic in distributed world. If the server-side successfully updates the CURRENT file via renaming, then a subsequent `DB::Open()` will try to look for the new MANIFEST and fail. As a fix, we can track the execution result of IO operations on the new MANIFEST. - If IO operations on the new MANIFEST fail, then we know the CURRENT must point to the original MANIFEST. Therefore, it is safe to remove the new MANIFEST. - If IO operations on the new MANIFEST all succeed, but somehow we end up in the clean up code block, then we do not know whether CURRENT points to the new or old MANIFEST. (For local POSIX-compliant FS, it should still point to old MANIFEST, but it does not matter if we keep the new MANIFEST.) Therefore, we keep the new MANIFEST. - Any future `LogAndApply()` will switch to a new MANIFEST and update CURRENT. - If process reopens the db immediately after the failure, then the CURRENT file can point to either the new MANIFEST or the old one, both of which exist. Therefore, recovery can succeed and ignore the other. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8192 Test Plan: make check Reviewed By: zhichao-cao Differential Revision: D27804648 Pulled By: riversand963 fbshipit-source-id: 9c16f2a5ce41bc6aadf085e48449b19ede8423e4
3 years ago
// If true, returns Status::NotSupported for file overwrite.
std::atomic<bool> no_file_overwrite_;
// Number of WAL files that are still open for write.
std::atomic<int> num_open_wal_file_;
bool count_random_reads_;
uint32_t rand_reads_fail_odd_ = 0;
std::atomic<uint64_t> num_reads_fails_;
anon::AtomicCounter random_read_counter_;
std::atomic<size_t> random_read_bytes_counter_;
std::atomic<int> random_file_open_counter_;
bool count_sequential_reads_;
anon::AtomicCounter sequential_read_counter_;
anon::AtomicCounter sleep_counter_;
std::atomic<int64_t> bytes_written_;
std::atomic<int> sync_counter_;
// If true, all fsync to files and directories are skipped.
bool skip_fsync_ = false;
// If true, ingest the corruption to file during sync.
bool corrupt_in_sync_ = false;
std::atomic<uint32_t> non_writeable_rate_;
std::atomic<uint32_t> new_writable_count_;
std::atomic<uint32_t> non_writable_count_;
std::function<void()>* table_write_callback_;
std::atomic<int> now_cpu_count_;
std::atomic<int> delete_count_;
std::atomic<bool> is_wal_sync_thread_safe_{true};
std::atomic<size_t> compaction_readahead_size_{};
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
4 years ago
private: // accessing these directly is prone to error
friend class DBTestBase;
std::atomic<int64_t> addon_microseconds_{0};
// Do not modify in the env of a running DB (could cause deadlock)
std::atomic<bool> time_elapse_only_sleep_;
bool no_slowdown_;
};
#ifndef ROCKSDB_LITE
class OnFileDeletionListener : public EventListener {
public:
OnFileDeletionListener() : matched_count_(0), expected_file_name_("") {}
const char* Name() const override { return kClassName(); }
static const char* kClassName() { return "OnFileDeletionListener"; }
void SetExpectedFileName(const std::string file_name) {
expected_file_name_ = file_name;
}
void VerifyMatchedCount(size_t expected_value) {
ASSERT_EQ(matched_count_, expected_value);
}
void OnTableFileDeleted(const TableFileDeletionInfo& info) override {
if (expected_file_name_ != "") {
ASSERT_EQ(expected_file_name_, info.file_path);
expected_file_name_ = "";
matched_count_++;
}
}
private:
size_t matched_count_;
std::string expected_file_name_;
};
class FlushCounterListener : public EventListener {
public:
const char* Name() const override { return kClassName(); }
static const char* kClassName() { return "FlushCounterListener"; }
std::atomic<int> count{0};
std::atomic<FlushReason> expected_flush_reason{FlushReason::kOthers};
void OnFlushBegin(DB* /*db*/, const FlushJobInfo& flush_job_info) override {
count++;
ASSERT_EQ(expected_flush_reason.load(), flush_job_info.flush_reason);
}
};
#endif
// A test merge operator mimics put but also fails if one of merge operands is
// "corrupted".
class TestPutOperator : public MergeOperator {
public:
virtual bool FullMergeV2(const MergeOperationInput& merge_in,
MergeOperationOutput* merge_out) const override {
if (merge_in.existing_value != nullptr &&
*(merge_in.existing_value) == "corrupted") {
return false;
}
for (auto value : merge_in.operand_list) {
if (value == "corrupted") {
return false;
}
}
merge_out->existing_operand = merge_in.operand_list.back();
return true;
}
virtual const char* Name() const override { return "TestPutOperator"; }
};
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
4 years ago
// A wrapper around Cache that can easily be extended with instrumentation,
// etc.
class CacheWrapper : public Cache {
public:
explicit CacheWrapper(std::shared_ptr<Cache> target)
: target_(std::move(target)) {}
const char* Name() const override { return target_->Name(); }
Initial support for secondary cache in LRUCache (#8271) Summary: Defined the abstract interface for a secondary cache in include/rocksdb/secondary_cache.h, and updated LRUCacheOptions to take a std::shared_ptr<SecondaryCache>. An item is initially inserted into the LRU (primary) cache. When it ages out and evicted from memory, its inserted into the secondary cache. On a LRU cache miss and successful lookup in the secondary cache, the item is promoted to the LRU cache. Only support synchronous lookup currently. The secondary cache would be used to implement a persistent (flash cache) or compressed cache. Tests: Results from cache_bench and db_bench don't show any regression due to these changes. cache_bench results before and after this change - Command ```./cache_bench -ops_per_thread=10000000 -threads=1``` Before ```Complete in 40.688 s; QPS = 245774``` ```Complete in 40.486 s; QPS = 246996``` ```Complete in 42.019 s; QPS = 237989``` After ```Complete in 40.672 s; QPS = 245869``` ```Complete in 44.622 s; QPS = 224107``` ```Complete in 42.445 s; QPS = 235599``` db_bench results before this change, and with this change + https://github.com/facebook/rocksdb/issues/8213 and https://github.com/facebook/rocksdb/issues/8191 - Commands ```./db_bench --benchmarks="fillseq,compact" -num=30000000 -key_size=32 -value_size=256 -use_direct_io_for_flush_and_compaction=true -db=/home/anand76/nvm_cache/db -partition_index_and_filters=true``` ```./db_bench -db=/home/anand76/nvm_cache/db -use_existing_db=true -benchmarks=readrandom -num=30000000 -key_size=32 -value_size=256 -use_direct_reads=true -cache_size=1073741824 -cache_numshardbits=6 -cache_index_and_filter_blocks=true -read_random_exp_range=17 -statistics -partition_index_and_filters=true -threads=16 -duration=300``` Before ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 80.702 micros/op 198104 ops/sec; 54.4 MB/s (3708999 of 3708999 found) ``` ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 87.124 micros/op 183625 ops/sec; 50.4 MB/s (3439999 of 3439999 found) ``` After ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 77.653 micros/op 206025 ops/sec; 56.6 MB/s (3866999 of 3866999 found) ``` ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 84.962 micros/op 188299 ops/sec; 51.7 MB/s (3535999 of 3535999 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/8271 Reviewed By: zhichao-cao Differential Revision: D28357511 Pulled By: anand1976 fbshipit-source-id: d1cfa236f00e649a18c53328be10a8062a4b6da2
3 years ago
using Cache::Insert;
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
4 years ago
Status Insert(const Slice& key, void* value, size_t charge,
void (*deleter)(const Slice& key, void* value),
Handle** handle = nullptr,
Priority priority = Priority::LOW) override {
return target_->Insert(key, value, charge, deleter, handle, priority);
}
Initial support for secondary cache in LRUCache (#8271) Summary: Defined the abstract interface for a secondary cache in include/rocksdb/secondary_cache.h, and updated LRUCacheOptions to take a std::shared_ptr<SecondaryCache>. An item is initially inserted into the LRU (primary) cache. When it ages out and evicted from memory, its inserted into the secondary cache. On a LRU cache miss and successful lookup in the secondary cache, the item is promoted to the LRU cache. Only support synchronous lookup currently. The secondary cache would be used to implement a persistent (flash cache) or compressed cache. Tests: Results from cache_bench and db_bench don't show any regression due to these changes. cache_bench results before and after this change - Command ```./cache_bench -ops_per_thread=10000000 -threads=1``` Before ```Complete in 40.688 s; QPS = 245774``` ```Complete in 40.486 s; QPS = 246996``` ```Complete in 42.019 s; QPS = 237989``` After ```Complete in 40.672 s; QPS = 245869``` ```Complete in 44.622 s; QPS = 224107``` ```Complete in 42.445 s; QPS = 235599``` db_bench results before this change, and with this change + https://github.com/facebook/rocksdb/issues/8213 and https://github.com/facebook/rocksdb/issues/8191 - Commands ```./db_bench --benchmarks="fillseq,compact" -num=30000000 -key_size=32 -value_size=256 -use_direct_io_for_flush_and_compaction=true -db=/home/anand76/nvm_cache/db -partition_index_and_filters=true``` ```./db_bench -db=/home/anand76/nvm_cache/db -use_existing_db=true -benchmarks=readrandom -num=30000000 -key_size=32 -value_size=256 -use_direct_reads=true -cache_size=1073741824 -cache_numshardbits=6 -cache_index_and_filter_blocks=true -read_random_exp_range=17 -statistics -partition_index_and_filters=true -threads=16 -duration=300``` Before ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 80.702 micros/op 198104 ops/sec; 54.4 MB/s (3708999 of 3708999 found) ``` ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 87.124 micros/op 183625 ops/sec; 50.4 MB/s (3439999 of 3439999 found) ``` After ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 77.653 micros/op 206025 ops/sec; 56.6 MB/s (3866999 of 3866999 found) ``` ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 84.962 micros/op 188299 ops/sec; 51.7 MB/s (3535999 of 3535999 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/8271 Reviewed By: zhichao-cao Differential Revision: D28357511 Pulled By: anand1976 fbshipit-source-id: d1cfa236f00e649a18c53328be10a8062a4b6da2
3 years ago
using Cache::Lookup;
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
4 years ago
Handle* Lookup(const Slice& key, Statistics* stats = nullptr) override {
return target_->Lookup(key, stats);
}
bool Ref(Handle* handle) override { return target_->Ref(handle); }
Initial support for secondary cache in LRUCache (#8271) Summary: Defined the abstract interface for a secondary cache in include/rocksdb/secondary_cache.h, and updated LRUCacheOptions to take a std::shared_ptr<SecondaryCache>. An item is initially inserted into the LRU (primary) cache. When it ages out and evicted from memory, its inserted into the secondary cache. On a LRU cache miss and successful lookup in the secondary cache, the item is promoted to the LRU cache. Only support synchronous lookup currently. The secondary cache would be used to implement a persistent (flash cache) or compressed cache. Tests: Results from cache_bench and db_bench don't show any regression due to these changes. cache_bench results before and after this change - Command ```./cache_bench -ops_per_thread=10000000 -threads=1``` Before ```Complete in 40.688 s; QPS = 245774``` ```Complete in 40.486 s; QPS = 246996``` ```Complete in 42.019 s; QPS = 237989``` After ```Complete in 40.672 s; QPS = 245869``` ```Complete in 44.622 s; QPS = 224107``` ```Complete in 42.445 s; QPS = 235599``` db_bench results before this change, and with this change + https://github.com/facebook/rocksdb/issues/8213 and https://github.com/facebook/rocksdb/issues/8191 - Commands ```./db_bench --benchmarks="fillseq,compact" -num=30000000 -key_size=32 -value_size=256 -use_direct_io_for_flush_and_compaction=true -db=/home/anand76/nvm_cache/db -partition_index_and_filters=true``` ```./db_bench -db=/home/anand76/nvm_cache/db -use_existing_db=true -benchmarks=readrandom -num=30000000 -key_size=32 -value_size=256 -use_direct_reads=true -cache_size=1073741824 -cache_numshardbits=6 -cache_index_and_filter_blocks=true -read_random_exp_range=17 -statistics -partition_index_and_filters=true -threads=16 -duration=300``` Before ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 80.702 micros/op 198104 ops/sec; 54.4 MB/s (3708999 of 3708999 found) ``` ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 87.124 micros/op 183625 ops/sec; 50.4 MB/s (3439999 of 3439999 found) ``` After ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 77.653 micros/op 206025 ops/sec; 56.6 MB/s (3866999 of 3866999 found) ``` ``` DB path: [/home/anand76/nvm_cache/db] readrandom : 84.962 micros/op 188299 ops/sec; 51.7 MB/s (3535999 of 3535999 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/8271 Reviewed By: zhichao-cao Differential Revision: D28357511 Pulled By: anand1976 fbshipit-source-id: d1cfa236f00e649a18c53328be10a8062a4b6da2
3 years ago
using Cache::Release;
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
4 years ago
bool Release(Handle* handle, bool force_erase = false) override {
return target_->Release(handle, force_erase);
}
void* Value(Handle* handle) override { return target_->Value(handle); }
void Erase(const Slice& key) override { target_->Erase(key); }
uint64_t NewId() override { return target_->NewId(); }
void SetCapacity(size_t capacity) override { target_->SetCapacity(capacity); }
void SetStrictCapacityLimit(bool strict_capacity_limit) override {
target_->SetStrictCapacityLimit(strict_capacity_limit);
}
bool HasStrictCapacityLimit() const override {
return target_->HasStrictCapacityLimit();
}
size_t GetCapacity() const override { return target_->GetCapacity(); }
size_t GetUsage() const override { return target_->GetUsage(); }
size_t GetUsage(Handle* handle) const override {
return target_->GetUsage(handle);
}
size_t GetPinnedUsage() const override { return target_->GetPinnedUsage(); }
size_t GetCharge(Handle* handle) const override {
return target_->GetCharge(handle);
}
Use deleters to label cache entries and collect stats (#8297) Summary: This change gathers and publishes statistics about the kinds of items in block cache. This is especially important for profiling relative usage of cache by index vs. filter vs. data blocks. It works by iterating over the cache during periodic stats dump (InternalStats, stats_dump_period_sec) or on demand when DB::Get(Map)Property(kBlockCacheEntryStats), except that for efficiency and sharing among column families, saved data from the last scan is used when the data is not considered too old. The new information can be seen in info LOG, for example: Block cache LRUCache@0x7fca62229330 capacity: 95.37 MB collections: 8 last_copies: 0 last_secs: 0.00178 secs_since: 0 Block cache entry stats(count,size,portion): DataBlock(7092,28.24 MB,29.6136%) FilterBlock(215,867.90 KB,0.888728%) FilterMetaBlock(2,5.31 KB,0.00544%) IndexBlock(217,180.11 KB,0.184432%) WriteBuffer(1,256.00 KB,0.262144%) Misc(1,0.00 KB,0%) And also through DB::GetProperty and GetMapProperty (here using ldb just for demonstration): $ ./ldb --db=/dev/shm/dbbench/ get_property rocksdb.block-cache-entry-stats rocksdb.block-cache-entry-stats.bytes.data-block: 0 rocksdb.block-cache-entry-stats.bytes.deprecated-filter-block: 0 rocksdb.block-cache-entry-stats.bytes.filter-block: 0 rocksdb.block-cache-entry-stats.bytes.filter-meta-block: 0 rocksdb.block-cache-entry-stats.bytes.index-block: 178992 rocksdb.block-cache-entry-stats.bytes.misc: 0 rocksdb.block-cache-entry-stats.bytes.other-block: 0 rocksdb.block-cache-entry-stats.bytes.write-buffer: 0 rocksdb.block-cache-entry-stats.capacity: 8388608 rocksdb.block-cache-entry-stats.count.data-block: 0 rocksdb.block-cache-entry-stats.count.deprecated-filter-block: 0 rocksdb.block-cache-entry-stats.count.filter-block: 0 rocksdb.block-cache-entry-stats.count.filter-meta-block: 0 rocksdb.block-cache-entry-stats.count.index-block: 215 rocksdb.block-cache-entry-stats.count.misc: 1 rocksdb.block-cache-entry-stats.count.other-block: 0 rocksdb.block-cache-entry-stats.count.write-buffer: 0 rocksdb.block-cache-entry-stats.id: LRUCache@0x7f3636661290 rocksdb.block-cache-entry-stats.percent.data-block: 0.000000 rocksdb.block-cache-entry-stats.percent.deprecated-filter-block: 0.000000 rocksdb.block-cache-entry-stats.percent.filter-block: 0.000000 rocksdb.block-cache-entry-stats.percent.filter-meta-block: 0.000000 rocksdb.block-cache-entry-stats.percent.index-block: 2.133751 rocksdb.block-cache-entry-stats.percent.misc: 0.000000 rocksdb.block-cache-entry-stats.percent.other-block: 0.000000 rocksdb.block-cache-entry-stats.percent.write-buffer: 0.000000 rocksdb.block-cache-entry-stats.secs_for_last_collection: 0.000052 rocksdb.block-cache-entry-stats.secs_since_last_collection: 0 Solution detail - We need some way to flag what kind of blocks each entry belongs to, preferably without changing the Cache API. One of the complications is that Cache is a general interface that could have other users that don't adhere to whichever convention we decide on for keys and values. Or we would pay for an extra field in the Handle that would only be used for this purpose. This change uses a back-door approach, the deleter, to indicate the "role" of a Cache entry (in addition to the value type, implicitly). This has the added benefit of ensuring proper code origin whenever we recognize a particular role for a cache entry; if the entry came from some other part of the code, it will use an unrecognized deleter, which we simply attribute to the "Misc" role. An internal API makes for simple instantiation and automatic registration of Cache deleters for a given value type and "role". Another internal API, CacheEntryStatsCollector, solves the problem of caching the results of a scan and sharing them, to ensure scans are neither excessive nor redundant so as not to harm Cache performance. Because code is added to BlocklikeTraits, it is pulled out of block_based_table_reader.cc into its own file. This is a reformulation of https://github.com/facebook/rocksdb/issues/8276, without the type checking option (could still be added), and with actual stat gathering. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8297 Test Plan: manual testing with db_bench, and a couple of basic unit tests Reviewed By: ltamasi Differential Revision: D28488721 Pulled By: pdillinger fbshipit-source-id: 472f524a9691b5afb107934be2d41d84f2b129fb
3 years ago
DeleterFn GetDeleter(Handle* handle) const override {
return target_->GetDeleter(handle);
}
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
4 years ago
void ApplyToAllCacheEntries(void (*callback)(void*, size_t),
bool thread_safe) override {
target_->ApplyToAllCacheEntries(callback, thread_safe);
}
New Cache API for gathering statistics (#8225) Summary: Adds a new Cache::ApplyToAllEntries API that we expect to use (in follow-up PRs) for efficiently gathering block cache statistics. Notable features vs. old ApplyToAllCacheEntries: * Includes key and deleter (in addition to value and charge). We could have passed in a Handle but then more virtual function calls would be needed to get the "fields" of each entry. We expect to use the 'deleter' to identify the origin of entries, perhaps even more. * Heavily tuned to minimize latency impact on operating cache. It does this by iterating over small sections of each cache shard while cycling through the shards. * Supports tuning roughly how many entries to operate on for each lock acquire and release, to control the impact on the latency of other operations without excessive lock acquire & release. The right balance can depend on the cost of the callback. Good default seems to be around 256. * There should be no need to disable thread safety. (I would expect uncontended locks to be sufficiently fast.) I have enhanced cache_bench to validate this approach: * Reports a histogram of ns per operation, so we can look at the ditribution of times, not just throughput (average). * Can add a thread for simulated "gather stats" which calls ApplyToAllEntries at a specified interval. We also generate a histogram of time to run ApplyToAllEntries. To make the iteration over some entries of each shard work as cleanly as possible, even with resize between next set of entries, I have re-arranged which hash bits are used for sharding and which for indexing within a shard. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8225 Test Plan: A couple of unit tests are added, but primary validation is manual, as the primary risk is to performance. The primary validation is using cache_bench to ensure that neither the minor hashing changes nor the simulated stats gathering significantly impact QPS or latency distribution. Note that adding op latency histogram seriously impacts the benchmark QPS, so for a fair baseline, we need the cache_bench changes (except remove simulated stat gathering to make it compile). In short, we don't see any reproducible difference in ops/sec or op latency unless we are gathering stats nearly continuously. Test uses 10GB block cache with 8KB values to be somewhat realistic in the number of items to iterate over. Baseline typical output: ``` Complete in 92.017 s; Rough parallel ops/sec = 869401 Thread ops/sec = 54662 Operation latency (ns): Count: 80000000 Average: 11223.9494 StdDev: 29.61 Min: 0 Median: 7759.3973 Max: 9620500 Percentiles: P50: 7759.40 P75: 14190.73 P99: 46922.75 P99.9: 77509.84 P99.99: 217030.58 ------------------------------------------------------ [ 0, 1 ] 68 0.000% 0.000% ( 2900, 4400 ] 89 0.000% 0.000% ( 4400, 6600 ] 33630240 42.038% 42.038% ######## ( 6600, 9900 ] 18129842 22.662% 64.700% ##### ( 9900, 14000 ] 7877533 9.847% 74.547% ## ( 14000, 22000 ] 15193238 18.992% 93.539% #### ( 22000, 33000 ] 3037061 3.796% 97.335% # ( 33000, 50000 ] 1626316 2.033% 99.368% ( 50000, 75000 ] 421532 0.527% 99.895% ( 75000, 110000 ] 56910 0.071% 99.966% ( 110000, 170000 ] 16134 0.020% 99.986% ( 170000, 250000 ] 5166 0.006% 99.993% ( 250000, 380000 ] 3017 0.004% 99.996% ( 380000, 570000 ] 1337 0.002% 99.998% ( 570000, 860000 ] 805 0.001% 99.999% ( 860000, 1200000 ] 319 0.000% 100.000% ( 1200000, 1900000 ] 231 0.000% 100.000% ( 1900000, 2900000 ] 100 0.000% 100.000% ( 2900000, 4300000 ] 39 0.000% 100.000% ( 4300000, 6500000 ] 16 0.000% 100.000% ( 6500000, 9800000 ] 7 0.000% 100.000% ``` New, gather_stats=false. Median thread ops/sec of 5 runs: ``` Complete in 92.030 s; Rough parallel ops/sec = 869285 Thread ops/sec = 54458 Operation latency (ns): Count: 80000000 Average: 11298.1027 StdDev: 42.18 Min: 0 Median: 7722.0822 Max: 6398720 Percentiles: P50: 7722.08 P75: 14294.68 P99: 47522.95 P99.9: 85292.16 P99.99: 228077.78 ------------------------------------------------------ [ 0, 1 ] 109 0.000% 0.000% ( 2900, 4400 ] 793 0.001% 0.001% ( 4400, 6600 ] 34054563 42.568% 42.569% ######### ( 6600, 9900 ] 17482646 21.853% 64.423% #### ( 9900, 14000 ] 7908180 9.885% 74.308% ## ( 14000, 22000 ] 15032072 18.790% 93.098% #### ( 22000, 33000 ] 3237834 4.047% 97.145% # ( 33000, 50000 ] 1736882 2.171% 99.316% ( 50000, 75000 ] 446851 0.559% 99.875% ( 75000, 110000 ] 68251 0.085% 99.960% ( 110000, 170000 ] 18592 0.023% 99.983% ( 170000, 250000 ] 7200 0.009% 99.992% ( 250000, 380000 ] 3334 0.004% 99.997% ( 380000, 570000 ] 1393 0.002% 99.998% ( 570000, 860000 ] 700 0.001% 99.999% ( 860000, 1200000 ] 293 0.000% 100.000% ( 1200000, 1900000 ] 196 0.000% 100.000% ( 1900000, 2900000 ] 69 0.000% 100.000% ( 2900000, 4300000 ] 32 0.000% 100.000% ( 4300000, 6500000 ] 10 0.000% 100.000% ``` New, gather_stats=true, 1 second delay between scans. Scans take about 1 second here so it's spending about 50% time scanning. Still the effect on ops/sec and latency seems to be in the noise. Median thread ops/sec of 5 runs: ``` Complete in 91.890 s; Rough parallel ops/sec = 870608 Thread ops/sec = 54551 Operation latency (ns): Count: 80000000 Average: 11311.2629 StdDev: 45.28 Min: 0 Median: 7686.5458 Max: 10018340 Percentiles: P50: 7686.55 P75: 14481.95 P99: 47232.60 P99.9: 79230.18 P99.99: 232998.86 ------------------------------------------------------ [ 0, 1 ] 71 0.000% 0.000% ( 2900, 4400 ] 291 0.000% 0.000% ( 4400, 6600 ] 34492060 43.115% 43.116% ######### ( 6600, 9900 ] 16727328 20.909% 64.025% #### ( 9900, 14000 ] 7845828 9.807% 73.832% ## ( 14000, 22000 ] 15510654 19.388% 93.220% #### ( 22000, 33000 ] 3216533 4.021% 97.241% # ( 33000, 50000 ] 1680859 2.101% 99.342% ( 50000, 75000 ] 439059 0.549% 99.891% ( 75000, 110000 ] 60540 0.076% 99.967% ( 110000, 170000 ] 14649 0.018% 99.985% ( 170000, 250000 ] 5242 0.007% 99.991% ( 250000, 380000 ] 3260 0.004% 99.995% ( 380000, 570000 ] 1599 0.002% 99.997% ( 570000, 860000 ] 1043 0.001% 99.999% ( 860000, 1200000 ] 471 0.001% 99.999% ( 1200000, 1900000 ] 275 0.000% 100.000% ( 1900000, 2900000 ] 143 0.000% 100.000% ( 2900000, 4300000 ] 60 0.000% 100.000% ( 4300000, 6500000 ] 27 0.000% 100.000% ( 6500000, 9800000 ] 7 0.000% 100.000% ( 9800000, 14000000 ] 1 0.000% 100.000% Gather stats latency (us): Count: 46 Average: 980387.5870 StdDev: 60911.18 Min: 879155 Median: 1033777.7778 Max: 1261431 Percentiles: P50: 1033777.78 P75: 1120666.67 P99: 1261431.00 P99.9: 1261431.00 P99.99: 1261431.00 ------------------------------------------------------ ( 860000, 1200000 ] 45 97.826% 97.826% #################### ( 1200000, 1900000 ] 1 2.174% 100.000% Most recent cache entry stats: Number of entries: 1295133 Total charge: 9.88 GB Average key size: 23.4982 Average charge: 8.00 KB Unique deleters: 3 ``` Reviewed By: mrambacher Differential Revision: D28295742 Pulled By: pdillinger fbshipit-source-id: bbc4a552f91ba0fe10e5cc025c42cef5a81f2b95
3 years ago
void ApplyToAllEntries(
const std::function<void(const Slice& key, void* value, size_t charge,
DeleterFn deleter)>& callback,
const ApplyToAllEntriesOptions& opts) override {
target_->ApplyToAllEntries(callback, opts);
}
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
4 years ago
void EraseUnRefEntries() override { target_->EraseUnRefEntries(); }
protected:
std::shared_ptr<Cache> target_;
};
class DBTestBase : public testing::Test {
public:
// Sequence of option configurations to try
enum OptionConfig : int {
kDefault = 0,
kBlockBasedTableWithPrefixHashIndex = 1,
kBlockBasedTableWithWholeKeyHashIndex = 2,
kPlainTableFirstBytePrefix = 3,
kPlainTableCappedPrefix = 4,
kPlainTableCappedPrefixNonMmap = 5,
kPlainTableAllBytesPrefix = 6,
kVectorRep = 7,
kHashLinkList = 8,
kMergePut = 9,
kFilter = 10,
kFullFilterWithNewTableReaderForCompactions = 11,
kUncompressed = 12,
kNumLevel_3 = 13,
kDBLogDir = 14,
kWalDirAndMmapReads = 15,
kManifestFileSize = 16,
kPerfOptions = 17,
kHashSkipList = 18,
kUniversalCompaction = 19,
kUniversalCompactionMultiLevel = 20,
kCompressedBlockCache = 21,
kInfiniteMaxOpenFiles = 22,
kxxHashChecksum = 23,
kFIFOCompaction = 24,
kOptimizeFiltersForHits = 25,
kRowCache = 26,
kRecycleLogFiles = 27,
kConcurrentSkipList = 28,
kPipelinedWrite = 29,
kConcurrentWALWrites = 30,
kDirectIO,
kLevelSubcompactions,
kBlockBasedTableWithIndexRestartInterval,
kBlockBasedTableWithPartitionedIndex,
kBlockBasedTableWithPartitionedIndexFormat4,
kPartitionedFilterWithNewTableReaderForCompactions,
kUniversalSubcompactions,
kxxHash64Checksum,
Unordered Writes (#5218) Summary: Performing unordered writes in rocksdb when unordered_write option is set to true. When enabled the writes to memtable are done without joining any write thread. This offers much higher write throughput since the upcoming writes would not have to wait for the slowest memtable write to finish. The tradeoff is that the writes visible to a snapshot might change over time. If the application cannot tolerate that, it should implement its own mechanisms to work around that. Using TransactionDB with WRITE_PREPARED write policy is one way to achieve that. Doing so increases the max throughput by 2.2x without however compromising the snapshot guarantees. The patch is prepared based on an original by siying Existing unit tests are extended to include unordered_write option. Benchmark Results: ``` TEST_TMPDIR=/dev/shm/ ./db_bench_unordered --benchmarks=fillrandom --threads=32 --num=10000000 -max_write_buffer_number=16 --max_background_jobs=64 --batch_size=8 --writes=3000000 -level0_file_num_compaction_trigger=99999 --level0_slowdown_writes_trigger=99999 --level0_stop_writes_trigger=99999 -enable_pipelined_write=false -disable_auto_compactions --unordered_write=1 ``` With WAL - Vanilla RocksDB: 78.6 MB/s - WRITER_PREPARED with unordered_write: 177.8 MB/s (2.2x) - unordered_write: 368.9 MB/s (4.7x with relaxed snapshot guarantees) Without WAL - Vanilla RocksDB: 111.3 MB/s - WRITER_PREPARED with unordered_write: 259.3 MB/s MB/s (2.3x) - unordered_write: 645.6 MB/s (5.8x with relaxed snapshot guarantees) - WRITER_PREPARED with unordered_write disable concurrency control: 185.3 MB/s MB/s (2.35x) Limitations: - The feature is not yet extended to `max_successive_merges` > 0. The feature is also incompatible with `enable_pipelined_write` = true as well as with `allow_concurrent_memtable_write` = false. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5218 Differential Revision: D15219029 Pulled By: maysamyabandeh fbshipit-source-id: 38f2abc4af8780148c6128acdba2b3227bc81759
5 years ago
kUnorderedWrite,
// This must be the last line
kEnd,
};
public:
std::string dbname_;
std::string alternative_wal_dir_;
std::string alternative_db_log_dir_;
MockEnv* mem_env_;
Encryption at rest support Summary: This PR adds support for encrypting data stored by RocksDB when written to disk. It adds an `EncryptedEnv` override of the `Env` class with matching overrides for sequential&random access files. The encryption itself is done through a configurable `EncryptionProvider`. This class creates is asked to create `BlockAccessCipherStream` for a file. This is where the actual encryption/decryption is being done. Currently there is a Counter mode implementation of `BlockAccessCipherStream` with a `ROT13` block cipher (NOTE the `ROT13` is for demo purposes only!!). The Counter operation mode uses an initial counter & random initialization vector (IV). Both are created randomly for each file and stored in a 4K (default size) block that is prefixed to that file. The `EncryptedEnv` implementation is such that clients of the `Env` class do not see this prefix (nor data, nor in filesize). The largest part of the prefix block is also encrypted, and there is room left for implementation specific settings/values/keys in there. To test the encryption, the `DBTestBase` class has been extended to consider a new environment variable called `ENCRYPTED_ENV`. If set, the test will setup a encrypted instance of the `Env` class to use for all tests. Typically you would run it like this: ``` ENCRYPTED_ENV=1 make check_some ``` There is also an added test that checks that some data inserted into the database is or is not "visible" on disk. With `ENCRYPTED_ENV` active it must not find plain text strings, with `ENCRYPTED_ENV` unset, it must find the plain text strings. Closes https://github.com/facebook/rocksdb/pull/2424 Differential Revision: D5322178 Pulled By: sdwilsh fbshipit-source-id: 253b0a9c2c498cc98f580df7f2623cbf7678a27f
7 years ago
Env* encrypted_env_;
SpecialEnv* env_;
std::shared_ptr<Env> env_guard_;
DB* db_;
std::vector<ColumnFamilyHandle*> handles_;
int option_config_;
Options last_options_;
// Skip some options, as they may not be applicable to a specific test.
// To add more skip constants, use values 4, 8, 16, etc.
enum OptionSkip {
kNoSkip = 0,
kSkipDeletesFilterFirst = 1,
kSkipUniversalCompaction = 2,
kSkipMergePut = 4,
kSkipPlainTable = 8,
kSkipHashIndex = 16,
kSkipNoSeekToLast = 32,
kSkipFIFOCompaction = 128,
kSkipMmapReads = 256,
};
const int kRangeDelSkipConfigs =
// Plain tables do not support range deletions.
kSkipPlainTable |
// MmapReads disables the iterator pinning that RangeDelAggregator
// requires.
kSkipMmapReads;
// `env_do_fsync` decides whether the special Env would do real
// fsync for files and directories. Skipping fsync can speed up
// tests, but won't cover the exact fsync logic.
DBTestBase(const std::string path, bool env_do_fsync);
~DBTestBase();
static std::string Key(int i) {
char buf[100];
snprintf(buf, sizeof(buf), "key%06d", i);
return std::string(buf);
}
static bool ShouldSkipOptions(int option_config, int skip_mask = kNoSkip);
// Switch to a fresh database with the next option configuration to
// test. Return false if there are no more configurations to test.
bool ChangeOptions(int skip_mask = kNoSkip);
// Switch between different compaction styles.
bool ChangeCompactOptions();
// Switch between different WAL-realted options.
bool ChangeWalOptions();
// Switch between different filter policy
// Jump from kDefault to kFilter to kFullFilter
bool ChangeFilterOptions();
// Switch between different DB options for file ingestion tests.
bool ChangeOptionsForFileIngestionTest();
// Return the current option configuration.
Options CurrentOptions(const anon::OptionsOverride& options_override =
anon::OptionsOverride()) const;
Options CurrentOptions(const Options& default_options,
const anon::OptionsOverride& options_override =
anon::OptionsOverride()) const;
Fix many tests to run with MEM_ENV and ENCRYPTED_ENV; Introduce a MemoryFileSystem class (#7566) Summary: This PR does a few things: 1. The MockFileSystem class was split out from the MockEnv. This change would theoretically allow a MockFileSystem to be used by other Environments as well (if we created a means of constructing one). The MockFileSystem implements a FileSystem in its entirety and does not rely on any Wrapper implementation. 2. Make the RocksDB test suite work when MOCK_ENV=1 and ENCRYPTED_ENV=1 are set. To accomplish this, a few things were needed: - The tests that tried to use the "wrong" environment (Env::Default() instead of env_) were updated - The MockFileSystem was changed to support the features it was missing or mishandled (such as recursively deleting files in a directory or supporting renaming of a directory). 3. Updated the test framework to have a ROCKSDB_GTEST_SKIP macro. This can be used to flag tests that are skipped. Currently, this defaults to doing nothing (marks the test as SUCCESS) but will mark the tests as SKIPPED when RocksDB is upgraded to a version of gtest that supports this (gtest-1.10). I have run a full "make check" with MEM_ENV, ENCRYPTED_ENV, both, and neither under both MacOS and RedHat. A few tests were disabled/skipped for the MEM/ENCRYPTED cases. The error_handler_fs_test fails/hangs for MEM_ENV (presumably a timing problem) and I will introduce another PR/issue to track that problem. (I will also push a change to disable those tests soon). There is one more test in DBTest2 that also fails which I need to investigate or skip before this PR is merged. Theoretically, this PR should also allow the test suite to run against an Env loaded from the registry, though I do not have one to try it with currently. Finally, once this is accepted, it would be nice if there was a CircleCI job to run these tests on a checkin so this effort does not become stale. I do not know how to do that, so if someone could write that job, it would be appreciated :) Pull Request resolved: https://github.com/facebook/rocksdb/pull/7566 Reviewed By: zhichao-cao Differential Revision: D24408980 Pulled By: jay-zhuang fbshipit-source-id: 911b1554a4d0da06fd51feca0c090a4abdcb4a5f
4 years ago
Options GetDefaultOptions() const;
Fix many tests to run with MEM_ENV and ENCRYPTED_ENV; Introduce a MemoryFileSystem class (#7566) Summary: This PR does a few things: 1. The MockFileSystem class was split out from the MockEnv. This change would theoretically allow a MockFileSystem to be used by other Environments as well (if we created a means of constructing one). The MockFileSystem implements a FileSystem in its entirety and does not rely on any Wrapper implementation. 2. Make the RocksDB test suite work when MOCK_ENV=1 and ENCRYPTED_ENV=1 are set. To accomplish this, a few things were needed: - The tests that tried to use the "wrong" environment (Env::Default() instead of env_) were updated - The MockFileSystem was changed to support the features it was missing or mishandled (such as recursively deleting files in a directory or supporting renaming of a directory). 3. Updated the test framework to have a ROCKSDB_GTEST_SKIP macro. This can be used to flag tests that are skipped. Currently, this defaults to doing nothing (marks the test as SUCCESS) but will mark the tests as SKIPPED when RocksDB is upgraded to a version of gtest that supports this (gtest-1.10). I have run a full "make check" with MEM_ENV, ENCRYPTED_ENV, both, and neither under both MacOS and RedHat. A few tests were disabled/skipped for the MEM/ENCRYPTED cases. The error_handler_fs_test fails/hangs for MEM_ENV (presumably a timing problem) and I will introduce another PR/issue to track that problem. (I will also push a change to disable those tests soon). There is one more test in DBTest2 that also fails which I need to investigate or skip before this PR is merged. Theoretically, this PR should also allow the test suite to run against an Env loaded from the registry, though I do not have one to try it with currently. Finally, once this is accepted, it would be nice if there was a CircleCI job to run these tests on a checkin so this effort does not become stale. I do not know how to do that, so if someone could write that job, it would be appreciated :) Pull Request resolved: https://github.com/facebook/rocksdb/pull/7566 Reviewed By: zhichao-cao Differential Revision: D24408980 Pulled By: jay-zhuang fbshipit-source-id: 911b1554a4d0da06fd51feca0c090a4abdcb4a5f
4 years ago
Options GetOptions(int option_config) const {
return GetOptions(option_config, GetDefaultOptions());
}
Options GetOptions(int option_config, const Options& default_options,
const anon::OptionsOverride& options_override =
anon::OptionsOverride()) const;
DBImpl* dbfull() { return static_cast_with_check<DBImpl>(db_); }
void CreateColumnFamilies(const std::vector<std::string>& cfs,
const Options& options);
void CreateAndReopenWithCF(const std::vector<std::string>& cfs,
const Options& options);
void ReopenWithColumnFamilies(const std::vector<std::string>& cfs,
const std::vector<Options>& options);
void ReopenWithColumnFamilies(const std::vector<std::string>& cfs,
const Options& options);
Status TryReopenWithColumnFamilies(const std::vector<std::string>& cfs,
const std::vector<Options>& options);
Status TryReopenWithColumnFamilies(const std::vector<std::string>& cfs,
const Options& options);
void Reopen(const Options& options);
void Close();
void DestroyAndReopen(const Options& options);
void Destroy(const Options& options, bool delete_cf_paths = false);
Status ReadOnlyReopen(const Options& options);
Status TryReopen(const Options& options);
bool IsDirectIOSupported();
Encryption at rest support Summary: This PR adds support for encrypting data stored by RocksDB when written to disk. It adds an `EncryptedEnv` override of the `Env` class with matching overrides for sequential&random access files. The encryption itself is done through a configurable `EncryptionProvider`. This class creates is asked to create `BlockAccessCipherStream` for a file. This is where the actual encryption/decryption is being done. Currently there is a Counter mode implementation of `BlockAccessCipherStream` with a `ROT13` block cipher (NOTE the `ROT13` is for demo purposes only!!). The Counter operation mode uses an initial counter & random initialization vector (IV). Both are created randomly for each file and stored in a 4K (default size) block that is prefixed to that file. The `EncryptedEnv` implementation is such that clients of the `Env` class do not see this prefix (nor data, nor in filesize). The largest part of the prefix block is also encrypted, and there is room left for implementation specific settings/values/keys in there. To test the encryption, the `DBTestBase` class has been extended to consider a new environment variable called `ENCRYPTED_ENV`. If set, the test will setup a encrypted instance of the `Env` class to use for all tests. Typically you would run it like this: ``` ENCRYPTED_ENV=1 make check_some ``` There is also an added test that checks that some data inserted into the database is or is not "visible" on disk. With `ENCRYPTED_ENV` active it must not find plain text strings, with `ENCRYPTED_ENV` unset, it must find the plain text strings. Closes https://github.com/facebook/rocksdb/pull/2424 Differential Revision: D5322178 Pulled By: sdwilsh fbshipit-source-id: 253b0a9c2c498cc98f580df7f2623cbf7678a27f
7 years ago
bool IsMemoryMappedAccessSupported() const;
Status Flush(int cf = 0);
Status Flush(const std::vector<int>& cf_ids);
Status Put(const Slice& k, const Slice& v, WriteOptions wo = WriteOptions());
Status Put(int cf, const Slice& k, const Slice& v,
WriteOptions wo = WriteOptions());
Status Merge(const Slice& k, const Slice& v,
WriteOptions wo = WriteOptions());
Status Merge(int cf, const Slice& k, const Slice& v,
WriteOptions wo = WriteOptions());
Status Delete(const std::string& k);
Status Delete(int cf, const std::string& k);
Support for SingleDelete() Summary: This patch fixes #7460559. It introduces SingleDelete as a new database operation. This operation can be used to delete keys that were never overwritten (no put following another put of the same key). If an overwritten key is single deleted the behavior is undefined. Single deletion of a non-existent key has no effect but multiple consecutive single deletions are not allowed (see limitations). In contrast to the conventional Delete() operation, the deletion entry is removed along with the value when the two are lined up in a compaction. Note: The semantics are similar to @igor's prototype that allowed to have this behavior on the granularity of a column family ( https://reviews.facebook.net/D42093 ). This new patch, however, is more aggressive when it comes to removing tombstones: It removes the SingleDelete together with the value whenever there is no snapshot between them while the older patch only did this when the sequence number of the deletion was older than the earliest snapshot. Most of the complex additions are in the Compaction Iterator, all other changes should be relatively straightforward. The patch also includes basic support for single deletions in db_stress and db_bench. Limitations: - Not compatible with cuckoo hash tables - Single deletions cannot be used in combination with merges and normal deletions on the same key (other keys are not affected by this) - Consecutive single deletions are currently not allowed (and older version of this patch supported this so it could be resurrected if needed) Test Plan: make all check Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor Reviewed By: igor Subscribers: maykov, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D43179
9 years ago
Status SingleDelete(const std::string& k);
Status SingleDelete(int cf, const std::string& k);
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
7 years ago
bool SetPreserveDeletesSequenceNumber(SequenceNumber sn);
std::string Get(const std::string& k, const Snapshot* snapshot = nullptr);
std::string Get(int cf, const std::string& k,
const Snapshot* snapshot = nullptr);
Status Get(const std::string& k, PinnableSlice* v);
std::vector<std::string> MultiGet(std::vector<int> cfs,
const std::vector<std::string>& k,
const Snapshot* snapshot,
const bool batched);
Introduce a new MultiGet batching implementation (#5011) Summary: This PR introduces a new MultiGet() API, with the underlying implementation grouping keys based on SST file and batching lookups in a file. The reason for the new API is twofold - the definition allows callers to allocate storage for status and values on stack instead of std::vector, as well as return values as PinnableSlices in order to avoid copying, and it keeps the original MultiGet() implementation intact while we experiment with batching. Batching is useful when there is some spatial locality to the keys being queries, as well as larger batch sizes. The main benefits are due to - 1. Fewer function calls, especially to BlockBasedTableReader::MultiGet() and FullFilterBlockReader::KeysMayMatch() 2. Bloom filter cachelines can be prefetched, hiding the cache miss latency The next step is to optimize the binary searches in the level_storage_info, index blocks and data blocks, since we could reduce the number of key comparisons if the keys are relatively close to each other. The batching optimizations also need to be extended to other formats, such as PlainTable and filter formats. This also needs to be added to db_stress. Benchmark results from db_bench for various batch size/locality of reference combinations are given below. Locality was simulated by offsetting the keys in a batch by a stride length. Each SST file is about 8.6MB uncompressed and key/value size is 16/100 uncompressed. To focus on the cpu benefit of batching, the runs were single threaded and bound to the same cpu to eliminate interference from other system events. The results show a 10-25% improvement in micros/op from smaller to larger batch sizes (4 - 32). Batch Sizes 1 | 2 | 4 | 8 | 16 | 32 Random pattern (Stride length 0) 4.158 | 4.109 | 4.026 | 4.05 | 4.1 | 4.074 - Get 4.438 | 4.302 | 4.165 | 4.122 | 4.096 | 4.075 - MultiGet (no batching) 4.461 | 4.256 | 4.277 | 4.11 | 4.182 | 4.14 - MultiGet (w/ batching) Good locality (Stride length 16) 4.048 | 3.659 | 3.248 | 2.99 | 2.84 | 2.753 4.429 | 3.728 | 3.406 | 3.053 | 2.911 | 2.781 4.452 | 3.45 | 2.833 | 2.451 | 2.233 | 2.135 Good locality (Stride length 256) 4.066 | 3.786 | 3.581 | 3.447 | 3.415 | 3.232 4.406 | 4.005 | 3.644 | 3.49 | 3.381 | 3.268 4.393 | 3.649 | 3.186 | 2.882 | 2.676 | 2.62 Medium locality (Stride length 4096) 4.012 | 3.922 | 3.768 | 3.61 | 3.582 | 3.555 4.364 | 4.057 | 3.791 | 3.65 | 3.57 | 3.465 4.479 | 3.758 | 3.316 | 3.077 | 2.959 | 2.891 dbbench command used (on a DB with 4 levels, 12 million keys)- TEST_TMPDIR=/dev/shm numactl -C 10 ./db_bench.tmp -use_existing_db=true -benchmarks="readseq,multireadrandom" -write_buffer_size=4194304 -target_file_size_base=4194304 -max_bytes_for_level_base=16777216 -num=12000000 -reads=12000000 -duration=90 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 Pull Request resolved: https://github.com/facebook/rocksdb/pull/5011 Differential Revision: D14348703 Pulled By: anand1976 fbshipit-source-id: 774406dab3776d979c809522a67bedac6c17f84b
5 years ago
std::vector<std::string> MultiGet(const std::vector<std::string>& k,
const Snapshot* snapshot = nullptr);
uint64_t GetNumSnapshots();
uint64_t GetTimeOldestSnapshots();
uint64_t GetSequenceOldestSnapshots();
// Return a string that contains all key,value pairs in order,
// formatted like "(k1->v1)(k2->v2)".
std::string Contents(int cf = 0);
std::string AllEntriesFor(const Slice& user_key, int cf = 0);
#ifndef ROCKSDB_LITE
int NumSortedRuns(int cf = 0);
uint64_t TotalSize(int cf = 0);
uint64_t SizeAtLevel(int level);
size_t TotalLiveFiles(int cf = 0);
size_t CountLiveFiles();
int NumTableFilesAtLevel(int level, int cf = 0);
double CompressionRatioAtLevel(int level, int cf = 0);
int TotalTableFiles(int cf = 0, int levels = -1);
#endif // ROCKSDB_LITE
std::vector<uint64_t> GetBlobFileNumbers();
// Return spread of files per level
std::string FilesPerLevel(int cf = 0);
size_t CountFiles();
Status CountFiles(size_t* count);
Status Size(const Slice& start, const Slice& limit, uint64_t* size) {
return Size(start, limit, 0, size);
}
Status Size(const Slice& start, const Slice& limit, int cf, uint64_t* size);
void Compact(int cf, const Slice& start, const Slice& limit,
uint32_t target_path_id);
void Compact(int cf, const Slice& start, const Slice& limit);
void Compact(const Slice& start, const Slice& limit);
// Do n memtable compactions, each of which produces an sstable
// covering the range [small,large].
void MakeTables(int n, const std::string& small, const std::string& large,
int cf = 0);
// Prevent pushing of new sstables into deeper levels by adding
// tables that cover a specified range to all levels.
void FillLevels(const std::string& smallest, const std::string& largest,
int cf);
void MoveFilesToLevel(int level, int cf = 0);
#ifndef ROCKSDB_LITE
void DumpFileCounts(const char* label);
#endif // ROCKSDB_LITE
std::string DumpSSTableList();
static void GetSstFiles(Env* env, std::string path,
std::vector<std::string>* files);
int GetSstFileCount(std::string path);
// this will generate non-overlapping files since it keeps increasing key_idx
void GenerateNewFile(Random* rnd, int* key_idx, bool nowait = false);
void GenerateNewFile(int fd, Random* rnd, int* key_idx, bool nowait = false);
static const int kNumKeysByGenerateNewRandomFile;
static const int KNumKeysByGenerateNewFile = 100;
void GenerateNewRandomFile(Random* rnd, bool nowait = false);
std::string IterStatus(Iterator* iter);
Options OptionsForLogIterTest();
std::string DummyString(size_t len, char c = 'a');
void VerifyIterLast(std::string expected_key, int cf = 0);
// Used to test InplaceUpdate
// If previous value is nullptr or delta is > than previous value,
// sets newValue with delta
// If previous value is not empty,
// updates previous value with 'b' string of previous value size - 1.
static UpdateStatus updateInPlaceSmallerSize(char* prevValue,
uint32_t* prevSize, Slice delta,
std::string* newValue);
static UpdateStatus updateInPlaceSmallerVarintSize(char* prevValue,
uint32_t* prevSize,
Slice delta,
std::string* newValue);
static UpdateStatus updateInPlaceLargerSize(char* prevValue,
uint32_t* prevSize, Slice delta,
std::string* newValue);
static UpdateStatus updateInPlaceNoAction(char* prevValue, uint32_t* prevSize,
Slice delta, std::string* newValue);
// Utility method to test InplaceUpdate
void validateNumberOfEntries(int numValues, int cf = 0);
void CopyFile(const std::string& source, const std::string& destination,
uint64_t size = 0);
Status GetAllDataFiles(const FileType file_type,
std::unordered_map<std::string, uint64_t>* sst_files,
uint64_t* total_size = nullptr);
std::vector<std::uint64_t> ListTableFiles(Env* env, const std::string& path);
void VerifyDBFromMap(
std::map<std::string, std::string> true_data,
size_t* total_reads_res = nullptr, bool tailing_iter = false,
std::map<std::string, Status> status = std::map<std::string, Status>());
void VerifyDBInternal(
std::vector<std::pair<std::string, std::string>> true_data);
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
8 years ago
#ifndef ROCKSDB_LITE
uint64_t GetNumberOfSstFilesForColumnFamily(DB* db,
std::string column_family_name);
#endif // ROCKSDB_LITE
uint64_t TestGetTickerCount(const Options& options, Tickers ticker_type) {
return options.statistics->getTickerCount(ticker_type);
}
uint64_t TestGetAndResetTickerCount(const Options& options,
Tickers ticker_type) {
return options.statistics->getAndResetTickerCount(ticker_type);
}
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
4 years ago
// Note: reverting this setting within the same test run is not yet
// supported
void SetTimeElapseOnlySleepOnReopen(DBOptions* options);
private: // Prone to error on direct use
void MaybeInstallTimeElapseOnlySleep(const DBOptions& options);
bool time_elapse_only_sleep_on_reopen_ = false;
};
} // namespace ROCKSDB_NAMESPACE