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

887 lines
28 KiB

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same 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
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
#endif
#include <fcntl.h>
#include <inttypes.h>
#include <algorithm>
#include <map>
#include <set>
#include <string>
#include <thread>
#include <unordered_set>
#include <utility>
#include <vector>
#include "db/db_impl.h"
#include "db/dbformat.h"
#include "env/mock_env.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/block_based_table_factory.h"
#include "table/mock_table.h"
#include "table/plain_table_factory.h"
#include "table/scoped_arena_iterator.h"
#include "util/compression.h"
#include "util/filename.h"
#include "util/mutexlock.h"
#include "util/string_util.h"
// SyncPoint is not supported in Released Windows Mode.
#if !(defined NDEBUG) || !defined(OS_WIN)
#include "util/sync_point.h"
#endif // !(defined NDEBUG) || !defined(OS_WIN)
#include "util/testharness.h"
#include "util/testutil.h"
#include "utilities/merge_operators.h"
namespace rocksdb {
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;
BlockBasedTableOptions::IndexType index_type =
BlockBasedTableOptions::IndexType::kBinarySearch;
// 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(MemTableAllocator* 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 {
memtable_->Insert(handle);
num_entries_++;
}
// 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:
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) {}
virtual MemTableRep* CreateMemTableRep(
const MemTableRep::KeyComparator& compare, MemTableAllocator* 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:
explicit SpecialEnv(Env* base);
Status NewWritableFile(const std::string& f, unique_ptr<WritableFile>* r,
const EnvOptions& soptions) override {
class SSTableFile : public WritableFile {
private:
SpecialEnv* env_;
unique_ptr<WritableFile> base_;
public:
SSTableFile(SpecialEnv* env, 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 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 Truncate(uint64_t size) override { return base_->Truncate(size); }
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)
return base_->Close();
}
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);
}
return base_->Sync();
}
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();
}
};
class ManifestFile : public WritableFile {
public:
ManifestFile(SpecialEnv* env, 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);
}
}
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 {
return base_->Sync();
}
}
uint64_t GetFileSize() override { return base_->GetFileSize(); }
private:
SpecialEnv* env_;
unique_ptr<WritableFile> base_;
};
class WalFile : public WritableFile {
public:
WalFile(SpecialEnv* env, 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;
}
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 {
// 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("DBTestWalFile.GetPreallocationStatus",
&preallocation_size);
#endif // !(defined NDEBUG) || !defined(OS_WIN)
return base_->Close();
}
Status Flush() override { return base_->Flush(); }
Status Sync() override {
++env_->sync_counter_;
return base_->Sync();
}
bool IsSyncThreadSafe() const override {
return env_->is_wal_sync_thread_safe_.load();
}
private:
SpecialEnv* env_;
unique_ptr<WritableFile> base_;
};
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)));
}
}
return s;
}
Status NewRandomAccessFile(const std::string& f,
unique_ptr<RandomAccessFile>* r,
const EnvOptions& soptions) override {
class CountingFile : public RandomAccessFile {
public:
CountingFile(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;
}
private:
unique_ptr<RandomAccessFile> target_;
anon::AtomicCounter* counter_;
std::atomic<size_t>* bytes_read_;
};
Status s = target()->NewRandomAccessFile(f, r, soptions);
random_file_open_counter_++;
if (s.ok() && count_random_reads_) {
r->reset(new CountingFile(std::move(*r), &random_read_counter_,
&random_read_bytes_counter_));
}
return s;
}
Status NewSequentialFile(const std::string& f, unique_ptr<SequentialFile>* r,
const EnvOptions& soptions) override {
class CountingFile : public SequentialFile {
public:
CountingFile(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:
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_) {
addon_time_.fetch_add(micros);
}
if (!no_slowdown_) {
target()->SleepForMicroseconds(micros);
}
}
virtual Status GetCurrentTime(int64_t* unix_time) override {
Status s;
if (!time_elapse_only_sleep_) {
s = target()->GetCurrentTime(unix_time);
}
if (s.ok()) {
*unix_time += addon_time_.load();
}
return s;
}
virtual uint64_t NowNanos() override {
return (time_elapse_only_sleep_ ? 0 : target()->NowNanos()) +
addon_time_.load() * 1000;
}
virtual uint64_t NowMicros() override {
return (time_elapse_only_sleep_ ? 0 : target()->NowMicros()) +
addon_time_.load();
}
virtual Status DeleteFile(const std::string& fname) override {
delete_count_.fetch_add(1);
return target()->DeleteFile(fname);
}
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_;
// Number of WAL files that are still open for write.
std::atomic<int> num_open_wal_file_;
bool count_random_reads_;
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_;
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<int64_t> addon_time_;
std::atomic<int> delete_count_;
bool time_elapse_only_sleep_;
bool no_slowdown_;
std::atomic<bool> is_wal_sync_thread_safe_{true};
};
#ifndef ROCKSDB_LITE
class OnFileDeletionListener : public EventListener {
public:
OnFileDeletionListener() : matched_count_(0), expected_file_name_("") {}
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_;
};
#endif
class DBTestBase : public testing::Test {
protected:
// Sequence of option configurations to try
enum OptionConfig {
kDefault = 0,
kBlockBasedTableWithPrefixHashIndex = 1,
kBlockBasedTableWithWholeKeyHashIndex = 2,
kPlainTableFirstBytePrefix = 3,
kPlainTableCappedPrefix = 4,
kPlainTableCappedPrefixNonMmap = 5,
kPlainTableAllBytesPrefix = 6,
kVectorRep = 7,
kHashLinkList = 8,
kHashCuckoo = 9,
kMergePut = 10,
kFilter = 11,
kFullFilterWithNewTableReaderForCompactions = 12,
kUncompressed = 13,
kNumLevel_3 = 14,
kDBLogDir = 15,
kWalDirAndMmapReads = 16,
kManifestFileSize = 17,
kPerfOptions = 18,
kHashSkipList = 19,
kUniversalCompaction = 20,
kUniversalCompactionMultiLevel = 21,
kCompressedBlockCache = 22,
kInfiniteMaxOpenFiles = 23,
kxxHashChecksum = 24,
kFIFOCompaction = 25,
kOptimizeFiltersForHits = 26,
kRowCache = 27,
kRecycleLogFiles = 28,
kConcurrentSkipList = 29,
kEnd = 30,
kLevelSubcompactions = 31,
kUniversalSubcompactions = 32,
kBlockBasedTableWithIndexRestartInterval = 33,
kBlockBasedTableWithPartitionedIndex = 34,
kPartitionedFilterWithNewTableReaderForCompactions = 35,
};
int option_config_;
public:
std::string dbname_;
std::string alternative_wal_dir_;
std::string alternative_db_log_dir_;
MockEnv* mem_env_;
SpecialEnv* env_;
DB* db_;
std::vector<ColumnFamilyHandle*> handles_;
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,
kSkipHashCuckoo = 64,
kSkipFIFOCompaction = 128,
kSkipMmapReads = 256,
};
explicit DBTestBase(const std::string path);
~DBTestBase();
static std::string RandomString(Random* rnd, int len) {
std::string r;
test::RandomString(rnd, len, &r);
return r;
}
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();
// Return the current option configuration.
Options CurrentOptions(
const anon::OptionsOverride& options_override = anon::OptionsOverride());
Options CurrentOptions(
const Options& defaultOptions,
const anon::OptionsOverride& options_override = anon::OptionsOverride());
DBImpl* dbfull() { return reinterpret_cast<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);
Status ReadOnlyReopen(const Options& options);
Status TryReopen(const Options& options);
bool IsDirectIOSupported();
Status Flush(int cf = 0);
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);
std::string Get(const std::string& k, const Snapshot* snapshot = nullptr);
std::string Get(int cf, const std::string& k,
const Snapshot* snapshot = nullptr);
uint64_t GetNumSnapshots();
uint64_t GetTimeOldestSnapshots();
// 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();
#endif // ROCKSDB_LITE
int NumTableFilesAtLevel(int level, int cf = 0);
double CompressionRatioAtLevel(int level, int cf = 0);
int TotalTableFiles(int cf = 0, int levels = -1);
// Return spread of files per level
std::string FilesPerLevel(int cf = 0);
size_t CountFiles();
uint64_t Size(const Slice& start, const Slice& limit, int cf = 0);
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);
void DumpFileCounts(const char* label);
std::string DumpSSTableList();
void GetSstFiles(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);
std::unordered_map<std::string, uint64_t> GetAllSSTFiles(
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);
}
};
} // namespace rocksdb