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/test_util/testutil.h

854 lines
28 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 <algorithm>
#include <deque>
#include <string>
#include <vector>
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
#include "env/composite_env_wrapper.h"
#include "file/writable_file_writer.h"
#include "rocksdb/compaction_filter.h"
#include "rocksdb/env.h"
#include "rocksdb/iterator.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/options.h"
#include "rocksdb/slice.h"
#include "rocksdb/table.h"
#include "table/internal_iterator.h"
LogAndApply() should fail if the column family has been dropped Summary: This patch finally fixes the ColumnFamilyTest.ReadDroppedColumnFamily test. The test has been failing very sporadically and it was hard to repro. However, I managed to write a new tests that reproes the failure deterministically. Here's what happens: 1. We start the flush for the column family 2. We check if the column family was dropped here: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/flush_job.cc#L149 3. This check goes through, ends up in InstallMemtableFlushResults() and it goes into LogAndApply() 4. At about this time, we start dropping the column family. Dropping the column family process gets to LogAndApply() at about the same time as LogAndApply() from flush process 5. Drop column family goes through LogAndApply() first, marking the column family as dropped. 6. Flush process gets woken up and gets a chance to write to the MANIFEST. However, this is where it gets stuck: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/version_set.cc#L1975 7. We see that the column family was dropped, so there is no need to write to the MANIFEST. We return OK. 8. Flush gets OK back from LogAndApply() and it deletes the memtable, thinking that the data is now safely persisted to sst file. The fix is pretty simple. Instead of OK, we return ShutdownInProgress. This is not really true, but we have been using this status code to also mean "this operation was canceled because the column family has been dropped". The fix is only one LOC. All other code is related to tests. I added a new test that reproes the failure. I also moved SleepingBackgroundTask to util/testutil.h (because I needed it in column_family_test for my new test). There's plenty of other places where we reimplement SleepingBackgroundTask, but I'll address that in a separate commit. Test Plan: 1. new test 2. make check 3. Make sure the ColumnFamilyTest.ReadDroppedColumnFamily doesn't fail on Travis: https://travis-ci.org/facebook/rocksdb/jobs/79952386 Reviewers: yhchiang, anthony, IslamAbdelRahman, kradhakrishnan, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D46773
9 years ago
#include "util/mutexlock.h"
#ifdef ROCKSDB_UNITTESTS_WITH_CUSTOM_OBJECTS_FROM_STATIC_LIBS
extern "C" {
void RegisterCustomObjects(int argc, char** argv);
}
#else
void RegisterCustomObjects(int argc, char** argv);
#endif // !ROCKSDB_UNITTESTS_WITH_CUSTOM_OBJECTS_FROM_STATIC_LIBS
namespace ROCKSDB_NAMESPACE {
Create a CustomEnv class; Add WinFileSystem; Make LegacyFileSystemWrapper private (#7703) Summary: This PR does the following: -> Creates a WinFileSystem class. This class is the Windows equivalent of the PosixFileSystem and will be used on Windows systems. -> Introduces a CustomEnv class. A CustomEnv is an Env that takes a FileSystem as constructor argument. I believe there will only ever be two implementations of this class (PosixEnv and WinEnv). There is still a CustomEnvWrapper class that takes an Env and a FileSystem and wraps the Env calls with the input Env but uses the FileSystem for the FileSystem calls -> Eliminates the public uses of the LegacyFileSystemWrapper. With this change in place, there are effectively the following patterns of Env: - "Base Env classes" (PosixEnv, WinEnv). These classes implement the core Env functions (e.g. Threads) and have a hard-coded input FileSystem. These classes inherit from CompositeEnv, implement the core Env functions (threads) and delegate the FileSystem-like calls to the input file system. - Wrapped Composite Env classes (MemEnv). These classes take in an Env and a FileSystem. The core env functions are re-directed to the wrapped env. The file system calls are redirected to the input file system - Legacy Wrapped Env classes. These classes take in an Env input (but no FileSystem). The core env functions are re-directed to the wrapped env. A "Legacy File System" is created using this env and the file system calls directed to the env itself. With these changes in place, the PosixEnv becomes a singleton -- there is only ever one created. Any other use of the PosixEnv is via another wrapped env. This cleans up some of the issues with the env construction and destruction. Additionally, there were places in the code that required had an Env when they required a FileSystem. Many of these places would wrap the Env with a LegacyFileSystemWrapper instead of using the env->GetFileSystem(). These places were changed, thereby removing layers of additional redirection (LegacyFileSystem --> Env --> Env::FileSystem). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7703 Reviewed By: zhichao-cao Differential Revision: D25762190 Pulled By: anand1976 fbshipit-source-id: 1a088e97fc916f28ac69c149cd1dcad0ab31704b
4 years ago
class FileSystem;
class MemTableRepFactory;
class ObjectLibrary;
class Random;
class SequentialFile;
class SequentialFileReader;
namespace test {
extern const uint32_t kDefaultFormatVersion;
extern const std::set<uint32_t> kFooterFormatVersionsToTest;
// Return a random key with the specified length that may contain interesting
// characters (e.g. \x00, \xff, etc.).
enum RandomKeyType : char { RANDOM, LARGEST, SMALLEST, MIDDLE };
extern std::string RandomKey(Random* rnd, int len,
RandomKeyType type = RandomKeyType::RANDOM);
// Store in *dst a string of length "len" that will compress to
// "N*compressed_fraction" bytes and return a Slice that references
// the generated data.
extern Slice CompressibleString(Random* rnd, double compressed_fraction,
int len, std::string* dst);
#ifndef NDEBUG
// An internal comparator that just forward comparing results from the
// user comparator in it. Can be used to test entities that have no dependency
// on internal key structure but consumes InternalKeyComparator, like
// BlockBasedTable.
class PlainInternalKeyComparator : public InternalKeyComparator {
public:
explicit PlainInternalKeyComparator(const Comparator* c)
: InternalKeyComparator(c) {}
virtual ~PlainInternalKeyComparator() {}
virtual int Compare(const Slice& a, const Slice& b) const override {
return user_comparator()->Compare(a, b);
}
};
#endif
// A test comparator which compare two strings in this way:
// (1) first compare prefix of 8 bytes in alphabet order,
// (2) if two strings share the same prefix, sort the other part of the string
// in the reverse alphabet order.
// This helps simulate the case of compounded key of [entity][timestamp] and
// latest timestamp first.
class SimpleSuffixReverseComparator : public Comparator {
public:
SimpleSuffixReverseComparator() {}
static const char* kClassName() { return "SimpleSuffixReverseComparator"; }
virtual const char* Name() const override { return kClassName(); }
virtual int Compare(const Slice& a, const Slice& b) const override {
Slice prefix_a = Slice(a.data(), 8);
Slice prefix_b = Slice(b.data(), 8);
int prefix_comp = prefix_a.compare(prefix_b);
if (prefix_comp != 0) {
return prefix_comp;
} else {
Slice suffix_a = Slice(a.data() + 8, a.size() - 8);
Slice suffix_b = Slice(b.data() + 8, b.size() - 8);
return -(suffix_a.compare(suffix_b));
}
}
virtual void FindShortestSeparator(std::string* /*start*/,
const Slice& /*limit*/) const override {}
virtual void FindShortSuccessor(std::string* /*key*/) const override {}
};
// Returns a user key comparator that can be used for comparing two uint64_t
// slices. Instead of comparing slices byte-wise, it compares all the 8 bytes
// at once. Assumes same endian-ness is used though the database's lifetime.
// Symantics of comparison would differ from Bytewise comparator in little
// endian machines.
extern const Comparator* Uint64Comparator();
// A wrapper api for getting the ComparatorWithU64Ts<BytewiseComparator>
extern const Comparator* BytewiseComparatorWithU64TsWrapper();
class StringSink : public FSWritableFile {
public:
std::string contents_;
explicit StringSink(Slice* reader_contents = nullptr)
: FSWritableFile(),
contents_(""),
reader_contents_(reader_contents),
last_flush_(0) {
if (reader_contents_ != nullptr) {
*reader_contents_ = Slice(contents_.data(), 0);
}
}
const std::string& contents() const { return contents_; }
IOStatus Truncate(uint64_t size, const IOOptions& /*opts*/,
IODebugContext* /*dbg*/) override {
contents_.resize(static_cast<size_t>(size));
return IOStatus::OK();
}
IOStatus Close(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
return IOStatus::OK();
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
}
IOStatus Flush(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
if (reader_contents_ != nullptr) {
assert(reader_contents_->size() <= last_flush_);
size_t offset = last_flush_ - reader_contents_->size();
*reader_contents_ = Slice(
contents_.data() + offset,
contents_.size() - offset);
last_flush_ = contents_.size();
}
return IOStatus::OK();
}
IOStatus Sync(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
return IOStatus::OK();
}
using FSWritableFile::Append;
IOStatus Append(const Slice& slice, const IOOptions& /*opts*/,
IODebugContext* /*dbg*/) override {
contents_.append(slice.data(), slice.size());
return IOStatus::OK();
}
void Drop(size_t bytes) {
if (reader_contents_ != nullptr) {
contents_.resize(contents_.size() - bytes);
*reader_contents_ = Slice(
reader_contents_->data(), reader_contents_->size() - bytes);
last_flush_ = contents_.size();
}
}
private:
Slice* reader_contents_;
size_t last_flush_;
};
// A wrapper around a StringSink to give it a RandomRWFile interface
class RandomRWStringSink : public FSRandomRWFile {
public:
explicit RandomRWStringSink(StringSink* ss) : ss_(ss) {}
IOStatus Write(uint64_t offset, const Slice& data, const IOOptions& /*opts*/,
IODebugContext* /*dbg*/) override {
if (offset + data.size() > ss_->contents_.size()) {
ss_->contents_.resize(static_cast<size_t>(offset) + data.size(), '\0');
}
char* pos = const_cast<char*>(ss_->contents_.data() + offset);
memcpy(pos, data.data(), data.size());
return IOStatus::OK();
}
IOStatus Read(uint64_t offset, size_t n, const IOOptions& /*opts*/,
Slice* result, char* /*scratch*/,
IODebugContext* /*dbg*/) const override {
*result = Slice(nullptr, 0);
if (offset < ss_->contents_.size()) {
size_t str_res_sz =
std::min(static_cast<size_t>(ss_->contents_.size() - offset), n);
*result = Slice(ss_->contents_.data() + offset, str_res_sz);
}
return IOStatus::OK();
}
IOStatus Flush(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
return IOStatus::OK();
}
IOStatus Sync(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
return IOStatus::OK();
}
IOStatus Close(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
return IOStatus::OK();
}
const std::string& contents() const { return ss_->contents(); }
private:
StringSink* ss_;
};
// Like StringSink, this writes into a string. Unlink StringSink, it
// has some initial content and overwrites it, just like a recycled
// log file.
class OverwritingStringSink : public FSWritableFile {
public:
explicit OverwritingStringSink(Slice* reader_contents)
: FSWritableFile(),
contents_(""),
reader_contents_(reader_contents),
last_flush_(0) {}
const std::string& contents() const { return contents_; }
IOStatus Truncate(uint64_t size, const IOOptions& /*opts*/,
IODebugContext* /*dbg*/) override {
contents_.resize(static_cast<size_t>(size));
return IOStatus::OK();
}
IOStatus Close(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
return IOStatus::OK();
}
IOStatus Flush(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
if (last_flush_ < contents_.size()) {
assert(reader_contents_->size() >= contents_.size());
memcpy((char*)reader_contents_->data() + last_flush_,
contents_.data() + last_flush_, contents_.size() - last_flush_);
last_flush_ = contents_.size();
}
return IOStatus::OK();
}
IOStatus Sync(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
return IOStatus::OK();
}
using FSWritableFile::Append;
IOStatus Append(const Slice& slice, const IOOptions& /*opts*/,
IODebugContext* /*dbg*/) override {
contents_.append(slice.data(), slice.size());
return IOStatus::OK();
}
void Drop(size_t bytes) {
contents_.resize(contents_.size() - bytes);
if (last_flush_ > contents_.size()) last_flush_ = contents_.size();
}
private:
std::string contents_;
Slice* reader_contents_;
size_t last_flush_;
};
class StringSource : public FSRandomAccessFile {
public:
explicit StringSource(const Slice& contents, uint64_t uniq_id = 0,
bool mmap = false)
: contents_(contents.data(), contents.size()),
uniq_id_(uniq_id),
mmap_(mmap),
total_reads_(0) {}
virtual ~StringSource() { }
uint64_t Size() const { return contents_.size(); }
IOStatus Prefetch(uint64_t /*offset*/, size_t /*n*/,
const IOOptions& /*options*/,
IODebugContext* /*dbg*/) override {
// If we are using mmap_, it is equivalent to performing a prefetch
if (mmap_) {
return IOStatus::OK();
} else {
return IOStatus::NotSupported("Prefetch not supported");
}
}
IOStatus Read(uint64_t offset, size_t n, const IOOptions& /*opts*/,
Slice* result, char* scratch,
IODebugContext* /*dbg*/) const override {
total_reads_++;
if (offset > contents_.size()) {
return IOStatus::InvalidArgument("invalid Read offset");
}
if (offset + n > contents_.size()) {
n = contents_.size() - static_cast<size_t>(offset);
}
if (!mmap_) {
memcpy(scratch, &contents_[static_cast<size_t>(offset)], n);
*result = Slice(scratch, n);
} else {
*result = Slice(&contents_[static_cast<size_t>(offset)], n);
}
return IOStatus::OK();
}
size_t GetUniqueId(char* id, size_t max_size) const override {
if (max_size < 20) {
return 0;
}
char* rid = id;
rid = EncodeVarint64(rid, uniq_id_);
rid = EncodeVarint64(rid, 0);
return static_cast<size_t>(rid-id);
}
int total_reads() const { return total_reads_; }
void set_total_reads(int tr) { total_reads_ = tr; }
private:
std::string contents_;
uint64_t uniq_id_;
bool mmap_;
mutable int total_reads_;
};
class NullLogger : public Logger {
public:
using Logger::Logv;
virtual void Logv(const char* /*format*/, va_list /*ap*/) override {}
virtual size_t GetLogFileSize() const override { return 0; }
};
// Corrupts key by changing the type
extern void CorruptKeyType(InternalKey* ikey);
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
extern std::string KeyStr(const std::string& user_key,
const SequenceNumber& seq, const ValueType& t,
bool corrupt = false);
Allow compaction iterator to perform garbage collection (#7556) Summary: Add a threshold timestamp, full_history_ts_low_ of type `std::string*` to `CompactionIterator`, so that RocksDB can also perform garbage collection during compaction. * If full_history_ts_low_ is nullptr, then compaction iterator does not perform GC, preserving all timestamp history for all keys. Compaction iterator will treat user key with different timestamps as different user keys. * If full_history_ts_low_ is not nullptr, then compaction iterator performs GC. GC will look at keys older than `*full_history_ts_low_` and determine their eligibility based on factors including snapshots. Current rules of GC: * If an internal key is in the same snapshot as a previous counterpart with the same user key, and this key is eligible for GC, and the key is not single-delete or merge operand, then this key can be dropped. Note that the previous internal key cannot be a merge operand either. * If a tombstone is the most recent one in the earliest snapshot and it is eligible for GC, and keyNotExistsBeyondLevel() is true, then this tombstone can be dropped. * If a tombstone is the most recent one in a snapshot and it is eligible for GC, and the compaction is at bottommost level, then all other older internal keys of the same user key must also be eligible for GC, thus can be dropped * Single-delete, delete-range and merge are not currently supported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7556 Test Plan: make check Reviewed By: ltamasi Differential Revision: D24507728 Pulled By: riversand963 fbshipit-source-id: 3c09c7301f41eed76dfcf4d1527e68cf6e0a8bb3
4 years ago
extern std::string KeyStr(uint64_t ts, const std::string& user_key,
const SequenceNumber& seq, const ValueType& t,
bool corrupt = false);
LogAndApply() should fail if the column family has been dropped Summary: This patch finally fixes the ColumnFamilyTest.ReadDroppedColumnFamily test. The test has been failing very sporadically and it was hard to repro. However, I managed to write a new tests that reproes the failure deterministically. Here's what happens: 1. We start the flush for the column family 2. We check if the column family was dropped here: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/flush_job.cc#L149 3. This check goes through, ends up in InstallMemtableFlushResults() and it goes into LogAndApply() 4. At about this time, we start dropping the column family. Dropping the column family process gets to LogAndApply() at about the same time as LogAndApply() from flush process 5. Drop column family goes through LogAndApply() first, marking the column family as dropped. 6. Flush process gets woken up and gets a chance to write to the MANIFEST. However, this is where it gets stuck: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/version_set.cc#L1975 7. We see that the column family was dropped, so there is no need to write to the MANIFEST. We return OK. 8. Flush gets OK back from LogAndApply() and it deletes the memtable, thinking that the data is now safely persisted to sst file. The fix is pretty simple. Instead of OK, we return ShutdownInProgress. This is not really true, but we have been using this status code to also mean "this operation was canceled because the column family has been dropped". The fix is only one LOC. All other code is related to tests. I added a new test that reproes the failure. I also moved SleepingBackgroundTask to util/testutil.h (because I needed it in column_family_test for my new test). There's plenty of other places where we reimplement SleepingBackgroundTask, but I'll address that in a separate commit. Test Plan: 1. new test 2. make check 3. Make sure the ColumnFamilyTest.ReadDroppedColumnFamily doesn't fail on Travis: https://travis-ci.org/facebook/rocksdb/jobs/79952386 Reviewers: yhchiang, anthony, IslamAbdelRahman, kradhakrishnan, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D46773
9 years ago
class SleepingBackgroundTask {
public:
SleepingBackgroundTask()
: bg_cv_(&mutex_),
should_sleep_(true),
done_with_sleep_(false),
sleeping_(false) {}
bool IsSleeping() {
MutexLock l(&mutex_);
return sleeping_;
}
LogAndApply() should fail if the column family has been dropped Summary: This patch finally fixes the ColumnFamilyTest.ReadDroppedColumnFamily test. The test has been failing very sporadically and it was hard to repro. However, I managed to write a new tests that reproes the failure deterministically. Here's what happens: 1. We start the flush for the column family 2. We check if the column family was dropped here: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/flush_job.cc#L149 3. This check goes through, ends up in InstallMemtableFlushResults() and it goes into LogAndApply() 4. At about this time, we start dropping the column family. Dropping the column family process gets to LogAndApply() at about the same time as LogAndApply() from flush process 5. Drop column family goes through LogAndApply() first, marking the column family as dropped. 6. Flush process gets woken up and gets a chance to write to the MANIFEST. However, this is where it gets stuck: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/version_set.cc#L1975 7. We see that the column family was dropped, so there is no need to write to the MANIFEST. We return OK. 8. Flush gets OK back from LogAndApply() and it deletes the memtable, thinking that the data is now safely persisted to sst file. The fix is pretty simple. Instead of OK, we return ShutdownInProgress. This is not really true, but we have been using this status code to also mean "this operation was canceled because the column family has been dropped". The fix is only one LOC. All other code is related to tests. I added a new test that reproes the failure. I also moved SleepingBackgroundTask to util/testutil.h (because I needed it in column_family_test for my new test). There's plenty of other places where we reimplement SleepingBackgroundTask, but I'll address that in a separate commit. Test Plan: 1. new test 2. make check 3. Make sure the ColumnFamilyTest.ReadDroppedColumnFamily doesn't fail on Travis: https://travis-ci.org/facebook/rocksdb/jobs/79952386 Reviewers: yhchiang, anthony, IslamAbdelRahman, kradhakrishnan, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D46773
9 years ago
void DoSleep() {
MutexLock l(&mutex_);
sleeping_ = true;
bg_cv_.SignalAll();
LogAndApply() should fail if the column family has been dropped Summary: This patch finally fixes the ColumnFamilyTest.ReadDroppedColumnFamily test. The test has been failing very sporadically and it was hard to repro. However, I managed to write a new tests that reproes the failure deterministically. Here's what happens: 1. We start the flush for the column family 2. We check if the column family was dropped here: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/flush_job.cc#L149 3. This check goes through, ends up in InstallMemtableFlushResults() and it goes into LogAndApply() 4. At about this time, we start dropping the column family. Dropping the column family process gets to LogAndApply() at about the same time as LogAndApply() from flush process 5. Drop column family goes through LogAndApply() first, marking the column family as dropped. 6. Flush process gets woken up and gets a chance to write to the MANIFEST. However, this is where it gets stuck: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/version_set.cc#L1975 7. We see that the column family was dropped, so there is no need to write to the MANIFEST. We return OK. 8. Flush gets OK back from LogAndApply() and it deletes the memtable, thinking that the data is now safely persisted to sst file. The fix is pretty simple. Instead of OK, we return ShutdownInProgress. This is not really true, but we have been using this status code to also mean "this operation was canceled because the column family has been dropped". The fix is only one LOC. All other code is related to tests. I added a new test that reproes the failure. I also moved SleepingBackgroundTask to util/testutil.h (because I needed it in column_family_test for my new test). There's plenty of other places where we reimplement SleepingBackgroundTask, but I'll address that in a separate commit. Test Plan: 1. new test 2. make check 3. Make sure the ColumnFamilyTest.ReadDroppedColumnFamily doesn't fail on Travis: https://travis-ci.org/facebook/rocksdb/jobs/79952386 Reviewers: yhchiang, anthony, IslamAbdelRahman, kradhakrishnan, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D46773
9 years ago
while (should_sleep_) {
bg_cv_.Wait();
}
sleeping_ = false;
LogAndApply() should fail if the column family has been dropped Summary: This patch finally fixes the ColumnFamilyTest.ReadDroppedColumnFamily test. The test has been failing very sporadically and it was hard to repro. However, I managed to write a new tests that reproes the failure deterministically. Here's what happens: 1. We start the flush for the column family 2. We check if the column family was dropped here: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/flush_job.cc#L149 3. This check goes through, ends up in InstallMemtableFlushResults() and it goes into LogAndApply() 4. At about this time, we start dropping the column family. Dropping the column family process gets to LogAndApply() at about the same time as LogAndApply() from flush process 5. Drop column family goes through LogAndApply() first, marking the column family as dropped. 6. Flush process gets woken up and gets a chance to write to the MANIFEST. However, this is where it gets stuck: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/version_set.cc#L1975 7. We see that the column family was dropped, so there is no need to write to the MANIFEST. We return OK. 8. Flush gets OK back from LogAndApply() and it deletes the memtable, thinking that the data is now safely persisted to sst file. The fix is pretty simple. Instead of OK, we return ShutdownInProgress. This is not really true, but we have been using this status code to also mean "this operation was canceled because the column family has been dropped". The fix is only one LOC. All other code is related to tests. I added a new test that reproes the failure. I also moved SleepingBackgroundTask to util/testutil.h (because I needed it in column_family_test for my new test). There's plenty of other places where we reimplement SleepingBackgroundTask, but I'll address that in a separate commit. Test Plan: 1. new test 2. make check 3. Make sure the ColumnFamilyTest.ReadDroppedColumnFamily doesn't fail on Travis: https://travis-ci.org/facebook/rocksdb/jobs/79952386 Reviewers: yhchiang, anthony, IslamAbdelRahman, kradhakrishnan, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D46773
9 years ago
done_with_sleep_ = true;
bg_cv_.SignalAll();
}
void WaitUntilSleeping() {
MutexLock l(&mutex_);
while (!sleeping_ || !should_sleep_) {
bg_cv_.Wait();
}
}
// Waits for the status to change to sleeping,
// otherwise times out.
// wait_time is in microseconds.
// Returns true when times out, false otherwise.
bool TimedWaitUntilSleeping(uint64_t wait_time);
LogAndApply() should fail if the column family has been dropped Summary: This patch finally fixes the ColumnFamilyTest.ReadDroppedColumnFamily test. The test has been failing very sporadically and it was hard to repro. However, I managed to write a new tests that reproes the failure deterministically. Here's what happens: 1. We start the flush for the column family 2. We check if the column family was dropped here: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/flush_job.cc#L149 3. This check goes through, ends up in InstallMemtableFlushResults() and it goes into LogAndApply() 4. At about this time, we start dropping the column family. Dropping the column family process gets to LogAndApply() at about the same time as LogAndApply() from flush process 5. Drop column family goes through LogAndApply() first, marking the column family as dropped. 6. Flush process gets woken up and gets a chance to write to the MANIFEST. However, this is where it gets stuck: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/version_set.cc#L1975 7. We see that the column family was dropped, so there is no need to write to the MANIFEST. We return OK. 8. Flush gets OK back from LogAndApply() and it deletes the memtable, thinking that the data is now safely persisted to sst file. The fix is pretty simple. Instead of OK, we return ShutdownInProgress. This is not really true, but we have been using this status code to also mean "this operation was canceled because the column family has been dropped". The fix is only one LOC. All other code is related to tests. I added a new test that reproes the failure. I also moved SleepingBackgroundTask to util/testutil.h (because I needed it in column_family_test for my new test). There's plenty of other places where we reimplement SleepingBackgroundTask, but I'll address that in a separate commit. Test Plan: 1. new test 2. make check 3. Make sure the ColumnFamilyTest.ReadDroppedColumnFamily doesn't fail on Travis: https://travis-ci.org/facebook/rocksdb/jobs/79952386 Reviewers: yhchiang, anthony, IslamAbdelRahman, kradhakrishnan, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D46773
9 years ago
void WakeUp() {
MutexLock l(&mutex_);
should_sleep_ = false;
bg_cv_.SignalAll();
}
void WaitUntilDone() {
MutexLock l(&mutex_);
while (!done_with_sleep_) {
bg_cv_.Wait();
}
}
// Similar to TimedWaitUntilSleeping.
// Waits until the task is done.
bool TimedWaitUntilDone(uint64_t wait_time);
LogAndApply() should fail if the column family has been dropped Summary: This patch finally fixes the ColumnFamilyTest.ReadDroppedColumnFamily test. The test has been failing very sporadically and it was hard to repro. However, I managed to write a new tests that reproes the failure deterministically. Here's what happens: 1. We start the flush for the column family 2. We check if the column family was dropped here: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/flush_job.cc#L149 3. This check goes through, ends up in InstallMemtableFlushResults() and it goes into LogAndApply() 4. At about this time, we start dropping the column family. Dropping the column family process gets to LogAndApply() at about the same time as LogAndApply() from flush process 5. Drop column family goes through LogAndApply() first, marking the column family as dropped. 6. Flush process gets woken up and gets a chance to write to the MANIFEST. However, this is where it gets stuck: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/version_set.cc#L1975 7. We see that the column family was dropped, so there is no need to write to the MANIFEST. We return OK. 8. Flush gets OK back from LogAndApply() and it deletes the memtable, thinking that the data is now safely persisted to sst file. The fix is pretty simple. Instead of OK, we return ShutdownInProgress. This is not really true, but we have been using this status code to also mean "this operation was canceled because the column family has been dropped". The fix is only one LOC. All other code is related to tests. I added a new test that reproes the failure. I also moved SleepingBackgroundTask to util/testutil.h (because I needed it in column_family_test for my new test). There's plenty of other places where we reimplement SleepingBackgroundTask, but I'll address that in a separate commit. Test Plan: 1. new test 2. make check 3. Make sure the ColumnFamilyTest.ReadDroppedColumnFamily doesn't fail on Travis: https://travis-ci.org/facebook/rocksdb/jobs/79952386 Reviewers: yhchiang, anthony, IslamAbdelRahman, kradhakrishnan, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D46773
9 years ago
bool WokenUp() {
MutexLock l(&mutex_);
return should_sleep_ == false;
}
void Reset() {
MutexLock l(&mutex_);
should_sleep_ = true;
done_with_sleep_ = false;
}
static void DoSleepTask(void* arg) {
reinterpret_cast<SleepingBackgroundTask*>(arg)->DoSleep();
}
private:
port::Mutex mutex_;
port::CondVar bg_cv_; // Signalled when background work finishes
bool should_sleep_;
bool done_with_sleep_;
bool sleeping_;
LogAndApply() should fail if the column family has been dropped Summary: This patch finally fixes the ColumnFamilyTest.ReadDroppedColumnFamily test. The test has been failing very sporadically and it was hard to repro. However, I managed to write a new tests that reproes the failure deterministically. Here's what happens: 1. We start the flush for the column family 2. We check if the column family was dropped here: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/flush_job.cc#L149 3. This check goes through, ends up in InstallMemtableFlushResults() and it goes into LogAndApply() 4. At about this time, we start dropping the column family. Dropping the column family process gets to LogAndApply() at about the same time as LogAndApply() from flush process 5. Drop column family goes through LogAndApply() first, marking the column family as dropped. 6. Flush process gets woken up and gets a chance to write to the MANIFEST. However, this is where it gets stuck: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/version_set.cc#L1975 7. We see that the column family was dropped, so there is no need to write to the MANIFEST. We return OK. 8. Flush gets OK back from LogAndApply() and it deletes the memtable, thinking that the data is now safely persisted to sst file. The fix is pretty simple. Instead of OK, we return ShutdownInProgress. This is not really true, but we have been using this status code to also mean "this operation was canceled because the column family has been dropped". The fix is only one LOC. All other code is related to tests. I added a new test that reproes the failure. I also moved SleepingBackgroundTask to util/testutil.h (because I needed it in column_family_test for my new test). There's plenty of other places where we reimplement SleepingBackgroundTask, but I'll address that in a separate commit. Test Plan: 1. new test 2. make check 3. Make sure the ColumnFamilyTest.ReadDroppedColumnFamily doesn't fail on Travis: https://travis-ci.org/facebook/rocksdb/jobs/79952386 Reviewers: yhchiang, anthony, IslamAbdelRahman, kradhakrishnan, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D46773
9 years ago
};
// Filters merge operands and values that are equal to `num`.
class FilterNumber : public CompactionFilter {
public:
explicit FilterNumber(uint64_t num) : num_(num) {}
std::string last_merge_operand_key() { return last_merge_operand_key_; }
bool Filter(int /*level*/, const ROCKSDB_NAMESPACE::Slice& /*key*/,
const ROCKSDB_NAMESPACE::Slice& value, std::string* /*new_value*/,
bool* /*value_changed*/) const override {
if (value.size() == sizeof(uint64_t)) {
return num_ == DecodeFixed64(value.data());
}
return true;
}
bool FilterMergeOperand(
int /*level*/, const ROCKSDB_NAMESPACE::Slice& key,
const ROCKSDB_NAMESPACE::Slice& value) const override {
last_merge_operand_key_ = key.ToString();
if (value.size() == sizeof(uint64_t)) {
return num_ == DecodeFixed64(value.data());
}
return true;
}
const char* Name() const override { return "FilterBadMergeOperand"; }
private:
mutable std::string last_merge_operand_key_;
uint64_t num_;
};
inline std::string EncodeInt(uint64_t x) {
std::string result;
PutFixed64(&result, x);
return result;
}
class SeqStringSource : public FSSequentialFile {
public:
SeqStringSource(const std::string& data, std::atomic<int>* read_count)
: data_(data), offset_(0), read_count_(read_count) {}
~SeqStringSource() override {}
IOStatus Read(size_t n, const IOOptions& /*opts*/, Slice* result,
char* scratch, IODebugContext* /*dbg*/) override {
std::string output;
if (offset_ < data_.size()) {
n = std::min(data_.size() - offset_, n);
memcpy(scratch, data_.data() + offset_, n);
offset_ += n;
*result = Slice(scratch, n);
} else {
return IOStatus::InvalidArgument(
"Attempt to read when it already reached eof.");
}
(*read_count_)++;
return IOStatus::OK();
}
IOStatus Skip(uint64_t n) override {
if (offset_ >= data_.size()) {
return IOStatus::InvalidArgument(
"Attempt to read when it already reached eof.");
}
// TODO(yhchiang): Currently doesn't handle the overflow case.
offset_ += static_cast<size_t>(n);
return IOStatus::OK();
}
private:
std::string data_;
size_t offset_;
std::atomic<int>* read_count_;
};
class StringFS : public FileSystemWrapper {
public:
class StringSink : public FSWritableFile {
public:
explicit StringSink(std::string* contents)
: FSWritableFile(), contents_(contents) {}
IOStatus Truncate(uint64_t size, const IOOptions& /*opts*/,
IODebugContext* /*dbg*/) override {
contents_->resize(static_cast<size_t>(size));
return IOStatus::OK();
}
IOStatus Close(const IOOptions& /*opts*/,
IODebugContext* /*dbg*/) override {
return IOStatus::OK();
}
IOStatus Flush(const IOOptions& /*opts*/,
IODebugContext* /*dbg*/) override {
return IOStatus::OK();
}
IOStatus Sync(const IOOptions& /*opts*/, IODebugContext* /*dbg*/) override {
return IOStatus::OK();
}
using FSWritableFile::Append;
IOStatus Append(const Slice& slice, const IOOptions& /*opts*/,
IODebugContext* /*dbg*/) override {
contents_->append(slice.data(), slice.size());
return IOStatus::OK();
}
private:
std::string* contents_;
};
explicit StringFS(const std::shared_ptr<FileSystem>& t)
: FileSystemWrapper(t) {}
~StringFS() override {}
static const char* kClassName() { return "StringFS"; }
const char* Name() const override { return kClassName(); }
const std::string& GetContent(const std::string& f) { return files_[f]; }
const IOStatus WriteToNewFile(const std::string& file_name,
const std::string& content) {
std::unique_ptr<FSWritableFile> r;
FileOptions file_opts;
IOOptions io_opts;
auto s = NewWritableFile(file_name, file_opts, &r, nullptr);
if (s.ok()) {
s = r->Append(content, io_opts, nullptr);
}
if (s.ok()) {
s = r->Flush(io_opts, nullptr);
}
if (s.ok()) {
s = r->Close(io_opts, nullptr);
}
assert(!s.ok() || files_[file_name] == content);
return s;
}
// The following text is boilerplate that forwards all methods to target()
IOStatus NewSequentialFile(const std::string& f,
const FileOptions& /*options*/,
std::unique_ptr<FSSequentialFile>* r,
IODebugContext* /*dbg*/) override {
auto iter = files_.find(f);
if (iter == files_.end()) {
return IOStatus::NotFound("The specified file does not exist", f);
}
r->reset(new SeqStringSource(iter->second, &num_seq_file_read_));
return IOStatus::OK();
}
IOStatus NewRandomAccessFile(const std::string& /*f*/,
const FileOptions& /*options*/,
std::unique_ptr<FSRandomAccessFile>* /*r*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus NewWritableFile(const std::string& f, const FileOptions& /*options*/,
std::unique_ptr<FSWritableFile>* r,
IODebugContext* /*dbg*/) override {
auto iter = files_.find(f);
if (iter != files_.end()) {
return IOStatus::IOError("The specified file already exists", f);
}
r->reset(new StringSink(&files_[f]));
return IOStatus::OK();
}
IOStatus NewDirectory(const std::string& /*name*/,
const IOOptions& /*options*/,
std::unique_ptr<FSDirectory>* /*result*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus FileExists(const std::string& f, const IOOptions& /*options*/,
IODebugContext* /*dbg*/) override {
if (files_.find(f) == files_.end()) {
return IOStatus::NotFound();
}
return IOStatus::OK();
}
IOStatus GetChildren(const std::string& /*dir*/, const IOOptions& /*options*/,
std::vector<std::string>* /*r*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus DeleteFile(const std::string& f, const IOOptions& /*options*/,
IODebugContext* /*dbg*/) override {
files_.erase(f);
return IOStatus::OK();
}
IOStatus CreateDir(const std::string& /*d*/, const IOOptions& /*options*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus CreateDirIfMissing(const std::string& /*d*/,
const IOOptions& /*options*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus DeleteDir(const std::string& /*d*/, const IOOptions& /*options*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus GetFileSize(const std::string& f, const IOOptions& /*options*/,
uint64_t* s, IODebugContext* /*dbg*/) override {
auto iter = files_.find(f);
if (iter == files_.end()) {
return IOStatus::NotFound("The specified file does not exist:", f);
}
*s = iter->second.size();
return IOStatus::OK();
}
IOStatus GetFileModificationTime(const std::string& /*fname*/,
const IOOptions& /*options*/,
uint64_t* /*file_mtime*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus RenameFile(const std::string& /*s*/, const std::string& /*t*/,
const IOOptions& /*options*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus LinkFile(const std::string& /*s*/, const std::string& /*t*/,
const IOOptions& /*options*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus LockFile(const std::string& /*f*/, const IOOptions& /*options*/,
FileLock** /*l*/, IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
IOStatus UnlockFile(FileLock* /*l*/, const IOOptions& /*options*/,
IODebugContext* /*dbg*/) override {
return IOStatus::NotSupported();
}
std::atomic<int> num_seq_file_read_;
protected:
std::unordered_map<std::string, std::string> files_;
};
// Randomly initialize the given DBOptions
void RandomInitDBOptions(DBOptions* db_opt, Random* rnd);
// Randomly initialize the given ColumnFamilyOptions
// Note that the caller is responsible for releasing non-null
// cf_opt->compaction_filter.
void RandomInitCFOptions(ColumnFamilyOptions* cf_opt, DBOptions&, Random* rnd);
// A dummy merge operator which can change its name
class ChanglingMergeOperator : public MergeOperator {
public:
explicit ChanglingMergeOperator(const std::string& name)
: name_(name + "MergeOperator") {}
~ChanglingMergeOperator() {}
void SetName(const std::string& name) { name_ = name; }
virtual bool FullMergeV2(const MergeOperationInput& /*merge_in*/,
MergeOperationOutput* /*merge_out*/) const override {
return false;
}
virtual bool PartialMergeMulti(const Slice& /*key*/,
const std::deque<Slice>& /*operand_list*/,
std::string* /*new_value*/,
Logger* /*logger*/) const override {
return false;
}
static const char* kClassName() { return "ChanglingMergeOperator"; }
const char* NickName() const override { return kNickName(); }
static const char* kNickName() { return "Changling"; }
bool IsInstanceOf(const std::string& id) const override {
if (id == kClassName()) {
return true;
} else {
return MergeOperator::IsInstanceOf(id);
}
}
virtual const char* Name() const override { return name_.c_str(); }
protected:
std::string name_;
};
// Returns a dummy merge operator with random name.
MergeOperator* RandomMergeOperator(Random* rnd);
// A dummy compaction filter which can change its name
class ChanglingCompactionFilter : public CompactionFilter {
public:
explicit ChanglingCompactionFilter(const std::string& name)
: name_(name + "CompactionFilter") {}
~ChanglingCompactionFilter() {}
void SetName(const std::string& name) { name_ = name; }
bool Filter(int /*level*/, const Slice& /*key*/,
const Slice& /*existing_value*/, std::string* /*new_value*/,
bool* /*value_changed*/) const override {
return false;
}
static const char* kClassName() { return "ChanglingCompactionFilter"; }
const char* NickName() const override { return kNickName(); }
static const char* kNickName() { return "Changling"; }
bool IsInstanceOf(const std::string& id) const override {
if (id == kClassName()) {
return true;
} else {
return CompactionFilter::IsInstanceOf(id);
}
}
const char* Name() const override { return name_.c_str(); }
private:
std::string name_;
};
// Returns a dummy compaction filter with a random name.
CompactionFilter* RandomCompactionFilter(Random* rnd);
// A dummy compaction filter factory which can change its name
class ChanglingCompactionFilterFactory : public CompactionFilterFactory {
public:
explicit ChanglingCompactionFilterFactory(const std::string& name)
: name_(name + "CompactionFilterFactory") {}
~ChanglingCompactionFilterFactory() {}
void SetName(const std::string& name) { name_ = name; }
std::unique_ptr<CompactionFilter> CreateCompactionFilter(
const CompactionFilter::Context& /*context*/) override {
return std::unique_ptr<CompactionFilter>();
}
// Returns a name that identifies this compaction filter factory.
const char* Name() const override { return name_.c_str(); }
static const char* kClassName() { return "ChanglingCompactionFilterFactory"; }
const char* NickName() const override { return kNickName(); }
static const char* kNickName() { return "Changling"; }
bool IsInstanceOf(const std::string& id) const override {
if (id == kClassName()) {
return true;
} else {
return CompactionFilterFactory::IsInstanceOf(id);
}
}
protected:
std::string name_;
};
// The factory for the hacky skip list mem table that triggers flush after
// number of entries exceeds a threshold.
extern MemTableRepFactory* NewSpecialSkipListFactory(int num_entries_per_flush);
CompressionType RandomCompressionType(Random* rnd);
void RandomCompressionTypeVector(const size_t count,
std::vector<CompressionType>* types,
Random* rnd);
CompactionFilterFactory* RandomCompactionFilterFactory(Random* rnd);
const SliceTransform* RandomSliceTransform(Random* rnd, int pre_defined = -1);
TableFactory* RandomTableFactory(Random* rnd, int pre_defined = -1);
std::string RandomName(Random* rnd, const size_t len);
bool IsDirectIOSupported(Env* env, const std::string& dir);
Create a CustomEnv class; Add WinFileSystem; Make LegacyFileSystemWrapper private (#7703) Summary: This PR does the following: -> Creates a WinFileSystem class. This class is the Windows equivalent of the PosixFileSystem and will be used on Windows systems. -> Introduces a CustomEnv class. A CustomEnv is an Env that takes a FileSystem as constructor argument. I believe there will only ever be two implementations of this class (PosixEnv and WinEnv). There is still a CustomEnvWrapper class that takes an Env and a FileSystem and wraps the Env calls with the input Env but uses the FileSystem for the FileSystem calls -> Eliminates the public uses of the LegacyFileSystemWrapper. With this change in place, there are effectively the following patterns of Env: - "Base Env classes" (PosixEnv, WinEnv). These classes implement the core Env functions (e.g. Threads) and have a hard-coded input FileSystem. These classes inherit from CompositeEnv, implement the core Env functions (threads) and delegate the FileSystem-like calls to the input file system. - Wrapped Composite Env classes (MemEnv). These classes take in an Env and a FileSystem. The core env functions are re-directed to the wrapped env. The file system calls are redirected to the input file system - Legacy Wrapped Env classes. These classes take in an Env input (but no FileSystem). The core env functions are re-directed to the wrapped env. A "Legacy File System" is created using this env and the file system calls directed to the env itself. With these changes in place, the PosixEnv becomes a singleton -- there is only ever one created. Any other use of the PosixEnv is via another wrapped env. This cleans up some of the issues with the env construction and destruction. Additionally, there were places in the code that required had an Env when they required a FileSystem. Many of these places would wrap the Env with a LegacyFileSystemWrapper instead of using the env->GetFileSystem(). These places were changed, thereby removing layers of additional redirection (LegacyFileSystem --> Env --> Env::FileSystem). Pull Request resolved: https://github.com/facebook/rocksdb/pull/7703 Reviewed By: zhichao-cao Differential Revision: D25762190 Pulled By: anand1976 fbshipit-source-id: 1a088e97fc916f28ac69c149cd1dcad0ab31704b
4 years ago
bool IsPrefetchSupported(const std::shared_ptr<FileSystem>& fs,
const std::string& dir);
// Return the number of lines where a given pattern was found in a file.
size_t GetLinesCount(const std::string& fname, const std::string& pattern);
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
Status CorruptFile(Env* env, const std::string& fname, int offset,
int bytes_to_corrupt, bool verify_checksum = true);
Status TruncateFile(Env* env, const std::string& fname, uint64_t length);
// Try and delete a directory if it exists
Status TryDeleteDir(Env* env, const std::string& dirname);
// Delete a directory if it exists
void DeleteDir(Env* env, const std::string& dirname);
// Creates an Env from the system environment by looking at the system
// environment variables.
Status CreateEnvFromSystem(const ConfigOptions& options, Env** result,
std::shared_ptr<Env>* guard);
#ifndef ROCKSDB_LITE
// Registers the testutil classes with the ObjectLibrary
int RegisterTestObjects(ObjectLibrary& library, const std::string& /*arg*/);
#endif // ROCKSDB_LITE
// Register the testutil classes with the default ObjectRegistry/Library
void RegisterTestLibrary(const std::string& arg = "");
} // namespace test
} // namespace ROCKSDB_NAMESPACE