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: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886main
parent
2a5463ae84
commit
9358178edc
@ -0,0 +1,356 @@ |
|||||||
|
// 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_secondary.h" |
||||||
|
#include "db/db_iter.h" |
||||||
|
#include "db/merge_context.h" |
||||||
|
#include "monitoring/perf_context_imp.h" |
||||||
|
#include "util/auto_roll_logger.h" |
||||||
|
|
||||||
|
namespace rocksdb { |
||||||
|
|
||||||
|
#ifndef ROCKSDB_LITE |
||||||
|
|
||||||
|
DBImplSecondary::DBImplSecondary(const DBOptions& db_options, |
||||||
|
const std::string& dbname) |
||||||
|
: DBImpl(db_options, dbname) { |
||||||
|
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_log_file_exist*/, |
||||||
|
bool /*error_if_data_exists_in_logs*/) { |
||||||
|
mutex_.AssertHeld(); |
||||||
|
|
||||||
|
Status s; |
||||||
|
s = static_cast<ReactiveVersionSet*>(versions_.get()) |
||||||
|
->Recover(column_families, &manifest_reader_, &manifest_reporter_, |
||||||
|
&manifest_reader_status_); |
||||||
|
if (!s.ok()) { |
||||||
|
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; |
||||||
|
} |
||||||
|
|
||||||
|
// TODO: attempt to recover from WAL files.
|
||||||
|
return s; |
||||||
|
} |
||||||
|
|
||||||
|
// 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, env_); |
||||||
|
StopWatch sw(env_, stats_, DB_GET); |
||||||
|
PERF_TIMER_GUARD(get_snapshot_time); |
||||||
|
|
||||||
|
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; |
||||||
|
if (super_version->mem->Get(lkey, pinnable_val->GetSelf(), &s, &merge_context, |
||||||
|
&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( |
||||||
|
lkey, pinnable_val->GetSelf(), &s, &merge_context, |
||||||
|
&max_covering_tombstone_seq, read_options)) { |
||||||
|
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); |
||||||
|
super_version->current->Get(read_options, lkey, pinnable_val, &s, |
||||||
|
&merge_context, &max_covering_tombstone_seq); |
||||||
|
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.")); |
||||||
|
} |
||||||
|
Iterator* result = nullptr; |
||||||
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family); |
||||||
|
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 { |
||||||
|
auto snapshot = versions_->LastSequence(); |
||||||
|
result = NewIteratorImpl(read_options, cfd, snapshot, read_callback); |
||||||
|
} |
||||||
|
return result; |
||||||
|
} |
||||||
|
|
||||||
|
ArenaWrappedDBIter* DBImplSecondary::NewIteratorImpl( |
||||||
|
const ReadOptions& read_options, ColumnFamilyData* cfd, |
||||||
|
SequenceNumber snapshot, ReadCallback* read_callback) { |
||||||
|
assert(nullptr != cfd); |
||||||
|
SuperVersion* super_version = cfd->GetReferencedSuperVersion(&mutex_); |
||||||
|
auto db_iter = NewArenaWrappedDbIterator( |
||||||
|
env_, read_options, *cfd->ioptions(), super_version->mutable_cf_options, |
||||||
|
snapshot, |
||||||
|
super_version->mutable_cf_options.max_sequential_skip_in_iterations, |
||||||
|
super_version->version_number, read_callback); |
||||||
|
auto internal_iter = |
||||||
|
NewInternalIterator(read_options, cfd, super_version, db_iter->GetArena(), |
||||||
|
db_iter->GetRangeDelAggregator(), snapshot); |
||||||
|
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"); |
||||||
|
} |
||||||
|
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::TryCatchUpWithPrimary() { |
||||||
|
assert(versions_.get() != nullptr); |
||||||
|
assert(manifest_reader_.get() != nullptr); |
||||||
|
Status s; |
||||||
|
std::unordered_set<ColumnFamilyData*> cfds_changed; |
||||||
|
InstrumentedMutexLock lock_guard(&mutex_); |
||||||
|
s = static_cast<ReactiveVersionSet*>(versions_.get()) |
||||||
|
->ReadAndApply(&mutex_, &manifest_reader_, &cfds_changed); |
||||||
|
if (s.ok()) { |
||||||
|
SuperVersionContext sv_context(true /* create_superversion */); |
||||||
|
for (auto cfd : cfds_changed) { |
||||||
|
sv_context.NewSuperVersion(); |
||||||
|
cfd->InstallSuperVersion(&sv_context, &mutex_); |
||||||
|
} |
||||||
|
sv_context.Clean(); |
||||||
|
} |
||||||
|
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); |
||||||
|
if (nullptr == tmp_opts.info_log) { |
||||||
|
Env* env = tmp_opts.env; |
||||||
|
assert(env != nullptr); |
||||||
|
std::string secondary_abs_path; |
||||||
|
env->GetAbsolutePath(secondary_path, &secondary_abs_path); |
||||||
|
std::string fname = InfoLogFileName(secondary_path, secondary_abs_path, |
||||||
|
tmp_opts.db_log_dir); |
||||||
|
|
||||||
|
env->CreateDirIfMissing(secondary_path); |
||||||
|
if (tmp_opts.log_file_time_to_roll > 0 || tmp_opts.max_log_file_size > 0) { |
||||||
|
AutoRollLogger* result = new AutoRollLogger( |
||||||
|
env, secondary_path, tmp_opts.db_log_dir, tmp_opts.max_log_file_size, |
||||||
|
tmp_opts.log_file_time_to_roll, tmp_opts.info_log_level); |
||||||
|
Status s = result->GetStatus(); |
||||||
|
if (!s.ok()) { |
||||||
|
delete result; |
||||||
|
} else { |
||||||
|
tmp_opts.info_log.reset(result); |
||||||
|
} |
||||||
|
} |
||||||
|
if (nullptr == tmp_opts.info_log) { |
||||||
|
env->RenameFile( |
||||||
|
fname, OldInfoLogFileName(secondary_path, env->NowMicros(), |
||||||
|
secondary_abs_path, tmp_opts.db_log_dir)); |
||||||
|
Status s = env->NewLogger(fname, &(tmp_opts.info_log)); |
||||||
|
if (tmp_opts.info_log != nullptr) { |
||||||
|
tmp_opts.info_log->SetInfoLogLevel(tmp_opts.info_log_level); |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
assert(tmp_opts.info_log != nullptr); |
||||||
|
|
||||||
|
handles->clear(); |
||||||
|
DBImplSecondary* impl = new DBImplSecondary(tmp_opts, dbname); |
||||||
|
impl->versions_.reset(new ReactiveVersionSet( |
||||||
|
dbname, &impl->immutable_db_options_, impl->env_options_, |
||||||
|
impl->table_cache_.get(), impl->write_buffer_manager_, |
||||||
|
&impl->write_controller_)); |
||||||
|
impl->column_family_memtables_.reset( |
||||||
|
new ColumnFamilyMemTablesImpl(impl->versions_->GetColumnFamilySet())); |
||||||
|
impl->mutex_.Lock(); |
||||||
|
Status s = impl->Recover(column_families, true, false, false); |
||||||
|
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); |
||||||
|
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( |
||||||
|
reinterpret_cast<ColumnFamilyHandleImpl*>(h)->cfd()); |
||||||
|
} |
||||||
|
} else { |
||||||
|
for (auto h : *handles) { |
||||||
|
delete h; |
||||||
|
} |
||||||
|
handles->clear(); |
||||||
|
delete impl; |
||||||
|
} |
||||||
|
return s; |
||||||
|
} |
||||||
|
#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
|
@ -0,0 +1,151 @@ |
|||||||
|
// 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).
|
||||||
|
|
||||||
|
#pragma once |
||||||
|
|
||||||
|
#ifndef ROCKSDB_LITE |
||||||
|
|
||||||
|
#include <string> |
||||||
|
#include <vector> |
||||||
|
#include "db/db_impl.h" |
||||||
|
|
||||||
|
namespace rocksdb { |
||||||
|
|
||||||
|
class DBImplSecondary : public DBImpl { |
||||||
|
public: |
||||||
|
DBImplSecondary(const DBOptions& options, const std::string& dbname); |
||||||
|
~DBImplSecondary() override; |
||||||
|
|
||||||
|
Status Recover(const std::vector<ColumnFamilyDescriptor>& column_families, |
||||||
|
bool read_only, bool error_if_log_file_exist, |
||||||
|
bool error_if_data_exists_in_logs) override; |
||||||
|
|
||||||
|
// Implementations of the DB interface
|
||||||
|
using DB::Get; |
||||||
|
Status Get(const ReadOptions& options, ColumnFamilyHandle* column_family, |
||||||
|
const Slice& key, PinnableSlice* value) override; |
||||||
|
|
||||||
|
Status GetImpl(const ReadOptions& options, ColumnFamilyHandle* column_family, |
||||||
|
const Slice& key, PinnableSlice* value); |
||||||
|
|
||||||
|
using DBImpl::NewIterator; |
||||||
|
Iterator* NewIterator(const ReadOptions&, |
||||||
|
ColumnFamilyHandle* column_family) override; |
||||||
|
|
||||||
|
ArenaWrappedDBIter* NewIteratorImpl(const ReadOptions& read_options, |
||||||
|
ColumnFamilyData* cfd, |
||||||
|
SequenceNumber snapshot, |
||||||
|
ReadCallback* read_callback); |
||||||
|
|
||||||
|
Status NewIterators(const ReadOptions& options, |
||||||
|
const std::vector<ColumnFamilyHandle*>& column_families, |
||||||
|
std::vector<Iterator*>* iterators) override; |
||||||
|
|
||||||
|
using DBImpl::Put; |
||||||
|
Status Put(const WriteOptions& /*options*/, |
||||||
|
ColumnFamilyHandle* /*column_family*/, const Slice& /*key*/, |
||||||
|
const Slice& /*value*/) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
using DBImpl::Merge; |
||||||
|
Status Merge(const WriteOptions& /*options*/, |
||||||
|
ColumnFamilyHandle* /*column_family*/, const Slice& /*key*/, |
||||||
|
const Slice& /*value*/) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
using DBImpl::Delete; |
||||||
|
Status Delete(const WriteOptions& /*options*/, |
||||||
|
ColumnFamilyHandle* /*column_family*/, |
||||||
|
const Slice& /*key*/) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
using DBImpl::SingleDelete; |
||||||
|
Status SingleDelete(const WriteOptions& /*options*/, |
||||||
|
ColumnFamilyHandle* /*column_family*/, |
||||||
|
const Slice& /*key*/) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
Status Write(const WriteOptions& /*options*/, |
||||||
|
WriteBatch* /*updates*/) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
using DBImpl::CompactRange; |
||||||
|
Status CompactRange(const CompactRangeOptions& /*options*/, |
||||||
|
ColumnFamilyHandle* /*column_family*/, |
||||||
|
const Slice* /*begin*/, const Slice* /*end*/) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
using DBImpl::CompactFiles; |
||||||
|
Status CompactFiles( |
||||||
|
const CompactionOptions& /*compact_options*/, |
||||||
|
ColumnFamilyHandle* /*column_family*/, |
||||||
|
const std::vector<std::string>& /*input_file_names*/, |
||||||
|
const int /*output_level*/, const int /*output_path_id*/ = -1, |
||||||
|
std::vector<std::string>* const /*output_file_names*/ = nullptr, |
||||||
|
CompactionJobInfo* /*compaction_job_info*/ = nullptr) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
Status DisableFileDeletions() override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
Status EnableFileDeletions(bool /*force*/) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
Status GetLiveFiles(std::vector<std::string>&, |
||||||
|
uint64_t* /*manifest_file_size*/, |
||||||
|
bool /*flush_memtable*/ = true) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
using DBImpl::Flush; |
||||||
|
Status Flush(const FlushOptions& /*options*/, |
||||||
|
ColumnFamilyHandle* /*column_family*/) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
using DBImpl::SyncWAL; |
||||||
|
Status SyncWAL() override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
using DB::IngestExternalFile; |
||||||
|
Status IngestExternalFile( |
||||||
|
ColumnFamilyHandle* /*column_family*/, |
||||||
|
const std::vector<std::string>& /*external_files*/, |
||||||
|
const IngestExternalFileOptions& /*ingestion_options*/) override { |
||||||
|
return Status::NotSupported("Not supported operation in read only mode."); |
||||||
|
} |
||||||
|
|
||||||
|
// Try to catch up with the primary by reading as much as possible from the
|
||||||
|
// log files until there is nothing more to read or encounters an error. If
|
||||||
|
// the amount of information in the log files to process is huge, this
|
||||||
|
// method can take long time due to all the I/O and CPU costs.
|
||||||
|
Status TryCatchUpWithPrimary() override; |
||||||
|
|
||||||
|
private: |
||||||
|
friend class DB; |
||||||
|
|
||||||
|
// No copying allowed
|
||||||
|
DBImplSecondary(const DBImplSecondary&); |
||||||
|
void operator=(const DBImplSecondary&); |
||||||
|
|
||||||
|
using DBImpl::Recover; |
||||||
|
|
||||||
|
std::unique_ptr<log::FragmentBufferedReader> manifest_reader_; |
||||||
|
std::unique_ptr<log::Reader::Reporter> manifest_reporter_; |
||||||
|
std::unique_ptr<Status> manifest_reader_status_; |
||||||
|
}; |
||||||
|
} // namespace rocksdb
|
||||||
|
|
||||||
|
#endif // !ROCKSDB_LITE
|
@ -0,0 +1,480 @@ |
|||||||
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
||||||
|
// This source code is licensed under both the GPLv2 (found in the
|
||||||
|
// COPYING file in the root directory) and Apache 2.0 License
|
||||||
|
// (found in the LICENSE.Apache file in the root directory).
|
||||||
|
//
|
||||||
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||||
|
// Use of this source code is governed by a BSD-style license that can be
|
||||||
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||||
|
|
||||||
|
#include "db/db_impl_secondary.h" |
||||||
|
#include "db/db_test_util.h" |
||||||
|
#include "port/stack_trace.h" |
||||||
|
#include "util/fault_injection_test_env.h" |
||||||
|
#include "util/sync_point.h" |
||||||
|
|
||||||
|
namespace rocksdb { |
||||||
|
|
||||||
|
#ifndef ROCKSDB_LITE |
||||||
|
class DBSecondaryTest : public DBTestBase { |
||||||
|
public: |
||||||
|
DBSecondaryTest() |
||||||
|
: DBTestBase("/db_secondary_test"), |
||||||
|
secondary_path_(), |
||||||
|
handles_secondary_(), |
||||||
|
db_secondary_(nullptr) { |
||||||
|
secondary_path_ = |
||||||
|
test::PerThreadDBPath(env_, "/db_secondary_test_secondary"); |
||||||
|
} |
||||||
|
|
||||||
|
~DBSecondaryTest() override { |
||||||
|
CloseSecondary(); |
||||||
|
if (getenv("KEEP_DB") != nullptr) { |
||||||
|
fprintf(stdout, "Secondary DB is still at %s\n", secondary_path_.c_str()); |
||||||
|
} else { |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
EXPECT_OK(DestroyDB(secondary_path_, options)); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
protected: |
||||||
|
Status ReopenAsSecondary(const Options& options) { |
||||||
|
return DB::OpenAsSecondary(options, dbname_, secondary_path_, &db_); |
||||||
|
} |
||||||
|
|
||||||
|
void OpenSecondary(const Options& options); |
||||||
|
|
||||||
|
void OpenSecondaryWithColumnFamilies( |
||||||
|
const std::vector<std::string>& column_families, const Options& options); |
||||||
|
|
||||||
|
void CloseSecondary() { |
||||||
|
for (auto h : handles_secondary_) { |
||||||
|
db_secondary_->DestroyColumnFamilyHandle(h); |
||||||
|
} |
||||||
|
handles_secondary_.clear(); |
||||||
|
delete db_secondary_; |
||||||
|
db_secondary_ = nullptr; |
||||||
|
} |
||||||
|
|
||||||
|
DBImplSecondary* db_secondary_full() { |
||||||
|
return static_cast<DBImplSecondary*>(db_secondary_); |
||||||
|
} |
||||||
|
|
||||||
|
void CheckFileTypeCounts(const std::string& dir, int expected_log, |
||||||
|
int expected_sst, int expected_manifest) const; |
||||||
|
|
||||||
|
std::string secondary_path_; |
||||||
|
std::vector<ColumnFamilyHandle*> handles_secondary_; |
||||||
|
DB* db_secondary_; |
||||||
|
}; |
||||||
|
|
||||||
|
void DBSecondaryTest::OpenSecondary(const Options& options) { |
||||||
|
Status s = |
||||||
|
DB::OpenAsSecondary(options, dbname_, secondary_path_, &db_secondary_); |
||||||
|
ASSERT_OK(s); |
||||||
|
} |
||||||
|
|
||||||
|
void DBSecondaryTest::OpenSecondaryWithColumnFamilies( |
||||||
|
const std::vector<std::string>& column_families, const Options& options) { |
||||||
|
std::vector<ColumnFamilyDescriptor> cf_descs; |
||||||
|
cf_descs.emplace_back(kDefaultColumnFamilyName, options); |
||||||
|
for (const auto& cf_name : column_families) { |
||||||
|
cf_descs.emplace_back(cf_name, options); |
||||||
|
} |
||||||
|
Status s = DB::OpenAsSecondary(options, dbname_, secondary_path_, cf_descs, |
||||||
|
&handles_secondary_, &db_secondary_); |
||||||
|
ASSERT_OK(s); |
||||||
|
} |
||||||
|
|
||||||
|
void DBSecondaryTest::CheckFileTypeCounts(const std::string& dir, |
||||||
|
int expected_log, int expected_sst, |
||||||
|
int expected_manifest) const { |
||||||
|
std::vector<std::string> filenames; |
||||||
|
env_->GetChildren(dir, &filenames); |
||||||
|
|
||||||
|
int log_cnt = 0, sst_cnt = 0, manifest_cnt = 0; |
||||||
|
for (auto file : filenames) { |
||||||
|
uint64_t number; |
||||||
|
FileType type; |
||||||
|
if (ParseFileName(file, &number, &type)) { |
||||||
|
log_cnt += (type == kLogFile); |
||||||
|
sst_cnt += (type == kTableFile); |
||||||
|
manifest_cnt += (type == kDescriptorFile); |
||||||
|
} |
||||||
|
} |
||||||
|
ASSERT_EQ(expected_log, log_cnt); |
||||||
|
ASSERT_EQ(expected_sst, sst_cnt); |
||||||
|
ASSERT_EQ(expected_manifest, manifest_cnt); |
||||||
|
} |
||||||
|
|
||||||
|
TEST_F(DBSecondaryTest, ReopenAsSecondary) { |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
Reopen(options); |
||||||
|
ASSERT_OK(Put("foo", "foo_value")); |
||||||
|
ASSERT_OK(Put("bar", "bar_value")); |
||||||
|
ASSERT_OK(dbfull()->Flush(FlushOptions())); |
||||||
|
Close(); |
||||||
|
|
||||||
|
ASSERT_OK(ReopenAsSecondary(options)); |
||||||
|
ASSERT_EQ("foo_value", Get("foo")); |
||||||
|
ASSERT_EQ("bar_value", Get("bar")); |
||||||
|
ReadOptions ropts; |
||||||
|
ropts.verify_checksums = true; |
||||||
|
auto db1 = static_cast<DBImplSecondary*>(db_); |
||||||
|
ASSERT_NE(nullptr, db1); |
||||||
|
Iterator* iter = db1->NewIterator(ropts); |
||||||
|
ASSERT_NE(nullptr, iter); |
||||||
|
size_t count = 0; |
||||||
|
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { |
||||||
|
if (0 == count) { |
||||||
|
ASSERT_EQ("bar", iter->key().ToString()); |
||||||
|
ASSERT_EQ("bar_value", iter->value().ToString()); |
||||||
|
} else if (1 == count) { |
||||||
|
ASSERT_EQ("foo", iter->key().ToString()); |
||||||
|
ASSERT_EQ("foo_value", iter->value().ToString()); |
||||||
|
} |
||||||
|
++count; |
||||||
|
} |
||||||
|
delete iter; |
||||||
|
ASSERT_EQ(2, count); |
||||||
|
} |
||||||
|
|
||||||
|
TEST_F(DBSecondaryTest, OpenAsSecondary) { |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
options.level0_file_num_compaction_trigger = 4; |
||||||
|
Reopen(options); |
||||||
|
for (int i = 0; i < 3; ++i) { |
||||||
|
ASSERT_OK(Put("foo", "foo_value" + std::to_string(i))); |
||||||
|
ASSERT_OK(Put("bar", "bar_value" + std::to_string(i))); |
||||||
|
ASSERT_OK(Flush()); |
||||||
|
} |
||||||
|
Options options1; |
||||||
|
options1.env = env_; |
||||||
|
options1.max_open_files = -1; |
||||||
|
OpenSecondary(options1); |
||||||
|
ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr)); |
||||||
|
ASSERT_OK(dbfull()->TEST_WaitForCompact()); |
||||||
|
|
||||||
|
ReadOptions ropts; |
||||||
|
ropts.verify_checksums = true; |
||||||
|
const auto verify_db_func = [&](const std::string& foo_val, |
||||||
|
const std::string& bar_val) { |
||||||
|
std::string value; |
||||||
|
ASSERT_OK(db_secondary_->Get(ropts, "foo", &value)); |
||||||
|
ASSERT_EQ(foo_val, value); |
||||||
|
ASSERT_OK(db_secondary_->Get(ropts, "bar", &value)); |
||||||
|
ASSERT_EQ(bar_val, value); |
||||||
|
Iterator* iter = db_secondary_->NewIterator(ropts); |
||||||
|
ASSERT_NE(nullptr, iter); |
||||||
|
iter->Seek("foo"); |
||||||
|
ASSERT_TRUE(iter->Valid()); |
||||||
|
ASSERT_EQ("foo", iter->key().ToString()); |
||||||
|
ASSERT_EQ(foo_val, iter->value().ToString()); |
||||||
|
iter->Seek("bar"); |
||||||
|
ASSERT_TRUE(iter->Valid()); |
||||||
|
ASSERT_EQ("bar", iter->key().ToString()); |
||||||
|
ASSERT_EQ(bar_val, iter->value().ToString()); |
||||||
|
size_t count = 0; |
||||||
|
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { |
||||||
|
++count; |
||||||
|
} |
||||||
|
ASSERT_EQ(2, count); |
||||||
|
delete iter; |
||||||
|
}; |
||||||
|
|
||||||
|
verify_db_func("foo_value2", "bar_value2"); |
||||||
|
|
||||||
|
ASSERT_OK(Put("foo", "new_foo_value")); |
||||||
|
ASSERT_OK(Put("bar", "new_bar_value")); |
||||||
|
ASSERT_OK(Flush()); |
||||||
|
|
||||||
|
ASSERT_OK(db_secondary_->TryCatchUpWithPrimary()); |
||||||
|
verify_db_func("new_foo_value", "new_bar_value"); |
||||||
|
} |
||||||
|
|
||||||
|
TEST_F(DBSecondaryTest, OpenWithNonExistColumnFamily) { |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
CreateAndReopenWithCF({"pikachu"}, options); |
||||||
|
|
||||||
|
Options options1; |
||||||
|
options1.env = env_; |
||||||
|
options1.max_open_files = -1; |
||||||
|
std::vector<ColumnFamilyDescriptor> cf_descs; |
||||||
|
cf_descs.emplace_back(kDefaultColumnFamilyName, options1); |
||||||
|
cf_descs.emplace_back("pikachu", options1); |
||||||
|
cf_descs.emplace_back("eevee", options1); |
||||||
|
Status s = DB::OpenAsSecondary(options1, dbname_, secondary_path_, cf_descs, |
||||||
|
&handles_secondary_, &db_secondary_); |
||||||
|
ASSERT_NOK(s); |
||||||
|
} |
||||||
|
|
||||||
|
TEST_F(DBSecondaryTest, OpenWithSubsetOfColumnFamilies) { |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
CreateAndReopenWithCF({"pikachu"}, options); |
||||||
|
Options options1; |
||||||
|
options1.env = env_; |
||||||
|
options1.max_open_files = -1; |
||||||
|
OpenSecondary(options1); |
||||||
|
ASSERT_EQ(0, handles_secondary_.size()); |
||||||
|
ASSERT_NE(nullptr, db_secondary_); |
||||||
|
|
||||||
|
ASSERT_OK(Put(0 /*cf*/, "foo", "foo_value")); |
||||||
|
ASSERT_OK(Put(1 /*cf*/, "foo", "foo_value")); |
||||||
|
ASSERT_OK(Flush(0 /*cf*/)); |
||||||
|
ASSERT_OK(Flush(1 /*cf*/)); |
||||||
|
ASSERT_OK(db_secondary_->TryCatchUpWithPrimary()); |
||||||
|
ReadOptions ropts; |
||||||
|
ropts.verify_checksums = true; |
||||||
|
std::string value; |
||||||
|
ASSERT_OK(db_secondary_->Get(ropts, "foo", &value)); |
||||||
|
ASSERT_EQ("foo_value", value); |
||||||
|
} |
||||||
|
|
||||||
|
TEST_F(DBSecondaryTest, SwitchToNewManifestDuringOpen) { |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
Reopen(options); |
||||||
|
Close(); |
||||||
|
|
||||||
|
SyncPoint::GetInstance()->DisableProcessing(); |
||||||
|
SyncPoint::GetInstance()->ClearAllCallBacks(); |
||||||
|
SyncPoint::GetInstance()->LoadDependency( |
||||||
|
{{"ReactiveVersionSet::MaybeSwitchManifest:AfterGetCurrentManifestPath:0", |
||||||
|
"VersionSet::ProcessManifestWrites:BeforeNewManifest"}, |
||||||
|
{"VersionSet::ProcessManifestWrites:AfterNewManifest", |
||||||
|
"ReactiveVersionSet::MaybeSwitchManifest:AfterGetCurrentManifestPath:" |
||||||
|
"1"}}); |
||||||
|
SyncPoint::GetInstance()->EnableProcessing(); |
||||||
|
|
||||||
|
// Make sure db calls RecoverLogFiles so as to trigger a manifest write,
|
||||||
|
// which causes the db to switch to a new MANIFEST upon start.
|
||||||
|
port::Thread ro_db_thread([&]() { |
||||||
|
Options options1; |
||||||
|
options1.env = env_; |
||||||
|
options1.max_open_files = -1; |
||||||
|
OpenSecondary(options1); |
||||||
|
CloseSecondary(); |
||||||
|
}); |
||||||
|
Reopen(options); |
||||||
|
ro_db_thread.join(); |
||||||
|
} |
||||||
|
|
||||||
|
TEST_F(DBSecondaryTest, MissingTableFileDuringOpen) { |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
options.level0_file_num_compaction_trigger = 4; |
||||||
|
Reopen(options); |
||||||
|
for (int i = 0; i != options.level0_file_num_compaction_trigger; ++i) { |
||||||
|
ASSERT_OK(Put("foo", "foo_value" + std::to_string(i))); |
||||||
|
ASSERT_OK(Put("bar", "bar_value" + std::to_string(i))); |
||||||
|
ASSERT_OK(dbfull()->Flush(FlushOptions())); |
||||||
|
} |
||||||
|
ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable()); |
||||||
|
ASSERT_OK(dbfull()->TEST_WaitForCompact()); |
||||||
|
Options options1; |
||||||
|
options1.env = env_; |
||||||
|
options1.max_open_files = -1; |
||||||
|
OpenSecondary(options1); |
||||||
|
ReadOptions ropts; |
||||||
|
ropts.verify_checksums = true; |
||||||
|
std::string value; |
||||||
|
ASSERT_OK(db_secondary_->Get(ropts, "foo", &value)); |
||||||
|
ASSERT_EQ("foo_value" + |
||||||
|
std::to_string(options.level0_file_num_compaction_trigger - 1), |
||||||
|
value); |
||||||
|
ASSERT_OK(db_secondary_->Get(ropts, "bar", &value)); |
||||||
|
ASSERT_EQ("bar_value" + |
||||||
|
std::to_string(options.level0_file_num_compaction_trigger - 1), |
||||||
|
value); |
||||||
|
Iterator* iter = db_secondary_->NewIterator(ropts); |
||||||
|
ASSERT_NE(nullptr, iter); |
||||||
|
iter->Seek("bar"); |
||||||
|
ASSERT_TRUE(iter->Valid()); |
||||||
|
ASSERT_EQ("bar", iter->key().ToString()); |
||||||
|
ASSERT_EQ("bar_value" + |
||||||
|
std::to_string(options.level0_file_num_compaction_trigger - 1), |
||||||
|
iter->value().ToString()); |
||||||
|
iter->Seek("foo"); |
||||||
|
ASSERT_TRUE(iter->Valid()); |
||||||
|
ASSERT_EQ("foo", iter->key().ToString()); |
||||||
|
ASSERT_EQ("foo_value" + |
||||||
|
std::to_string(options.level0_file_num_compaction_trigger - 1), |
||||||
|
iter->value().ToString()); |
||||||
|
size_t count = 0; |
||||||
|
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { |
||||||
|
++count; |
||||||
|
} |
||||||
|
ASSERT_EQ(2, count); |
||||||
|
delete iter; |
||||||
|
} |
||||||
|
|
||||||
|
TEST_F(DBSecondaryTest, MissingTableFile) { |
||||||
|
int table_files_not_exist = 0; |
||||||
|
SyncPoint::GetInstance()->DisableProcessing(); |
||||||
|
SyncPoint::GetInstance()->ClearAllCallBacks(); |
||||||
|
SyncPoint::GetInstance()->SetCallBack( |
||||||
|
"ReactiveVersionSet::ReadAndApply:AfterLoadTableHandlers", |
||||||
|
[&](void* arg) { |
||||||
|
Status s = *reinterpret_cast<Status*>(arg); |
||||||
|
if (s.IsPathNotFound()) { |
||||||
|
++table_files_not_exist; |
||||||
|
} else if (!s.ok()) { |
||||||
|
assert(false); // Should not reach here
|
||||||
|
} |
||||||
|
}); |
||||||
|
SyncPoint::GetInstance()->EnableProcessing(); |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
options.level0_file_num_compaction_trigger = 4; |
||||||
|
Reopen(options); |
||||||
|
|
||||||
|
Options options1; |
||||||
|
options1.env = env_; |
||||||
|
options1.max_open_files = -1; |
||||||
|
OpenSecondary(options1); |
||||||
|
|
||||||
|
for (int i = 0; i != options.level0_file_num_compaction_trigger; ++i) { |
||||||
|
ASSERT_OK(Put("foo", "foo_value" + std::to_string(i))); |
||||||
|
ASSERT_OK(Put("bar", "bar_value" + std::to_string(i))); |
||||||
|
ASSERT_OK(dbfull()->Flush(FlushOptions())); |
||||||
|
} |
||||||
|
ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable()); |
||||||
|
ASSERT_OK(dbfull()->TEST_WaitForCompact()); |
||||||
|
|
||||||
|
ASSERT_NE(nullptr, db_secondary_full()); |
||||||
|
ReadOptions ropts; |
||||||
|
ropts.verify_checksums = true; |
||||||
|
std::string value; |
||||||
|
ASSERT_NOK(db_secondary_->Get(ropts, "foo", &value)); |
||||||
|
ASSERT_NOK(db_secondary_->Get(ropts, "bar", &value)); |
||||||
|
|
||||||
|
ASSERT_OK(db_secondary_->TryCatchUpWithPrimary()); |
||||||
|
ASSERT_EQ(options.level0_file_num_compaction_trigger, table_files_not_exist); |
||||||
|
ASSERT_OK(db_secondary_->Get(ropts, "foo", &value)); |
||||||
|
ASSERT_EQ("foo_value" + |
||||||
|
std::to_string(options.level0_file_num_compaction_trigger - 1), |
||||||
|
value); |
||||||
|
ASSERT_OK(db_secondary_->Get(ropts, "bar", &value)); |
||||||
|
ASSERT_EQ("bar_value" + |
||||||
|
std::to_string(options.level0_file_num_compaction_trigger - 1), |
||||||
|
value); |
||||||
|
Iterator* iter = db_secondary_->NewIterator(ropts); |
||||||
|
ASSERT_NE(nullptr, iter); |
||||||
|
iter->Seek("bar"); |
||||||
|
ASSERT_TRUE(iter->Valid()); |
||||||
|
ASSERT_EQ("bar", iter->key().ToString()); |
||||||
|
ASSERT_EQ("bar_value" + |
||||||
|
std::to_string(options.level0_file_num_compaction_trigger - 1), |
||||||
|
iter->value().ToString()); |
||||||
|
iter->Seek("foo"); |
||||||
|
ASSERT_TRUE(iter->Valid()); |
||||||
|
ASSERT_EQ("foo", iter->key().ToString()); |
||||||
|
ASSERT_EQ("foo_value" + |
||||||
|
std::to_string(options.level0_file_num_compaction_trigger - 1), |
||||||
|
iter->value().ToString()); |
||||||
|
size_t count = 0; |
||||||
|
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { |
||||||
|
++count; |
||||||
|
} |
||||||
|
ASSERT_EQ(2, count); |
||||||
|
delete iter; |
||||||
|
} |
||||||
|
|
||||||
|
TEST_F(DBSecondaryTest, PrimaryDropColumnFamily) { |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
const std::string kCfName1 = "pikachu"; |
||||||
|
CreateAndReopenWithCF({kCfName1}, options); |
||||||
|
|
||||||
|
Options options1; |
||||||
|
options1.env = env_; |
||||||
|
options1.max_open_files = -1; |
||||||
|
OpenSecondaryWithColumnFamilies({kCfName1}, options1); |
||||||
|
ASSERT_EQ(2, handles_secondary_.size()); |
||||||
|
|
||||||
|
ASSERT_OK(Put(1 /*cf*/, "foo", "foo_val_1")); |
||||||
|
ASSERT_OK(Flush(1 /*cf*/)); |
||||||
|
|
||||||
|
ASSERT_OK(db_secondary_->TryCatchUpWithPrimary()); |
||||||
|
ReadOptions ropts; |
||||||
|
ropts.verify_checksums = true; |
||||||
|
std::string value; |
||||||
|
ASSERT_OK(db_secondary_->Get(ropts, handles_secondary_[1], "foo", &value)); |
||||||
|
ASSERT_EQ("foo_val_1", value); |
||||||
|
|
||||||
|
ASSERT_OK(dbfull()->DropColumnFamily(handles_[1])); |
||||||
|
Close(); |
||||||
|
CheckFileTypeCounts(dbname_, 1, 0, 1); |
||||||
|
ASSERT_OK(db_secondary_->TryCatchUpWithPrimary()); |
||||||
|
value.clear(); |
||||||
|
ASSERT_OK(db_secondary_->Get(ropts, handles_secondary_[1], "foo", &value)); |
||||||
|
ASSERT_EQ("foo_val_1", value); |
||||||
|
} |
||||||
|
|
||||||
|
TEST_F(DBSecondaryTest, SwitchManifest) { |
||||||
|
Options options; |
||||||
|
options.env = env_; |
||||||
|
options.level0_file_num_compaction_trigger = 4; |
||||||
|
Reopen(options); |
||||||
|
|
||||||
|
Options options1; |
||||||
|
options1.env = env_; |
||||||
|
options1.max_open_files = -1; |
||||||
|
OpenSecondary(options1); |
||||||
|
|
||||||
|
const int kNumFiles = options.level0_file_num_compaction_trigger - 1; |
||||||
|
// Keep it smaller than 10 so that key0, key1, ..., key9 are sorted as 0, 1,
|
||||||
|
// ..., 9.
|
||||||
|
const int kNumKeys = 10; |
||||||
|
// Create two sst
|
||||||
|
for (int i = 0; i != kNumFiles; ++i) { |
||||||
|
for (int j = 0; j != kNumKeys; ++j) { |
||||||
|
ASSERT_OK(Put("key" + std::to_string(j), "value_" + std::to_string(i))); |
||||||
|
} |
||||||
|
ASSERT_OK(Flush()); |
||||||
|
} |
||||||
|
|
||||||
|
ASSERT_OK(db_secondary_->TryCatchUpWithPrimary()); |
||||||
|
const auto& range_scan_db = [&]() { |
||||||
|
ReadOptions tmp_ropts; |
||||||
|
tmp_ropts.total_order_seek = true; |
||||||
|
tmp_ropts.verify_checksums = true; |
||||||
|
std::unique_ptr<Iterator> iter(db_secondary_->NewIterator(tmp_ropts)); |
||||||
|
int cnt = 0; |
||||||
|
for (iter->SeekToFirst(); iter->Valid(); iter->Next(), ++cnt) { |
||||||
|
ASSERT_EQ("key" + std::to_string(cnt), iter->key().ToString()); |
||||||
|
ASSERT_EQ("value_" + std::to_string(kNumFiles - 1), |
||||||
|
iter->value().ToString()); |
||||||
|
} |
||||||
|
}; |
||||||
|
|
||||||
|
range_scan_db(); |
||||||
|
|
||||||
|
// While secondary instance still keeps old MANIFEST open, we close primary,
|
||||||
|
// restart primary, performs full compaction, close again, restart again so
|
||||||
|
// that next time secondary tries to catch up with primary, the secondary
|
||||||
|
// will skip the MANIFEST in middle.
|
||||||
|
Reopen(options); |
||||||
|
ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr)); |
||||||
|
ASSERT_OK(dbfull()->TEST_WaitForCompact()); |
||||||
|
|
||||||
|
Reopen(options); |
||||||
|
ASSERT_OK(dbfull()->SetOptions({{"disable_auto_compactions", "false"}})); |
||||||
|
|
||||||
|
ASSERT_OK(db_secondary_->TryCatchUpWithPrimary()); |
||||||
|
range_scan_db(); |
||||||
|
} |
||||||
|
#endif //! ROCKSDB_LITE
|
||||||
|
|
||||||
|
} // namespace rocksdb
|
||||||
|
|
||||||
|
int main(int argc, char** argv) { |
||||||
|
rocksdb::port::InstallStackTraceHandler(); |
||||||
|
::testing::InitGoogleTest(&argc, argv); |
||||||
|
return RUN_ALL_TESTS(); |
||||||
|
} |
@ -0,0 +1,395 @@ |
|||||||
|
// 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).
|
||||||
|
|
||||||
|
// How to use this example
|
||||||
|
// Open two terminals, in one of them, run `./multi_processes_example 0` to
|
||||||
|
// start a process running the primary instance. This will create a new DB in
|
||||||
|
// kDBPath. The process will run for a while inserting keys to the normal
|
||||||
|
// RocksDB database.
|
||||||
|
// Next, go to the other terminal and run `./multi_processes_example 1` to
|
||||||
|
// start a process running the secondary instance. This will create a secondary
|
||||||
|
// instance following the aforementioned primary instance. This process will
|
||||||
|
// run for a while, tailing the logs of the primary. After process with primary
|
||||||
|
// instance exits, this process will keep running until you hit 'CTRL+C'.
|
||||||
|
|
||||||
|
#include <inttypes.h> |
||||||
|
#include <chrono> |
||||||
|
#include <cstdio> |
||||||
|
#include <cstdlib> |
||||||
|
#include <ctime> |
||||||
|
#include <string> |
||||||
|
#include <thread> |
||||||
|
#include <vector> |
||||||
|
|
||||||
|
#if defined(OS_LINUX) |
||||||
|
#include <dirent.h> |
||||||
|
#include <signal.h> |
||||||
|
#include <sys/stat.h> |
||||||
|
#include <sys/types.h> |
||||||
|
#include <sys/wait.h> |
||||||
|
#include <unistd.h> |
||||||
|
#endif // !OS_LINUX
|
||||||
|
|
||||||
|
#include "rocksdb/db.h" |
||||||
|
#include "rocksdb/options.h" |
||||||
|
#include "rocksdb/slice.h" |
||||||
|
|
||||||
|
using rocksdb::ColumnFamilyDescriptor; |
||||||
|
using rocksdb::ColumnFamilyHandle; |
||||||
|
using rocksdb::ColumnFamilyOptions; |
||||||
|
using rocksdb::DB; |
||||||
|
using rocksdb::FlushOptions; |
||||||
|
using rocksdb::Iterator; |
||||||
|
using rocksdb::Options; |
||||||
|
using rocksdb::ReadOptions; |
||||||
|
using rocksdb::Slice; |
||||||
|
using rocksdb::Status; |
||||||
|
using rocksdb::WriteOptions; |
||||||
|
|
||||||
|
const std::string kDBPath = "/tmp/rocksdb_multi_processes_example"; |
||||||
|
const std::string kPrimaryStatusFile = |
||||||
|
"/tmp/rocksdb_multi_processes_example_primary_status"; |
||||||
|
const uint64_t kMaxKey = 600000; |
||||||
|
const size_t kMaxValueLength = 256; |
||||||
|
const size_t kNumKeysPerFlush = 1000; |
||||||
|
|
||||||
|
const std::vector<std::string>& GetColumnFamilyNames() { |
||||||
|
static std::vector<std::string> column_family_names = { |
||||||
|
rocksdb::kDefaultColumnFamilyName, "pikachu"}; |
||||||
|
return column_family_names; |
||||||
|
} |
||||||
|
|
||||||
|
inline bool IsLittleEndian() { |
||||||
|
uint32_t x = 1; |
||||||
|
return *reinterpret_cast<char*>(&x) != 0; |
||||||
|
} |
||||||
|
|
||||||
|
static std::atomic<int>& ShouldSecondaryWait() { |
||||||
|
static std::atomic<int> should_secondary_wait{1}; |
||||||
|
return should_secondary_wait; |
||||||
|
} |
||||||
|
|
||||||
|
static std::string Key(uint64_t k) { |
||||||
|
std::string ret; |
||||||
|
if (IsLittleEndian()) { |
||||||
|
ret.append(reinterpret_cast<char*>(&k), sizeof(k)); |
||||||
|
} else { |
||||||
|
char buf[sizeof(k)]; |
||||||
|
buf[0] = k & 0xff; |
||||||
|
buf[1] = (k >> 8) & 0xff; |
||||||
|
buf[2] = (k >> 16) & 0xff; |
||||||
|
buf[3] = (k >> 24) & 0xff; |
||||||
|
buf[4] = (k >> 32) & 0xff; |
||||||
|
buf[5] = (k >> 40) & 0xff; |
||||||
|
buf[6] = (k >> 48) & 0xff; |
||||||
|
buf[7] = (k >> 56) & 0xff; |
||||||
|
ret.append(buf, sizeof(k)); |
||||||
|
} |
||||||
|
size_t i = 0, j = ret.size() - 1; |
||||||
|
while (i < j) { |
||||||
|
char tmp = ret[i]; |
||||||
|
ret[i] = ret[j]; |
||||||
|
ret[j] = tmp; |
||||||
|
++i; |
||||||
|
--j; |
||||||
|
} |
||||||
|
return ret; |
||||||
|
} |
||||||
|
|
||||||
|
static uint64_t Key(std::string key) { |
||||||
|
assert(key.size() == sizeof(uint64_t)); |
||||||
|
size_t i = 0, j = key.size() - 1; |
||||||
|
while (i < j) { |
||||||
|
char tmp = key[i]; |
||||||
|
key[i] = key[j]; |
||||||
|
key[j] = tmp; |
||||||
|
++i; |
||||||
|
--j; |
||||||
|
} |
||||||
|
uint64_t ret = 0; |
||||||
|
if (IsLittleEndian()) { |
||||||
|
memcpy(&ret, key.c_str(), sizeof(uint64_t)); |
||||||
|
} else { |
||||||
|
const char* buf = key.c_str(); |
||||||
|
ret |= static_cast<uint64_t>(buf[0]); |
||||||
|
ret |= (static_cast<uint64_t>(buf[1]) << 8); |
||||||
|
ret |= (static_cast<uint64_t>(buf[2]) << 16); |
||||||
|
ret |= (static_cast<uint64_t>(buf[3]) << 24); |
||||||
|
ret |= (static_cast<uint64_t>(buf[4]) << 32); |
||||||
|
ret |= (static_cast<uint64_t>(buf[5]) << 40); |
||||||
|
ret |= (static_cast<uint64_t>(buf[6]) << 48); |
||||||
|
ret |= (static_cast<uint64_t>(buf[7]) << 56); |
||||||
|
} |
||||||
|
return ret; |
||||||
|
} |
||||||
|
|
||||||
|
static Slice GenerateRandomValue(const size_t max_length, char scratch[]) { |
||||||
|
size_t sz = 1 + (std::rand() % max_length); |
||||||
|
int rnd = std::rand(); |
||||||
|
for (size_t i = 0; i != sz; ++i) { |
||||||
|
scratch[i] = static_cast<char>(rnd ^ i); |
||||||
|
} |
||||||
|
return Slice(scratch, sz); |
||||||
|
} |
||||||
|
|
||||||
|
static bool ShouldCloseDB() { return true; } |
||||||
|
|
||||||
|
// TODO: port this example to other systems. It should be straightforward for
|
||||||
|
// POSIX-compliant systems.
|
||||||
|
#if defined(OS_LINUX) |
||||||
|
void CreateDB() { |
||||||
|
long my_pid = static_cast<long>(getpid()); |
||||||
|
Options options; |
||||||
|
Status s = rocksdb::DestroyDB(kDBPath, options); |
||||||
|
if (!s.ok()) { |
||||||
|
fprintf(stderr, "[process %ld] Failed to destroy DB: %s\n", my_pid, |
||||||
|
s.ToString().c_str()); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
options.create_if_missing = true; |
||||||
|
DB* db = nullptr; |
||||||
|
s = DB::Open(options, kDBPath, &db); |
||||||
|
if (!s.ok()) { |
||||||
|
fprintf(stderr, "[process %ld] Failed to open DB: %s\n", my_pid, |
||||||
|
s.ToString().c_str()); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
std::vector<ColumnFamilyHandle*> handles; |
||||||
|
ColumnFamilyOptions cf_opts(options); |
||||||
|
for (const auto& cf_name : GetColumnFamilyNames()) { |
||||||
|
if (rocksdb::kDefaultColumnFamilyName != cf_name) { |
||||||
|
ColumnFamilyHandle* handle = nullptr; |
||||||
|
s = db->CreateColumnFamily(cf_opts, cf_name, &handle); |
||||||
|
if (!s.ok()) { |
||||||
|
fprintf(stderr, "[process %ld] Failed to create CF %s: %s\n", my_pid, |
||||||
|
cf_name.c_str(), s.ToString().c_str()); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
handles.push_back(handle); |
||||||
|
} |
||||||
|
} |
||||||
|
fprintf(stdout, "[process %ld] Column families created\n", my_pid); |
||||||
|
for (auto h : handles) { |
||||||
|
delete h; |
||||||
|
} |
||||||
|
handles.clear(); |
||||||
|
delete db; |
||||||
|
} |
||||||
|
|
||||||
|
void RunPrimary() { |
||||||
|
long my_pid = static_cast<long>(getpid()); |
||||||
|
fprintf(stdout, "[process %ld] Primary instance starts\n", my_pid); |
||||||
|
CreateDB(); |
||||||
|
std::srand(time(nullptr)); |
||||||
|
DB* db = nullptr; |
||||||
|
Options options; |
||||||
|
options.create_if_missing = false; |
||||||
|
std::vector<ColumnFamilyDescriptor> column_families; |
||||||
|
for (const auto& cf_name : GetColumnFamilyNames()) { |
||||||
|
column_families.push_back(ColumnFamilyDescriptor(cf_name, options)); |
||||||
|
} |
||||||
|
std::vector<ColumnFamilyHandle*> handles; |
||||||
|
WriteOptions write_opts; |
||||||
|
char val_buf[kMaxValueLength] = {0}; |
||||||
|
uint64_t curr_key = 0; |
||||||
|
while (curr_key < kMaxKey) { |
||||||
|
Status s; |
||||||
|
if (nullptr == db) { |
||||||
|
s = DB::Open(options, kDBPath, column_families, &handles, &db); |
||||||
|
if (!s.ok()) { |
||||||
|
fprintf(stderr, "[process %ld] Failed to open DB: %s\n", my_pid, |
||||||
|
s.ToString().c_str()); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
} |
||||||
|
assert(nullptr != db); |
||||||
|
assert(handles.size() == GetColumnFamilyNames().size()); |
||||||
|
for (auto h : handles) { |
||||||
|
assert(nullptr != h); |
||||||
|
for (size_t i = 0; i != kNumKeysPerFlush; ++i) { |
||||||
|
Slice key = Key(curr_key + static_cast<uint64_t>(i)); |
||||||
|
Slice value = GenerateRandomValue(kMaxValueLength, val_buf); |
||||||
|
s = db->Put(write_opts, h, key, value); |
||||||
|
if (!s.ok()) { |
||||||
|
fprintf(stderr, "[process %ld] Failed to insert\n", my_pid); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
} |
||||||
|
s = db->Flush(FlushOptions(), h); |
||||||
|
if (!s.ok()) { |
||||||
|
fprintf(stderr, "[process %ld] Failed to flush\n", my_pid); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
} |
||||||
|
curr_key += static_cast<uint64_t>(kNumKeysPerFlush); |
||||||
|
if (ShouldCloseDB()) { |
||||||
|
for (auto h : handles) { |
||||||
|
delete h; |
||||||
|
} |
||||||
|
handles.clear(); |
||||||
|
delete db; |
||||||
|
db = nullptr; |
||||||
|
} |
||||||
|
} |
||||||
|
if (nullptr != db) { |
||||||
|
for (auto h : handles) { |
||||||
|
delete h; |
||||||
|
} |
||||||
|
handles.clear(); |
||||||
|
delete db; |
||||||
|
db = nullptr; |
||||||
|
} |
||||||
|
fprintf(stdout, "[process %ld] Finished adding keys\n", my_pid); |
||||||
|
} |
||||||
|
|
||||||
|
void secondary_instance_sigint_handler(int signal) { |
||||||
|
ShouldSecondaryWait().store(0, std::memory_order_relaxed); |
||||||
|
fprintf(stdout, "\n"); |
||||||
|
fflush(stdout); |
||||||
|
}; |
||||||
|
|
||||||
|
void RunSecondary() { |
||||||
|
::signal(SIGINT, secondary_instance_sigint_handler); |
||||||
|
long my_pid = static_cast<long>(getpid()); |
||||||
|
const std::string kSecondaryPath = |
||||||
|
"/tmp/rocksdb_multi_processes_example_secondary"; |
||||||
|
// Create directory if necessary
|
||||||
|
if (nullptr == opendir(kSecondaryPath.c_str())) { |
||||||
|
int ret = |
||||||
|
mkdir(kSecondaryPath.c_str(), S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH); |
||||||
|
if (ret < 0) { |
||||||
|
perror("failed to create directory for secondary instance"); |
||||||
|
exit(0); |
||||||
|
} |
||||||
|
} |
||||||
|
DB* db = nullptr; |
||||||
|
Options options; |
||||||
|
options.create_if_missing = false; |
||||||
|
options.max_open_files = -1; |
||||||
|
Status s = DB::OpenAsSecondary(options, kDBPath, kSecondaryPath, &db); |
||||||
|
if (!s.ok()) { |
||||||
|
fprintf(stderr, "[process %ld] Failed to open in secondary mode: %s\n", |
||||||
|
my_pid, s.ToString().c_str()); |
||||||
|
assert(false); |
||||||
|
} else { |
||||||
|
fprintf(stdout, "[process %ld] Secondary instance starts\n", my_pid); |
||||||
|
} |
||||||
|
|
||||||
|
ReadOptions ropts; |
||||||
|
ropts.verify_checksums = true; |
||||||
|
ropts.total_order_seek = true; |
||||||
|
|
||||||
|
std::vector<std::thread> test_threads; |
||||||
|
test_threads.emplace_back([&]() { |
||||||
|
while (1 == ShouldSecondaryWait().load(std::memory_order_relaxed)) { |
||||||
|
std::unique_ptr<Iterator> iter(db->NewIterator(ropts)); |
||||||
|
iter->SeekToFirst(); |
||||||
|
size_t count = 0; |
||||||
|
for (; iter->Valid(); iter->Next()) { |
||||||
|
++count; |
||||||
|
} |
||||||
|
} |
||||||
|
fprintf(stdout, "[process %ld] Range_scan thread finished\n", my_pid); |
||||||
|
}); |
||||||
|
|
||||||
|
test_threads.emplace_back([&]() { |
||||||
|
std::srand(time(nullptr)); |
||||||
|
while (1 == ShouldSecondaryWait().load(std::memory_order_relaxed)) { |
||||||
|
Slice key = Key(std::rand() % kMaxKey); |
||||||
|
std::string value; |
||||||
|
db->Get(ropts, key, &value); |
||||||
|
} |
||||||
|
fprintf(stdout, "[process %ld] Point lookup thread finished\n"); |
||||||
|
}); |
||||||
|
|
||||||
|
uint64_t curr_key = 0; |
||||||
|
while (1 == ShouldSecondaryWait().load(std::memory_order_relaxed)) { |
||||||
|
s = db->TryCatchUpWithPrimary(); |
||||||
|
if (!s.ok()) { |
||||||
|
fprintf(stderr, |
||||||
|
"[process %ld] error while trying to catch up with " |
||||||
|
"primary %s\n", |
||||||
|
my_pid, s.ToString().c_str()); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
{ |
||||||
|
std::unique_ptr<Iterator> iter(db->NewIterator(ropts)); |
||||||
|
if (!iter) { |
||||||
|
fprintf(stderr, "[process %ld] Failed to create iterator\n", my_pid); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
iter->SeekToLast(); |
||||||
|
if (iter->Valid()) { |
||||||
|
uint64_t curr_max_key = Key(iter->key().ToString()); |
||||||
|
if (curr_max_key != curr_key) { |
||||||
|
fprintf(stdout, "[process %ld] Observed key %" PRIu64 "\n", my_pid, |
||||||
|
curr_key); |
||||||
|
curr_key = curr_max_key; |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
std::this_thread::sleep_for(std::chrono::seconds(1)); |
||||||
|
} |
||||||
|
s = db->TryCatchUpWithPrimary(); |
||||||
|
if (!s.ok()) { |
||||||
|
fprintf(stderr, |
||||||
|
"[process %ld] error while trying to catch up with " |
||||||
|
"primary %s\n", |
||||||
|
my_pid, s.ToString().c_str()); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
|
||||||
|
std::vector<ColumnFamilyDescriptor> column_families; |
||||||
|
for (const auto& cf_name : GetColumnFamilyNames()) { |
||||||
|
column_families.push_back(ColumnFamilyDescriptor(cf_name, options)); |
||||||
|
} |
||||||
|
std::vector<ColumnFamilyHandle*> handles; |
||||||
|
DB* verification_db = nullptr; |
||||||
|
s = DB::OpenForReadOnly(options, kDBPath, column_families, &handles, |
||||||
|
&verification_db); |
||||||
|
assert(s.ok()); |
||||||
|
Iterator* iter1 = verification_db->NewIterator(ropts); |
||||||
|
iter1->SeekToFirst(); |
||||||
|
|
||||||
|
Iterator* iter = db->NewIterator(ropts); |
||||||
|
iter->SeekToFirst(); |
||||||
|
for (; iter->Valid() && iter1->Valid(); iter->Next(), iter1->Next()) { |
||||||
|
if (iter->key().ToString() != iter1->key().ToString()) { |
||||||
|
fprintf(stderr, "%" PRIu64 "!= %" PRIu64 "\n", |
||||||
|
Key(iter->key().ToString()), Key(iter1->key().ToString())); |
||||||
|
assert(false); |
||||||
|
} else if (iter->value().ToString() != iter1->value().ToString()) { |
||||||
|
fprintf(stderr, "Value mismatch\n"); |
||||||
|
assert(false); |
||||||
|
} |
||||||
|
} |
||||||
|
fprintf(stdout, "[process %ld] Verification succeeded\n", my_pid); |
||||||
|
for (auto& thr : test_threads) { |
||||||
|
thr.join(); |
||||||
|
} |
||||||
|
delete iter; |
||||||
|
delete iter1; |
||||||
|
delete db; |
||||||
|
delete verification_db; |
||||||
|
} |
||||||
|
|
||||||
|
int main(int argc, char** argv) { |
||||||
|
if (argc < 2) { |
||||||
|
fprintf(stderr, "%s <0 for primary, 1 for secondary>\n", argv[0]); |
||||||
|
return 0; |
||||||
|
} |
||||||
|
if (atoi(argv[1]) == 0) { |
||||||
|
RunPrimary(); |
||||||
|
} else { |
||||||
|
RunSecondary(); |
||||||
|
} |
||||||
|
return 0; |
||||||
|
} |
||||||
|
#else // OS_LINUX
|
||||||
|
int main() { |
||||||
|
fpritnf(stderr, "Not implemented.\n"); |
||||||
|
return 0; |
||||||
|
} |
||||||
|
#endif // !OS_LINUX
|
Loading…
Reference in new issue