rocksdb/db/db_impl/db_impl_secondary.cc

903 lines
33 KiB
C++
Raw Normal View History

Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
// 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).
#include "db/db_impl/db_impl_secondary.h"
#include <cinttypes>
#include "db/arena_wrapped_db_iter.h"
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
#include "db/merge_context.h"
#include "logging/auto_roll_logger.h"
#include "logging/logging.h"
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
#include "monitoring/perf_context_imp.h"
#include "rocksdb/configurable.h"
#include "util/cast_util.h"
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
namespace ROCKSDB_NAMESPACE {
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
#ifndef ROCKSDB_LITE
DBImplSecondary::DBImplSecondary(const DBOptions& db_options,
const std::string& dbname,
std::string secondary_path)
: DBImpl(db_options, dbname, false, true, true),
secondary_path_(std::move(secondary_path)) {
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"Opening the db in secondary mode");
LogFlush(immutable_db_options_.info_log);
}
DBImplSecondary::~DBImplSecondary() {}
Status DBImplSecondary::Recover(
const std::vector<ColumnFamilyDescriptor>& column_families,
bool /*readonly*/, bool /*error_if_wal_file_exists*/,
bool /*error_if_data_exists_in_wals*/, uint64_t*) {
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
mutex_.AssertHeld();
JobContext job_context(0);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
Status s;
s = static_cast<ReactiveVersionSet*>(versions_.get())
->Recover(column_families, &manifest_reader_, &manifest_reporter_,
&manifest_reader_status_);
if (!s.ok()) {
if (manifest_reader_status_) {
manifest_reader_status_->PermitUncheckedError();
}
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
return s;
}
if (immutable_db_options_.paranoid_checks && s.ok()) {
s = CheckConsistency();
}
// Initial max_total_in_memory_state_ before recovery logs.
max_total_in_memory_state_ = 0;
for (auto cfd : *versions_->GetColumnFamilySet()) {
auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
max_total_in_memory_state_ += mutable_cf_options->write_buffer_size *
mutable_cf_options->max_write_buffer_number;
}
if (s.ok()) {
default_cf_handle_ = new ColumnFamilyHandleImpl(
versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
single_column_family_mode_ =
versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
std::unordered_set<ColumnFamilyData*> cfds_changed;
s = FindAndRecoverLogFiles(&cfds_changed, &job_context);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
}
if (s.IsPathNotFound()) {
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"Secondary tries to read WAL, but WAL file(s) have already "
"been purged by primary.");
s = Status::OK();
}
// TODO: update options_file_number_ needed?
job_context.Clean();
return s;
}
// find new WAL and apply them in order to the secondary instance
Status DBImplSecondary::FindAndRecoverLogFiles(
std::unordered_set<ColumnFamilyData*>* cfds_changed,
JobContext* job_context) {
assert(nullptr != cfds_changed);
assert(nullptr != job_context);
Status s;
std::vector<uint64_t> logs;
s = FindNewLogNumbers(&logs);
if (s.ok() && !logs.empty()) {
SequenceNumber next_sequence(kMaxSequenceNumber);
s = RecoverLogFiles(logs, &next_sequence, cfds_changed, job_context);
}
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
return s;
}
// List wal_dir and find all new WALs, return these log numbers
Status DBImplSecondary::FindNewLogNumbers(std::vector<uint64_t>* logs) {
assert(logs != nullptr);
std::vector<std::string> filenames;
Status s;
s = env_->GetChildren(immutable_db_options_.GetWalDir(), &filenames);
if (s.IsNotFound()) {
return Status::InvalidArgument("Failed to open wal_dir",
immutable_db_options_.GetWalDir());
} else if (!s.ok()) {
return s;
}
// if log_readers_ is non-empty, it means we have applied all logs with log
// numbers smaller than the smallest log in log_readers_, so there is no
// need to pass these logs to RecoverLogFiles
uint64_t log_number_min = 0;
if (!log_readers_.empty()) {
log_number_min = log_readers_.begin()->first;
}
for (size_t i = 0; i < filenames.size(); i++) {
uint64_t number;
FileType type;
if (ParseFileName(filenames[i], &number, &type) && type == kWalFile &&
number >= log_number_min) {
logs->push_back(number);
}
}
// Recover logs in the order that they were generated
if (!logs->empty()) {
std::sort(logs->begin(), logs->end());
}
return s;
}
Status DBImplSecondary::MaybeInitLogReader(
uint64_t log_number, log::FragmentBufferedReader** log_reader) {
auto iter = log_readers_.find(log_number);
// make sure the log file is still present
if (iter == log_readers_.end() ||
iter->second->reader_->GetLogNumber() != log_number) {
// delete the obsolete log reader if log number mismatch
if (iter != log_readers_.end()) {
log_readers_.erase(iter);
}
// initialize log reader from log_number
// TODO: min_log_number_to_keep_2pc check needed?
// Open the log file
std::string fname =
LogFileName(immutable_db_options_.GetWalDir(), log_number);
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"Recovering log #%" PRIu64 " mode %d", log_number,
static_cast<int>(immutable_db_options_.wal_recovery_mode));
std::unique_ptr<SequentialFileReader> file_reader;
{
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
2019-12-13 22:47:08 +00:00
std::unique_ptr<FSSequentialFile> file;
Status status = fs_->NewSequentialFile(
fname, fs_->OptimizeForLogRead(file_options_), &file,
nullptr);
if (!status.ok()) {
*log_reader = nullptr;
return status;
}
file_reader.reset(new SequentialFileReader(
std::move(file), fname, immutable_db_options_.log_readahead_size,
io_tracer_));
}
// Create the log reader.
LogReaderContainer* log_reader_container = new LogReaderContainer(
env_, immutable_db_options_.info_log, std::move(fname),
std::move(file_reader), log_number);
log_readers_.insert(std::make_pair(
log_number, std::unique_ptr<LogReaderContainer>(log_reader_container)));
}
iter = log_readers_.find(log_number);
assert(iter != log_readers_.end());
*log_reader = iter->second->reader_;
return Status::OK();
}
// After manifest recovery, replay WALs and refresh log_readers_ if necessary
// REQUIRES: log_numbers are sorted in ascending order
Status DBImplSecondary::RecoverLogFiles(
const std::vector<uint64_t>& log_numbers, SequenceNumber* next_sequence,
std::unordered_set<ColumnFamilyData*>* cfds_changed,
JobContext* job_context) {
assert(nullptr != cfds_changed);
assert(nullptr != job_context);
mutex_.AssertHeld();
Status status;
for (auto log_number : log_numbers) {
log::FragmentBufferedReader* reader = nullptr;
status = MaybeInitLogReader(log_number, &reader);
if (!status.ok()) {
return status;
}
assert(reader != nullptr);
}
for (auto log_number : log_numbers) {
auto it = log_readers_.find(log_number);
assert(it != log_readers_.end());
log::FragmentBufferedReader* reader = it->second->reader_;
Status* wal_read_status = it->second->status_;
assert(wal_read_status);
// Manually update the file number allocation counter in VersionSet.
versions_->MarkFileNumberUsed(log_number);
// Determine if we should tolerate incomplete records at the tail end of the
// Read all the records and add to a memtable
std::string scratch;
Slice record;
WriteBatch batch;
while (reader->ReadRecord(&record, &scratch,
immutable_db_options_.wal_recovery_mode) &&
wal_read_status->ok() && status.ok()) {
if (record.size() < WriteBatchInternal::kHeader) {
reader->GetReporter()->Corruption(
record.size(), Status::Corruption("log record too small"));
continue;
}
status = WriteBatchInternal::SetContents(&batch, record);
if (!status.ok()) {
break;
}
SequenceNumber seq_of_batch = WriteBatchInternal::Sequence(&batch);
std::vector<uint32_t> column_family_ids;
status = CollectColumnFamilyIdsFromWriteBatch(batch, &column_family_ids);
if (status.ok()) {
for (const auto id : column_family_ids) {
ColumnFamilyData* cfd =
versions_->GetColumnFamilySet()->GetColumnFamily(id);
if (cfd == nullptr) {
continue;
}
if (cfds_changed->count(cfd) == 0) {
cfds_changed->insert(cfd);
}
Improve memtable earliest seqno assignment for secondary instance (#5413) Summary: In regular RocksDB instance, `MemTable::earliest_seqno_` is "db sequence number at the time of creation". However, we cannot use the db sequence number to set the value of `MemTable::earliest_seqno_` for secondary instance, i.e. `DBImplSecondary` due to the logic of MANIFEST and WAL replay. When replaying the log files of the primary, the secondary instance first replays MANIFEST and updates the db sequence number if necessary. Next, the secondary replays WAL files, creates new memtables if necessary and inserts key-value pairs into memtables. The following can occur when the db has two or more column families. Assume the db has column family "default" and "cf1". At a certain in time, both "default" and "cf1" have data in memtables. 1. Primary triggers a flush and flushes "cf1". "default" is **not** flushed. 2. Secondary replays the MANIFEST updates its db sequence number to the latest value learned from the MANIFEST. 3. Secondary starts to replay WAL that contains the writes to "default". It is possible that the write batches' sequence numbers are smaller than the db sequence number. In this case, these write batches will be skipped, and these updates will not be visible to reader until "default" is later flushed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5413 Differential Revision: D15637407 Pulled By: riversand963 fbshipit-source-id: 3de3fe35cfc6f1b9f844f3f926f0df29717b6580
2019-06-10 19:53:56 +00:00
const std::vector<FileMetaData*>& l0_files =
cfd->current()->storage_info()->LevelFiles(0);
SequenceNumber seq =
l0_files.empty() ? 0 : l0_files.back()->fd.largest_seqno;
// If the write batch's sequence number is smaller than the last
// sequence number of the largest sequence persisted for this column
// family, then its data must reside in an SST that has already been
// added in the prior MANIFEST replay.
if (seq_of_batch <= seq) {
continue;
}
auto curr_log_num = std::numeric_limits<uint64_t>::max();
if (cfd_to_current_log_.count(cfd) > 0) {
curr_log_num = cfd_to_current_log_[cfd];
}
// If the active memtable contains records added by replaying an
// earlier WAL, then we need to seal the memtable, add it to the
// immutable memtable list and create a new active memtable.
if (!cfd->mem()->IsEmpty() &&
(curr_log_num == std::numeric_limits<uint64_t>::max() ||
curr_log_num != log_number)) {
const MutableCFOptions mutable_cf_options =
*cfd->GetLatestMutableCFOptions();
MemTable* new_mem =
cfd->ConstructNewMemtable(mutable_cf_options, seq_of_batch);
cfd->mem()->SetNextLogNumber(log_number);
cfd->imm()->Add(cfd->mem(), &job_context->memtables_to_free);
new_mem->Ref();
cfd->SetMemtable(new_mem);
}
}
bool has_valid_writes = false;
status = WriteBatchInternal::InsertInto(
&batch, column_family_memtables_.get(),
Refactor trimming logic for immutable memtables (#5022) Summary: MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory. We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one. The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming. In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022 Differential Revision: D14394062 Pulled By: miasantreble fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 20:54:09 +00:00
nullptr /* flush_scheduler */, nullptr /* trim_history_scheduler*/,
true, log_number, this, false /* concurrent_memtable_writes */,
next_sequence, &has_valid_writes, seq_per_batch_, batch_per_txn_);
}
// If column family was not found, it might mean that the WAL write
// batch references to the column family that was dropped after the
// insert. We don't want to fail the whole write batch in that case --
// we just ignore the update.
// That's why we set ignore missing column families to true
// passing null flush_scheduler will disable memtable flushing which is
// needed for secondary instances
if (status.ok()) {
for (const auto id : column_family_ids) {
ColumnFamilyData* cfd =
versions_->GetColumnFamilySet()->GetColumnFamily(id);
if (cfd == nullptr) {
continue;
}
std::unordered_map<ColumnFamilyData*, uint64_t>::iterator iter =
cfd_to_current_log_.find(cfd);
if (iter == cfd_to_current_log_.end()) {
cfd_to_current_log_.insert({cfd, log_number});
} else if (log_number > iter->second) {
iter->second = log_number;
}
}
auto last_sequence = *next_sequence - 1;
if ((*next_sequence != kMaxSequenceNumber) &&
(versions_->LastSequence() <= last_sequence)) {
versions_->SetLastAllocatedSequence(last_sequence);
versions_->SetLastPublishedSequence(last_sequence);
versions_->SetLastSequence(last_sequence);
}
} else {
// We are treating this as a failure while reading since we read valid
// blocks that do not form coherent data
reader->GetReporter()->Corruption(record.size(), status);
}
}
if (status.ok() && !wal_read_status->ok()) {
status = *wal_read_status;
}
if (!status.ok()) {
return status;
}
}
// remove logreaders from map after successfully recovering the WAL
if (log_readers_.size() > 1) {
auto erase_iter = log_readers_.begin();
std::advance(erase_iter, log_readers_.size() - 1);
log_readers_.erase(log_readers_.begin(), erase_iter);
}
return status;
}
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
// Implementation of the DB interface
Status DBImplSecondary::Get(const ReadOptions& read_options,
ColumnFamilyHandle* column_family, const Slice& key,
PinnableSlice* value) {
return GetImpl(read_options, column_family, key, value);
}
Status DBImplSecondary::GetImpl(const ReadOptions& read_options,
ColumnFamilyHandle* column_family,
const Slice& key, PinnableSlice* pinnable_val) {
assert(pinnable_val != nullptr);
PERF_CPU_TIMER_GUARD(get_cpu_nanos, immutable_db_options_.clock);
StopWatch sw(immutable_db_options_.clock, stats_, DB_GET);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
PERF_TIMER_GUARD(get_snapshot_time);
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2022-02-02 06:17:46 +00:00
assert(column_family);
const Comparator* ucmp = column_family->GetComparator();
assert(ucmp);
if (ucmp->timestamp_size() || read_options.timestamp) {
// TODO: support timestamp
return Status::NotSupported();
}
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
auto cfh = static_cast<ColumnFamilyHandleImpl*>(column_family);
ColumnFamilyData* cfd = cfh->cfd();
if (tracer_) {
InstrumentedMutexLock lock(&trace_mutex_);
if (tracer_) {
tracer_->Get(column_family, key);
}
}
// Acquire SuperVersion
SuperVersion* super_version = GetAndRefSuperVersion(cfd);
SequenceNumber snapshot = versions_->LastSequence();
MergeContext merge_context;
SequenceNumber max_covering_tombstone_seq = 0;
Status s;
LookupKey lkey(key, snapshot);
PERF_TIMER_STOP(get_snapshot_time);
bool done = false;
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
2020-03-02 23:58:32 +00:00
if (super_version->mem->Get(lkey, pinnable_val->GetSelf(),
/*timestamp=*/nullptr, &s, &merge_context,
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
&max_covering_tombstone_seq, read_options)) {
done = true;
pinnable_val->PinSelf();
RecordTick(stats_, MEMTABLE_HIT);
} else if ((s.ok() || s.IsMergeInProgress()) &&
super_version->imm->Get(
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
2020-03-02 23:58:32 +00:00
lkey, pinnable_val->GetSelf(), /*timestamp=*/nullptr, &s,
&merge_context, &max_covering_tombstone_seq, read_options)) {
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
done = true;
pinnable_val->PinSelf();
RecordTick(stats_, MEMTABLE_HIT);
}
if (!done && !s.ok() && !s.IsMergeInProgress()) {
ReturnAndCleanupSuperVersion(cfd, super_version);
return s;
}
if (!done) {
PERF_TIMER_GUARD(get_from_output_files_time);
Fix PinSelf() read-after-free in DB::GetMergeOperands() (#9507) Summary: **Context:** Running the new test `DBMergeOperandTest.MergeOperandReadAfterFreeBug` prior to this fix surfaces the read-after-free bug of PinSef() as below: ``` READ of size 8 at 0x60400002529d thread T0 https://github.com/facebook/rocksdb/issues/5 0x7f199a in rocksdb::PinnableSlice::PinSelf(rocksdb::Slice const&) include/rocksdb/slice.h:171 https://github.com/facebook/rocksdb/issues/6 0x7f199a in rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::Slice const&, rocksdb::DBImpl::GetImplOptions&) db/db_impl/db_impl.cc:1919 https://github.com/facebook/rocksdb/issues/7 0x540d63 in rocksdb::DBImpl::GetMergeOperands(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::PinnableSlice*, rocksdb::GetMergeOperandsOptions*, int*) db/db_impl/db_impl.h:203 freed by thread T0 here: https://github.com/facebook/rocksdb/issues/3 0x1191399 in rocksdb::cache_entry_roles_detail::RegisteredDeleter<rocksdb::Block, (rocksdb::CacheEntryRole)0>::Delete(rocksdb::Slice const&, void*) cache/cache_entry_roles.h:99 https://github.com/facebook/rocksdb/issues/4 0x719348 in rocksdb::LRUHandle::Free() cache/lru_cache.h:205 https://github.com/facebook/rocksdb/issues/5 0x71047f in rocksdb::LRUCacheShard::Release(rocksdb::Cache::Handle*, bool) cache/lru_cache.cc:547 https://github.com/facebook/rocksdb/issues/6 0xa78f0a in rocksdb::Cleanable::DoCleanup() include/rocksdb/cleanable.h:60 https://github.com/facebook/rocksdb/issues/7 0xa78f0a in rocksdb::Cleanable::Reset() include/rocksdb/cleanable.h:38 https://github.com/facebook/rocksdb/issues/8 0xa78f0a in rocksdb::PinnedIteratorsManager::ReleasePinnedData() db/pinned_iterators_manager.h:71 https://github.com/facebook/rocksdb/issues/9 0xd0c21b in rocksdb::PinnedIteratorsManager::~PinnedIteratorsManager() db/pinned_iterators_manager.h:24 https://github.com/facebook/rocksdb/issues/10 0xd0c21b in rocksdb::Version::Get(rocksdb::ReadOptions const&, rocksdb::LookupKey const&, rocksdb::PinnableSlice*, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >*, rocksdb::Status*, rocksdb::MergeContext*, unsigned long*, bool*, bool*, unsigned long*, rocksdb::ReadCallback*, bool*, bool) db/pinned_iterators_manager.h:22 https://github.com/facebook/rocksdb/issues/11 0x7f0fdf in rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::Slice const&, rocksdb::DBImpl::GetImplOptions&) db/db_impl/db_impl.cc:1886 https://github.com/facebook/rocksdb/issues/12 0x540d63 in rocksdb::DBImpl::GetMergeOperands(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::PinnableSlice*, rocksdb::GetMergeOperandsOptions*, int*) db/db_impl/db_impl.h:203 previously allocated by thread T0 here: https://github.com/facebook/rocksdb/issues/1 0x1239896 in rocksdb::AllocateBlock(unsigned long, **rocksdb::MemoryAllocator*)** memory/memory_allocator.h:35 https://github.com/facebook/rocksdb/issues/2 0x1239896 in rocksdb::BlockFetcher::CopyBufferToHeapBuf() table/block_fetcher.cc:171 https://github.com/facebook/rocksdb/issues/3 0x1239896 in rocksdb::BlockFetcher::GetBlockContents() table/block_fetcher.cc:206 https://github.com/facebook/rocksdb/issues/4 0x122eae5 in rocksdb::BlockFetcher::ReadBlockContents() table/block_fetcher.cc:325 https://github.com/facebook/rocksdb/issues/5 0x11b1f45 in rocksdb::Status rocksdb::BlockBasedTable::MaybeReadBlockAndLoadToCache<rocksdb::Block>(rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, rocksdb::BlockHandle const&, rocksdb::UncompressionDict const&, bool, rocksdb::CachableEntry<rocksdb::Block>*, rocksdb::BlockType, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::BlockContents*) const table/block_based/block_based_table_reader.cc:1503 ``` Here is the analysis: - We have [PinnedIteratorsManager](https://github.com/facebook/rocksdb/blob/6.28.fb/db/version_set.cc#L1980) with `Cleanable` capability in our `Version::Get()` path. It's responsible for managing the life-time of pinned iterator and invoking registered cleanup functions during its own destruction. - For example in case above, the merge operands's clean-up gets associated with this manger in [GetContext::push_operand](https://github.com/facebook/rocksdb/blob/6.28.fb/table/get_context.cc#L405). During PinnedIteratorsManager's [destruction](https://github.com/facebook/rocksdb/blob/6.28.fb/db/pinned_iterators_manager.h#L67), the release function associated with those merge operand data is invoked. **And that's what we see in "freed by thread T955 here" in ASAN.** - Bug 🐛: `PinnedIteratorsManager` is local to `Version::Get()` while the data of merge operands need to outlive `Version::Get` and stay till they get [PinSelf()](https://github.com/facebook/rocksdb/blob/6.28.fb/db/db_impl/db_impl.cc#L1905), **which is the read-after-free in ASAN.** - This bug is likely to be an overlook of `PinnedIteratorsManager` when developing the API `DB::GetMergeOperands` cuz the current logic works fine with the existing case of getting the *merged value* where the operands do not need to live that long. - This bug was not surfaced much (even in its unit test) due to the release function associated with the merge operands (which are actually blocks put in cache as you can see in `BlockBasedTable::MaybeReadBlockAndLoadToCache` **in "previously allocated by" in ASAN report**) is a cache entry deleter. The deleter will call `Cache::Release()` which, for LRU cache, won't immediately deallocate the block based on LRU policy [unless the cache is full or being instructed to force erase](https://github.com/facebook/rocksdb/blob/6.28.fb/cache/lru_cache.cc#L521-L531) - `DBMergeOperandTest.MergeOperandReadAfterFreeBug` makes the cache extremely small to force cache full. **Summary:** - Fix the bug by align `PinnedIteratorsManager`'s lifetime with the merge operands Pull Request resolved: https://github.com/facebook/rocksdb/pull/9507 Test Plan: - New test `DBMergeOperandTest.MergeOperandReadAfterFreeBug` - db bench on read path - Setup (LSM tree with several levels, cache the whole db to avoid read IO, warm cache with readseq to avoid read IO): `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="fillrandom,readseq -num=1000000 -cache_size=100000000 -write_buffer_size=10000 -statistics=1 -max_bytes_for_level_base=10000 -level0_file_num_compaction_trigger=1``TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="readrandom" -num=1000000 -cache_size=100000000 ` - Actual command run (run 20-run for 20 times and then average the 20-run's average micros/op) - `for j in {1..20}; do (for i in {1..20}; do rm -rf /dev/shm/rocksdb/ && TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="fillrandom,readseq,readrandom" -num=1000000 -cache_size=100000000 -write_buffer_size=10000 -statistics=1 -max_bytes_for_level_base=10000 -level0_file_num_compaction_trigger=1 | egrep 'readrandom'; done > rr_output_pre.txt && (awk '{sum+=$3; sum_sqrt+=$3^2}END{print sum/20, sqrt(sum_sqrt/20-(sum/20)^2)}' rr_output_pre.txt) >> rr_output_pre_2.txt); done` - **Result: Pre-change: 3.79193 micros/op; Post-change: 3.79528 micros/op (+0.09%)** (pre-change)sorted avg micros/op of each 20-run | std of micros/op of each 20-run | (post-change) sorted avg micros/op of each 20-run | std of micros/op of each 20-run -- | -- | -- | -- 3.58355 | 0.265209 | 3.48715 | 0.382076 3.58845 | 0.519927 | 3.5832 | 0.382726 3.66415 | 0.452097 | 3.677 | 0.563831 3.68495 | 0.430897 | 3.68405 | 0.495355 3.70295 | 0.482893 | 3.68465 | 0.431438 3.719 | 0.463806 | 3.71945 | 0.457157 3.7393 | 0.453423 | 3.72795 | 0.538604 3.7806 | 0.527613 | 3.75075 | 0.444509 3.7817 | 0.426704 | 3.7683 | 0.468065 3.809 | 0.381033 | 3.8086 | 0.557378 3.80985 | 0.466011 | 3.81805 | 0.524833 3.8165 | 0.500351 | 3.83405 | 0.529339 3.8479 | 0.430326 | 3.86285 | 0.44831 3.85125 | 0.434108 | 3.8717 | 0.544098 3.8556 | 0.524602 | 3.895 | 0.411679 3.8656 | 0.476383 | 3.90965 | 0.566636 3.8911 | 0.488477 | 3.92735 | 0.608038 3.898 | 0.493978 | 3.9439 | 0.524511 3.97235 | 0.515008 | 3.9623 | 0.477416 3.9768 | 0.519993 | 3.98965 | 0.521481 - CI Reviewed By: ajkr Differential Revision: D34030519 Pulled By: hx235 fbshipit-source-id: a99ac585c11704c5ed93af033cb29ba0a7b16ae8
2022-02-15 20:24:05 +00:00
PinnedIteratorsManager pinned_iters_mgr;
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
2020-03-02 23:58:32 +00:00
super_version->current->Get(read_options, lkey, pinnable_val,
/*timestamp=*/nullptr, &s, &merge_context,
Fix PinSelf() read-after-free in DB::GetMergeOperands() (#9507) Summary: **Context:** Running the new test `DBMergeOperandTest.MergeOperandReadAfterFreeBug` prior to this fix surfaces the read-after-free bug of PinSef() as below: ``` READ of size 8 at 0x60400002529d thread T0 https://github.com/facebook/rocksdb/issues/5 0x7f199a in rocksdb::PinnableSlice::PinSelf(rocksdb::Slice const&) include/rocksdb/slice.h:171 https://github.com/facebook/rocksdb/issues/6 0x7f199a in rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::Slice const&, rocksdb::DBImpl::GetImplOptions&) db/db_impl/db_impl.cc:1919 https://github.com/facebook/rocksdb/issues/7 0x540d63 in rocksdb::DBImpl::GetMergeOperands(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::PinnableSlice*, rocksdb::GetMergeOperandsOptions*, int*) db/db_impl/db_impl.h:203 freed by thread T0 here: https://github.com/facebook/rocksdb/issues/3 0x1191399 in rocksdb::cache_entry_roles_detail::RegisteredDeleter<rocksdb::Block, (rocksdb::CacheEntryRole)0>::Delete(rocksdb::Slice const&, void*) cache/cache_entry_roles.h:99 https://github.com/facebook/rocksdb/issues/4 0x719348 in rocksdb::LRUHandle::Free() cache/lru_cache.h:205 https://github.com/facebook/rocksdb/issues/5 0x71047f in rocksdb::LRUCacheShard::Release(rocksdb::Cache::Handle*, bool) cache/lru_cache.cc:547 https://github.com/facebook/rocksdb/issues/6 0xa78f0a in rocksdb::Cleanable::DoCleanup() include/rocksdb/cleanable.h:60 https://github.com/facebook/rocksdb/issues/7 0xa78f0a in rocksdb::Cleanable::Reset() include/rocksdb/cleanable.h:38 https://github.com/facebook/rocksdb/issues/8 0xa78f0a in rocksdb::PinnedIteratorsManager::ReleasePinnedData() db/pinned_iterators_manager.h:71 https://github.com/facebook/rocksdb/issues/9 0xd0c21b in rocksdb::PinnedIteratorsManager::~PinnedIteratorsManager() db/pinned_iterators_manager.h:24 https://github.com/facebook/rocksdb/issues/10 0xd0c21b in rocksdb::Version::Get(rocksdb::ReadOptions const&, rocksdb::LookupKey const&, rocksdb::PinnableSlice*, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >*, rocksdb::Status*, rocksdb::MergeContext*, unsigned long*, bool*, bool*, unsigned long*, rocksdb::ReadCallback*, bool*, bool) db/pinned_iterators_manager.h:22 https://github.com/facebook/rocksdb/issues/11 0x7f0fdf in rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::Slice const&, rocksdb::DBImpl::GetImplOptions&) db/db_impl/db_impl.cc:1886 https://github.com/facebook/rocksdb/issues/12 0x540d63 in rocksdb::DBImpl::GetMergeOperands(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::PinnableSlice*, rocksdb::GetMergeOperandsOptions*, int*) db/db_impl/db_impl.h:203 previously allocated by thread T0 here: https://github.com/facebook/rocksdb/issues/1 0x1239896 in rocksdb::AllocateBlock(unsigned long, **rocksdb::MemoryAllocator*)** memory/memory_allocator.h:35 https://github.com/facebook/rocksdb/issues/2 0x1239896 in rocksdb::BlockFetcher::CopyBufferToHeapBuf() table/block_fetcher.cc:171 https://github.com/facebook/rocksdb/issues/3 0x1239896 in rocksdb::BlockFetcher::GetBlockContents() table/block_fetcher.cc:206 https://github.com/facebook/rocksdb/issues/4 0x122eae5 in rocksdb::BlockFetcher::ReadBlockContents() table/block_fetcher.cc:325 https://github.com/facebook/rocksdb/issues/5 0x11b1f45 in rocksdb::Status rocksdb::BlockBasedTable::MaybeReadBlockAndLoadToCache<rocksdb::Block>(rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, rocksdb::BlockHandle const&, rocksdb::UncompressionDict const&, bool, rocksdb::CachableEntry<rocksdb::Block>*, rocksdb::BlockType, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::BlockContents*) const table/block_based/block_based_table_reader.cc:1503 ``` Here is the analysis: - We have [PinnedIteratorsManager](https://github.com/facebook/rocksdb/blob/6.28.fb/db/version_set.cc#L1980) with `Cleanable` capability in our `Version::Get()` path. It's responsible for managing the life-time of pinned iterator and invoking registered cleanup functions during its own destruction. - For example in case above, the merge operands's clean-up gets associated with this manger in [GetContext::push_operand](https://github.com/facebook/rocksdb/blob/6.28.fb/table/get_context.cc#L405). During PinnedIteratorsManager's [destruction](https://github.com/facebook/rocksdb/blob/6.28.fb/db/pinned_iterators_manager.h#L67), the release function associated with those merge operand data is invoked. **And that's what we see in "freed by thread T955 here" in ASAN.** - Bug 🐛: `PinnedIteratorsManager` is local to `Version::Get()` while the data of merge operands need to outlive `Version::Get` and stay till they get [PinSelf()](https://github.com/facebook/rocksdb/blob/6.28.fb/db/db_impl/db_impl.cc#L1905), **which is the read-after-free in ASAN.** - This bug is likely to be an overlook of `PinnedIteratorsManager` when developing the API `DB::GetMergeOperands` cuz the current logic works fine with the existing case of getting the *merged value* where the operands do not need to live that long. - This bug was not surfaced much (even in its unit test) due to the release function associated with the merge operands (which are actually blocks put in cache as you can see in `BlockBasedTable::MaybeReadBlockAndLoadToCache` **in "previously allocated by" in ASAN report**) is a cache entry deleter. The deleter will call `Cache::Release()` which, for LRU cache, won't immediately deallocate the block based on LRU policy [unless the cache is full or being instructed to force erase](https://github.com/facebook/rocksdb/blob/6.28.fb/cache/lru_cache.cc#L521-L531) - `DBMergeOperandTest.MergeOperandReadAfterFreeBug` makes the cache extremely small to force cache full. **Summary:** - Fix the bug by align `PinnedIteratorsManager`'s lifetime with the merge operands Pull Request resolved: https://github.com/facebook/rocksdb/pull/9507 Test Plan: - New test `DBMergeOperandTest.MergeOperandReadAfterFreeBug` - db bench on read path - Setup (LSM tree with several levels, cache the whole db to avoid read IO, warm cache with readseq to avoid read IO): `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="fillrandom,readseq -num=1000000 -cache_size=100000000 -write_buffer_size=10000 -statistics=1 -max_bytes_for_level_base=10000 -level0_file_num_compaction_trigger=1``TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="readrandom" -num=1000000 -cache_size=100000000 ` - Actual command run (run 20-run for 20 times and then average the 20-run's average micros/op) - `for j in {1..20}; do (for i in {1..20}; do rm -rf /dev/shm/rocksdb/ && TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="fillrandom,readseq,readrandom" -num=1000000 -cache_size=100000000 -write_buffer_size=10000 -statistics=1 -max_bytes_for_level_base=10000 -level0_file_num_compaction_trigger=1 | egrep 'readrandom'; done > rr_output_pre.txt && (awk '{sum+=$3; sum_sqrt+=$3^2}END{print sum/20, sqrt(sum_sqrt/20-(sum/20)^2)}' rr_output_pre.txt) >> rr_output_pre_2.txt); done` - **Result: Pre-change: 3.79193 micros/op; Post-change: 3.79528 micros/op (+0.09%)** (pre-change)sorted avg micros/op of each 20-run | std of micros/op of each 20-run | (post-change) sorted avg micros/op of each 20-run | std of micros/op of each 20-run -- | -- | -- | -- 3.58355 | 0.265209 | 3.48715 | 0.382076 3.58845 | 0.519927 | 3.5832 | 0.382726 3.66415 | 0.452097 | 3.677 | 0.563831 3.68495 | 0.430897 | 3.68405 | 0.495355 3.70295 | 0.482893 | 3.68465 | 0.431438 3.719 | 0.463806 | 3.71945 | 0.457157 3.7393 | 0.453423 | 3.72795 | 0.538604 3.7806 | 0.527613 | 3.75075 | 0.444509 3.7817 | 0.426704 | 3.7683 | 0.468065 3.809 | 0.381033 | 3.8086 | 0.557378 3.80985 | 0.466011 | 3.81805 | 0.524833 3.8165 | 0.500351 | 3.83405 | 0.529339 3.8479 | 0.430326 | 3.86285 | 0.44831 3.85125 | 0.434108 | 3.8717 | 0.544098 3.8556 | 0.524602 | 3.895 | 0.411679 3.8656 | 0.476383 | 3.90965 | 0.566636 3.8911 | 0.488477 | 3.92735 | 0.608038 3.898 | 0.493978 | 3.9439 | 0.524511 3.97235 | 0.515008 | 3.9623 | 0.477416 3.9768 | 0.519993 | 3.98965 | 0.521481 - CI Reviewed By: ajkr Differential Revision: D34030519 Pulled By: hx235 fbshipit-source-id: a99ac585c11704c5ed93af033cb29ba0a7b16ae8
2022-02-15 20:24:05 +00:00
&max_covering_tombstone_seq, &pinned_iters_mgr);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
RecordTick(stats_, MEMTABLE_MISS);
}
{
PERF_TIMER_GUARD(get_post_process_time);
ReturnAndCleanupSuperVersion(cfd, super_version);
RecordTick(stats_, NUMBER_KEYS_READ);
size_t size = pinnable_val->size();
RecordTick(stats_, BYTES_READ, size);
RecordTimeToHistogram(stats_, BYTES_PER_READ, size);
PERF_COUNTER_ADD(get_read_bytes, size);
}
return s;
}
Iterator* DBImplSecondary::NewIterator(const ReadOptions& read_options,
ColumnFamilyHandle* column_family) {
if (read_options.managed) {
return NewErrorIterator(
Status::NotSupported("Managed iterator is not supported anymore."));
}
if (read_options.read_tier == kPersistedTier) {
return NewErrorIterator(Status::NotSupported(
"ReadTier::kPersistedData is not yet supported in iterators."));
}
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2022-02-02 06:17:46 +00:00
assert(column_family);
const Comparator* ucmp = column_family->GetComparator();
assert(ucmp);
if (ucmp->timestamp_size() || read_options.timestamp) {
// TODO: support timestamp
return NewErrorIterator(Status::NotSupported());
}
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
Iterator* result = nullptr;
auto cfh = static_cast_with_check<ColumnFamilyHandleImpl>(column_family);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
auto cfd = cfh->cfd();
ReadCallback* read_callback = nullptr; // No read callback provided.
if (read_options.tailing) {
return NewErrorIterator(Status::NotSupported(
"tailing iterator not supported in secondary mode"));
} else if (read_options.snapshot != nullptr) {
// TODO (yanqin) support snapshot.
return NewErrorIterator(
Status::NotSupported("snapshot not supported in secondary mode"));
} else {
SequenceNumber snapshot(kMaxSequenceNumber);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
result = NewIteratorImpl(read_options, cfd, snapshot, read_callback);
}
return result;
}
ArenaWrappedDBIter* DBImplSecondary::NewIteratorImpl(
const ReadOptions& read_options, ColumnFamilyData* cfd,
SequenceNumber snapshot, ReadCallback* read_callback,
bool expose_blob_index, bool allow_refresh) {
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
assert(nullptr != cfd);
SuperVersion* super_version = cfd->GetReferencedSuperVersion(this);
assert(snapshot == kMaxSequenceNumber);
snapshot = versions_->LastSequence();
assert(snapshot != kMaxSequenceNumber);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
auto db_iter = NewArenaWrappedDbIterator(
env_, read_options, *cfd->ioptions(), super_version->mutable_cf_options,
super_version->current, snapshot,
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
super_version->mutable_cf_options.max_sequential_skip_in_iterations,
super_version->version_number, read_callback, this, cfd,
expose_blob_index, read_options.snapshot ? false : allow_refresh);
auto internal_iter = NewInternalIterator(
db_iter->GetReadOptions(), cfd, super_version, db_iter->GetArena(),
db_iter->GetRangeDelAggregator(), snapshot,
/* allow_unprepared_value */ true);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
db_iter->SetIterUnderDBIter(internal_iter);
return db_iter;
}
Status DBImplSecondary::NewIterators(
const ReadOptions& read_options,
const std::vector<ColumnFamilyHandle*>& column_families,
std::vector<Iterator*>* iterators) {
if (read_options.managed) {
return Status::NotSupported("Managed iterator is not supported anymore.");
}
if (read_options.read_tier == kPersistedTier) {
return Status::NotSupported(
"ReadTier::kPersistedData is not yet supported in iterators.");
}
ReadCallback* read_callback = nullptr; // No read callback provided.
if (iterators == nullptr) {
return Status::InvalidArgument("iterators not allowed to be nullptr");
}
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2022-02-02 06:17:46 +00:00
if (read_options.timestamp) {
// TODO: support timestamp
return Status::NotSupported();
} else {
for (auto* cf : column_families) {
assert(cf);
const Comparator* ucmp = cf->GetComparator();
assert(ucmp);
if (ucmp->timestamp_size()) {
// TODO: support timestamp
return Status::NotSupported();
}
}
}
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
iterators->clear();
iterators->reserve(column_families.size());
if (read_options.tailing) {
return Status::NotSupported(
"tailing iterator not supported in secondary mode");
} else if (read_options.snapshot != nullptr) {
// TODO (yanqin) support snapshot.
return Status::NotSupported("snapshot not supported in secondary mode");
} else {
SequenceNumber read_seq = versions_->LastSequence();
for (auto cfh : column_families) {
ColumnFamilyData* cfd = static_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
iterators->push_back(
NewIteratorImpl(read_options, cfd, read_seq, read_callback));
}
}
return Status::OK();
}
Status DBImplSecondary::CheckConsistency() {
mutex_.AssertHeld();
Status s = DBImpl::CheckConsistency();
// If DBImpl::CheckConsistency() which is stricter returns success, then we
// do not need to give a second chance.
if (s.ok()) {
return s;
}
// It's possible that DBImpl::CheckConssitency() can fail because the primary
// may have removed certain files, causing the GetFileSize(name) call to
// fail and returning a PathNotFound. In this case, we take a best-effort
// approach and just proceed.
TEST_SYNC_POINT_CALLBACK(
"DBImplSecondary::CheckConsistency:AfterFirstAttempt", &s);
if (immutable_db_options_.skip_checking_sst_file_sizes_on_db_open) {
return Status::OK();
}
std::vector<LiveFileMetaData> metadata;
versions_->GetLiveFilesMetaData(&metadata);
std::string corruption_messages;
for (const auto& md : metadata) {
// md.name has a leading "/".
std::string file_path = md.db_path + md.name;
uint64_t fsize = 0;
s = env_->GetFileSize(file_path, &fsize);
if (!s.ok() &&
(env_->GetFileSize(Rocks2LevelTableFileName(file_path), &fsize).ok() ||
s.IsPathNotFound())) {
s = Status::OK();
}
if (!s.ok()) {
corruption_messages +=
"Can't access " + md.name + ": " + s.ToString() + "\n";
}
}
return corruption_messages.empty() ? Status::OK()
: Status::Corruption(corruption_messages);
}
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
Status DBImplSecondary::TryCatchUpWithPrimary() {
assert(versions_.get() != nullptr);
assert(manifest_reader_.get() != nullptr);
Status s;
// read the manifest and apply new changes to the secondary instance
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
std::unordered_set<ColumnFamilyData*> cfds_changed;
JobContext job_context(0, true /*create_superversion*/);
{
InstrumentedMutexLock lock_guard(&mutex_);
s = static_cast_with_check<ReactiveVersionSet>(versions_.get())
->ReadAndApply(&mutex_, &manifest_reader_,
manifest_reader_status_.get(), &cfds_changed);
ROCKS_LOG_INFO(immutable_db_options_.info_log, "Last sequence is %" PRIu64,
static_cast<uint64_t>(versions_->LastSequence()));
for (ColumnFamilyData* cfd : cfds_changed) {
if (cfd->IsDropped()) {
ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "[%s] is dropped\n",
cfd->GetName().c_str());
continue;
}
VersionStorageInfo::LevelSummaryStorage tmp;
ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
"[%s] Level summary: %s\n", cfd->GetName().c_str(),
cfd->current()->storage_info()->LevelSummary(&tmp));
Improve memtable earliest seqno assignment for secondary instance (#5413) Summary: In regular RocksDB instance, `MemTable::earliest_seqno_` is "db sequence number at the time of creation". However, we cannot use the db sequence number to set the value of `MemTable::earliest_seqno_` for secondary instance, i.e. `DBImplSecondary` due to the logic of MANIFEST and WAL replay. When replaying the log files of the primary, the secondary instance first replays MANIFEST and updates the db sequence number if necessary. Next, the secondary replays WAL files, creates new memtables if necessary and inserts key-value pairs into memtables. The following can occur when the db has two or more column families. Assume the db has column family "default" and "cf1". At a certain in time, both "default" and "cf1" have data in memtables. 1. Primary triggers a flush and flushes "cf1". "default" is **not** flushed. 2. Secondary replays the MANIFEST updates its db sequence number to the latest value learned from the MANIFEST. 3. Secondary starts to replay WAL that contains the writes to "default". It is possible that the write batches' sequence numbers are smaller than the db sequence number. In this case, these write batches will be skipped, and these updates will not be visible to reader until "default" is later flushed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5413 Differential Revision: D15637407 Pulled By: riversand963 fbshipit-source-id: 3de3fe35cfc6f1b9f844f3f926f0df29717b6580
2019-06-10 19:53:56 +00:00
}
// list wal_dir to discover new WALs and apply new changes to the secondary
// instance
if (s.ok()) {
s = FindAndRecoverLogFiles(&cfds_changed, &job_context);
}
if (s.IsPathNotFound()) {
ROCKS_LOG_INFO(
immutable_db_options_.info_log,
"Secondary tries to read WAL, but WAL file(s) have already "
"been purged by primary.");
s = Status::OK();
}
if (s.ok()) {
for (auto cfd : cfds_changed) {
cfd->imm()->RemoveOldMemTables(cfd->GetLogNumber(),
&job_context.memtables_to_free);
auto& sv_context = job_context.superversion_contexts.back();
cfd->InstallSuperVersion(&sv_context, &mutex_);
sv_context.NewSuperVersion();
}
}
}
job_context.Clean();
// Cleanup unused, obsolete files.
JobContext purge_files_job_context(0);
{
InstrumentedMutexLock lock_guard(&mutex_);
// Currently, secondary instance does not own the database files, thus it
// is unnecessary for the secondary to force full scan.
FindObsoleteFiles(&purge_files_job_context, /*force=*/false);
}
if (purge_files_job_context.HaveSomethingToDelete()) {
PurgeObsoleteFiles(purge_files_job_context);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
}
purge_files_job_context.Clean();
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
return s;
}
Status DB::OpenAsSecondary(const Options& options, const std::string& dbname,
const std::string& secondary_path, DB** dbptr) {
*dbptr = nullptr;
DBOptions db_options(options);
ColumnFamilyOptions cf_options(options);
std::vector<ColumnFamilyDescriptor> column_families;
column_families.emplace_back(kDefaultColumnFamilyName, cf_options);
std::vector<ColumnFamilyHandle*> handles;
Status s = DB::OpenAsSecondary(db_options, dbname, secondary_path,
column_families, &handles, dbptr);
if (s.ok()) {
assert(handles.size() == 1);
delete handles[0];
}
return s;
}
Status DB::OpenAsSecondary(
const DBOptions& db_options, const std::string& dbname,
const std::string& secondary_path,
const std::vector<ColumnFamilyDescriptor>& column_families,
std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
*dbptr = nullptr;
if (db_options.max_open_files != -1) {
// TODO (yanqin) maybe support max_open_files != -1 by creating hard links
// on SST files so that db secondary can still have access to old SSTs
// while primary instance may delete original.
return Status::InvalidArgument("require max_open_files to be -1");
}
DBOptions tmp_opts(db_options);
Status s;
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
if (nullptr == tmp_opts.info_log) {
s = CreateLoggerFromOptions(secondary_path, tmp_opts, &tmp_opts.info_log);
if (!s.ok()) {
tmp_opts.info_log = nullptr;
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
}
}
handles->clear();
DBImplSecondary* impl = new DBImplSecondary(tmp_opts, dbname, secondary_path);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
impl->versions_.reset(new ReactiveVersionSet(
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
2019-12-13 22:47:08 +00:00
dbname, &impl->immutable_db_options_, impl->file_options_,
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
impl->table_cache_.get(), impl->write_buffer_manager_,
&impl->write_controller_, impl->io_tracer_));
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
impl->column_family_memtables_.reset(
new ColumnFamilyMemTablesImpl(impl->versions_->GetColumnFamilySet()));
impl->wal_in_db_path_ = impl->immutable_db_options_.IsWalDirSameAsDBPath();
setup wal_in_db_path_ for secondary instance (#5545) Summary: PR https://github.com/facebook/rocksdb/pull/5520 adds DBImpl:: wal_in_db_path_ and initializes it in DBImpl::Open, this PR fixes the valgrind error for secondary instance: ``` ==236417== Conditional jump or move depends on uninitialised value(s) ==236417== at 0x62242A: rocksdb::DeleteDBFile(rocksdb::ImmutableDBOptions const*, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, bool, bool) (file_util.cc:96) ==236417== by 0x512432: rocksdb::DBImpl::DeleteObsoleteFileImpl(int, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileType, unsigned long) (db_impl_files.cc:261) ==236417== by 0x515A7A: rocksdb::DBImpl::PurgeObsoleteFiles(rocksdb::JobContext&, bool) (db_impl_files.cc:492) ==236417== by 0x499153: rocksdb::ColumnFamilyHandleImpl::~ColumnFamilyHandleImpl() (column_family.cc:75) ==236417== by 0x499880: rocksdb::ColumnFamilyHandleImpl::~ColumnFamilyHandleImpl() (column_family.cc:84) ==236417== by 0x4C9AF9: rocksdb::DB::DestroyColumnFamilyHandle(rocksdb::ColumnFamilyHandle*) (db_impl.cc:3105) ==236417== by 0x44E853: CloseSecondary (db_secondary_test.cc:53) ==236417== by 0x44E853: rocksdb::DBSecondaryTest::~DBSecondaryTest() (db_secondary_test.cc:31) ==236417== by 0x44EC77: ~DBSecondaryTest_PrimaryDropColumnFamily_Test (db_secondary_test.cc:443) ==236417== by 0x44EC77: rocksdb::DBSecondaryTest_PrimaryDropColumnFamily_Test::~DBSecondaryTest_PrimaryDropColumnFamily_Test() (db_secondary_test.cc:443) ==236417== by 0x83D1D7: HandleSehExceptionsInMethodIfSupported<testing::Test, void> (gtest-all.cc:3824) ==236417== by 0x83D1D7: void testing::internal::HandleExceptionsInMethodIfSupported<testing::Test, void>(testing::Test*, void (testing::Test::*)(), char const*) (gtest-all.cc:3860) ==236417== by 0x8346DB: testing::TestInfo::Run() [clone .part.486] (gtest-all.cc:4078) ==236417== by 0x8348D4: Run (gtest-all.cc:4047) ==236417== by 0x8348D4: testing::TestCase::Run() [clone .part.487] (gtest-all.cc:4190) ==236417== by 0x834D14: Run (gtest-all.cc:6100) ==236417== by 0x834D14: testing::internal::UnitTestImpl::RunAllTests() (gtest-all.cc:6062) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/5545 Differential Revision: D16146224 Pulled By: miasantreble fbshipit-source-id: 184c90e451352951da4e955f054d4b1a1f29ea29
2019-07-08 04:29:39 +00:00
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
impl->mutex_.Lock();
s = impl->Recover(column_families, true, false, false);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
if (s.ok()) {
for (auto cf : column_families) {
auto cfd =
impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
if (nullptr == cfd) {
s = Status::InvalidArgument("Column family not found", cf.name);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
break;
}
handles->push_back(new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
}
}
SuperVersionContext sv_context(true /* create_superversion */);
if (s.ok()) {
for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
sv_context.NewSuperVersion();
cfd->InstallSuperVersion(&sv_context, &impl->mutex_);
}
}
impl->mutex_.Unlock();
sv_context.Clean();
if (s.ok()) {
*dbptr = impl;
for (auto h : *handles) {
impl->NewThreadStatusCfInfo(
static_cast_with_check<ColumnFamilyHandleImpl>(h)->cfd());
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
}
} else {
for (auto h : *handles) {
delete h;
}
handles->clear();
delete impl;
}
return s;
}
Status DBImplSecondary::CompactWithoutInstallation(
const OpenAndCompactOptions& options, ColumnFamilyHandle* cfh,
const CompactionServiceInput& input, CompactionServiceResult* result) {
if (options.canceled && options.canceled->load(std::memory_order_acquire)) {
return Status::Incomplete(Status::SubCode::kManualCompactionPaused);
}
InstrumentedMutexLock l(&mutex_);
auto cfd = static_cast_with_check<ColumnFamilyHandleImpl>(cfh)->cfd();
if (!cfd) {
return Status::InvalidArgument("Cannot find column family" +
cfh->GetName());
}
std::unordered_set<uint64_t> input_set;
for (const auto& file_name : input.input_files) {
input_set.insert(TableFileNameToNumber(file_name));
}
auto* version = cfd->current();
ColumnFamilyMetaData cf_meta;
version->GetColumnFamilyMetaData(&cf_meta);
const MutableCFOptions* mutable_cf_options = cfd->GetLatestMutableCFOptions();
ColumnFamilyOptions cf_options = cfd->GetLatestCFOptions();
VersionStorageInfo* vstorage = version->storage_info();
// Use comp_options to reuse some CompactFiles functions
CompactionOptions comp_options;
comp_options.compression = kDisableCompressionOption;
comp_options.output_file_size_limit = MaxFileSizeForLevel(
*mutable_cf_options, input.output_level, cf_options.compaction_style,
vstorage->base_level(), cf_options.level_compaction_dynamic_level_bytes);
std::vector<CompactionInputFiles> input_files;
Status s = cfd->compaction_picker()->GetCompactionInputsFromFileNumbers(
&input_files, &input_set, vstorage, comp_options);
if (!s.ok()) {
return s;
}
std::unique_ptr<Compaction> c;
assert(cfd->compaction_picker());
c.reset(cfd->compaction_picker()->CompactFiles(
comp_options, input_files, input.output_level, vstorage,
*mutable_cf_options, mutable_db_options_, 0));
assert(c != nullptr);
c->SetInputVersion(version);
// Create output directory if it's not existed yet
std::unique_ptr<FSDirectory> output_dir;
s = CreateAndNewDirectory(fs_.get(), secondary_path_, &output_dir);
if (!s.ok()) {
return s;
}
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL,
immutable_db_options_.info_log.get());
const int job_id = next_job_id_.fetch_add(1);
CompactionServiceCompactionJob compaction_job(
job_id, c.get(), immutable_db_options_, mutable_db_options_,
file_options_for_compaction_, versions_.get(), &shutting_down_,
&log_buffer, output_dir.get(), stats_, &mutex_, &error_handler_,
input.snapshots, table_cache_, &event_logger_, dbname_, io_tracer_,
options.canceled, db_id_, db_session_id_, secondary_path_, input, result);
mutex_.Unlock();
s = compaction_job.Run();
mutex_.Lock();
// clean up
compaction_job.io_status().PermitUncheckedError();
compaction_job.CleanupCompaction();
c->ReleaseCompactionFiles(s);
c.reset();
TEST_SYNC_POINT_CALLBACK("DBImplSecondary::CompactWithoutInstallation::End",
&s);
result->status = s;
return s;
}
Status DB::OpenAndCompact(
const OpenAndCompactOptions& options, const std::string& name,
const std::string& output_directory, const std::string& input,
std::string* output,
const CompactionServiceOptionsOverride& override_options) {
if (options.canceled && options.canceled->load(std::memory_order_acquire)) {
return Status::Incomplete(Status::SubCode::kManualCompactionPaused);
}
CompactionServiceInput compaction_input;
Status s = CompactionServiceInput::Read(input, &compaction_input);
if (!s.ok()) {
return s;
}
compaction_input.db_options.max_open_files = -1;
compaction_input.db_options.compaction_service = nullptr;
if (compaction_input.db_options.statistics) {
compaction_input.db_options.statistics.reset();
}
compaction_input.db_options.env = override_options.env;
compaction_input.db_options.file_checksum_gen_factory =
override_options.file_checksum_gen_factory;
compaction_input.db_options.statistics = override_options.statistics;
compaction_input.column_family.options.comparator =
override_options.comparator;
compaction_input.column_family.options.merge_operator =
override_options.merge_operator;
compaction_input.column_family.options.compaction_filter =
override_options.compaction_filter;
compaction_input.column_family.options.compaction_filter_factory =
override_options.compaction_filter_factory;
compaction_input.column_family.options.prefix_extractor =
override_options.prefix_extractor;
compaction_input.column_family.options.table_factory =
override_options.table_factory;
compaction_input.column_family.options.sst_partitioner_factory =
override_options.sst_partitioner_factory;
compaction_input.column_family.options.table_properties_collector_factories =
override_options.table_properties_collector_factories;
compaction_input.db_options.listeners = override_options.listeners;
std::vector<ColumnFamilyDescriptor> column_families;
column_families.push_back(compaction_input.column_family);
// TODO: we have to open default CF, because of an implementation limitation,
// currently we just use the same CF option from input, which is not collect
// and open may fail.
if (compaction_input.column_family.name != kDefaultColumnFamilyName) {
column_families.emplace_back(kDefaultColumnFamilyName,
compaction_input.column_family.options);
}
DB* db;
std::vector<ColumnFamilyHandle*> handles;
s = DB::OpenAsSecondary(compaction_input.db_options, name, output_directory,
column_families, &handles, &db);
if (!s.ok()) {
return s;
}
CompactionServiceResult compaction_result;
DBImplSecondary* db_secondary = static_cast_with_check<DBImplSecondary>(db);
assert(handles.size() > 0);
s = db_secondary->CompactWithoutInstallation(
options, handles[0], compaction_input, &compaction_result);
Status serialization_status = compaction_result.Write(output);
for (auto& handle : handles) {
delete handle;
}
delete db;
if (s.ok()) {
return serialization_status;
}
return s;
}
Status DB::OpenAndCompact(
const std::string& name, const std::string& output_directory,
const std::string& input, std::string* output,
const CompactionServiceOptionsOverride& override_options) {
return OpenAndCompact(OpenAndCompactOptions(), name, output_directory, input,
output, override_options);
}
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
2019-03-26 23:41:31 +00:00
#else // !ROCKSDB_LITE
Status DB::OpenAsSecondary(const Options& /*options*/,
const std::string& /*name*/,
const std::string& /*secondary_path*/,
DB** /*dbptr*/) {
return Status::NotSupported("Not supported in ROCKSDB_LITE.");
}
Status DB::OpenAsSecondary(
const DBOptions& /*db_options*/, const std::string& /*dbname*/,
const std::string& /*secondary_path*/,
const std::vector<ColumnFamilyDescriptor>& /*column_families*/,
std::vector<ColumnFamilyHandle*>* /*handles*/, DB** /*dbptr*/) {
return Status::NotSupported("Not supported in ROCKSDB_LITE.");
}
#endif // !ROCKSDB_LITE
} // namespace ROCKSDB_NAMESPACE