提交 e633983c 编写于 作者: Y Yanqin Jin 提交者: Facebook Github Bot

Add support to flush multiple CFs atomically (#4262)

Summary:
Leverage existing `FlushJob` to implement atomic flush of multiple column families.

This PR depends on other PRs and is a subset of #3752 . This PR itself is not sufficient in fulfilling atomic flush.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4262

Differential Revision: D9283109

Pulled By: riversand963

fbshipit-source-id: 65401f913e4160b0a61c0be6cd02adc15dad28ed
上级 32b4d4ad
......@@ -219,7 +219,8 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname,
own_sfm_(options.sst_file_manager == nullptr),
preserve_deletes_(options.preserve_deletes),
closed_(false),
error_handler_(this, immutable_db_options_, &mutex_) {
error_handler_(this, immutable_db_options_, &mutex_),
atomic_flush_commit_in_progress_(false) {
// !batch_per_trx_ implies seq_per_batch_ because it is only unset for
// WriteUnprepared, which should use seq_per_batch_.
assert(batch_per_txn_ || seq_per_batch_);
......
......@@ -908,18 +908,18 @@ class DBImpl : public DB {
// Argument required by background flush thread.
struct BGFlushArg {
BGFlushArg()
: cfd_(nullptr), memtable_id_(0), superversion_context_(nullptr) {}
BGFlushArg(ColumnFamilyData* cfd, uint64_t memtable_id,
: cfd_(nullptr), max_memtable_id_(0), superversion_context_(nullptr) {}
BGFlushArg(ColumnFamilyData* cfd, uint64_t max_memtable_id,
SuperVersionContext* superversion_context)
: cfd_(cfd),
memtable_id_(memtable_id),
max_memtable_id_(max_memtable_id),
superversion_context_(superversion_context) {}
// Column family to flush.
ColumnFamilyData* cfd_;
// Maximum ID of memtable to flush. In this column family, memtables with
// IDs smaller than this value must be flushed before this flush completes.
uint64_t memtable_id_;
uint64_t max_memtable_id_;
// Pointer to a SuperVersionContext object. After flush completes, RocksDB
// installs a new superversion for the column family. This operation
// requires a SuperVersionContext object (currently embedded in JobContext).
......@@ -932,6 +932,10 @@ class DBImpl : public DB {
const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
JobContext* job_context, LogBuffer* log_buffer);
Status AtomicFlushMemTablesToOutputFiles(
const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
JobContext* job_context, LogBuffer* log_buffer);
// REQUIRES: log_numbers are sorted in ascending order
Status RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
SequenceNumber* next_sequence, bool read_only);
......@@ -1579,6 +1583,16 @@ class DBImpl : public DB {
bool closed_;
ErrorHandler error_handler_;
// True if the DB is committing atomic flush.
// TODO (yanqin) the current impl assumes that the entire DB belongs to
// a single atomic flush group. In the future we need to add a new class
// (struct) similar to the following to make it more general.
// struct AtomicFlushGroup {
// bool commit_in_progress_;
// std::vector<MemTableList*> imm_lists;
// };
bool atomic_flush_commit_in_progress_;
};
extern Options SanitizeOptions(const std::string& db,
......
......@@ -125,11 +125,13 @@ Status DBImpl::FlushMemTableToOutputFile(
}
FlushJob flush_job(
dbname_, cfd, immutable_db_options_, mutable_cf_options,
env_options_for_compaction_, versions_.get(), &mutex_, &shutting_down_,
snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker,
job_context, log_buffer, directories_.GetDbDir(), GetDataDir(cfd, 0U),
nullptr /* memtable_id */, env_options_for_compaction_, versions_.get(),
&mutex_, &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot,
snapshot_checker, job_context, log_buffer, directories_.GetDbDir(),
GetDataDir(cfd, 0U),
GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_,
&event_logger_, mutable_cf_options.report_bg_io_stats);
&event_logger_, mutable_cf_options.report_bg_io_stats,
true /* sync_output_directory */, true /* write_manifest */);
FileMetaData file_meta;
......@@ -169,7 +171,7 @@ Status DBImpl::FlushMemTableToOutputFile(
InstallSuperVersionAndScheduleWork(cfd, superversion_context,
mutable_cf_options);
if (made_progress) {
*made_progress = 1;
*made_progress = true;
}
VersionStorageInfo::LevelSummaryStorage tmp;
ROCKS_LOG_BUFFER(log_buffer, "[%s] Level summary: %s\n",
......@@ -225,6 +227,194 @@ Status DBImpl::FlushMemTablesToOutputFiles(
return s;
}
/*
* Atomically flushes multiple column families.
*
* For each column family, all memtables with ID smaller than or equal to the
* ID specified in bg_flush_args will be flushed. Only after all column
* families finish flush will this function commit to MANIFEST. If any of the
* column families are not flushed successfully, this function does not have
* any side-effect on the state of the database.
*/
Status DBImpl::AtomicFlushMemTablesToOutputFiles(
const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
JobContext* job_context, LogBuffer* log_buffer) {
mutex_.AssertHeld();
autovector<ColumnFamilyData*> cfds;
for (const auto& arg : bg_flush_args) {
cfds.emplace_back(arg.cfd_);
}
#ifndef NDEBUG
for (const auto cfd : cfds) {
assert(cfd->imm()->NumNotFlushed() != 0);
assert(cfd->imm()->IsFlushPending());
}
#endif /* !NDEBUG */
SequenceNumber earliest_write_conflict_snapshot;
std::vector<SequenceNumber> snapshot_seqs =
snapshots_.GetAll(&earliest_write_conflict_snapshot);
auto snapshot_checker = snapshot_checker_.get();
if (use_custom_gc_ && snapshot_checker == nullptr) {
snapshot_checker = DisableGCSnapshotChecker::Instance();
}
autovector<Directory*> distinct_output_dirs;
std::vector<FlushJob> jobs;
int num_cfs = static_cast<int>(cfds.size());
for (int i = 0; i < num_cfs; ++i) {
auto cfd = cfds[i];
Directory* data_dir = GetDataDir(cfd, 0U);
// Add to distinct output directories if eligible. Use linear search. Since
// the number of elements in the vector is not large, performance should be
// tolerable.
bool found = false;
for (const auto dir : distinct_output_dirs) {
if (dir == data_dir) {
found = true;
break;
}
}
if (!found) {
distinct_output_dirs.emplace_back(data_dir);
}
const MutableCFOptions& mutable_cf_options =
*cfd->GetLatestMutableCFOptions();
const uint64_t* max_memtable_id = &(bg_flush_args[i].max_memtable_id_);
jobs.emplace_back(
dbname_, cfds[i], immutable_db_options_, mutable_cf_options,
max_memtable_id, env_options_for_compaction_, versions_.get(), &mutex_,
&shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot,
snapshot_checker, job_context, log_buffer, directories_.GetDbDir(),
data_dir, GetCompressionFlush(*cfd->ioptions(), mutable_cf_options),
stats_, &event_logger_, mutable_cf_options.report_bg_io_stats,
false /* sync_output_directory */, false /* write_manifest */);
jobs.back().PickMemTable();
}
autovector<FileMetaData> file_meta;
Status s;
assert(num_cfs == static_cast<int>(jobs.size()));
for (int i = 0; i != num_cfs; ++i) {
file_meta.emplace_back(FileMetaData());
#ifndef ROCKSDB_LITE
const MutableCFOptions& mutable_cf_options =
*cfds[i]->GetLatestMutableCFOptions();
// may temporarily unlock and lock the mutex.
NotifyOnFlushBegin(cfds[i], &file_meta[i], mutable_cf_options,
job_context->job_id, jobs[i].GetTableProperties());
#endif /* !ROCKSDB_LITE */
}
if (logfile_number_ > 0) {
// TODO (yanqin) investigate whether we should sync the closed logs for
// single column family case.
s = SyncClosedLogs(job_context);
}
if (s.ok()) {
// TODO (yanqin): parallelize jobs with threads.
for (int i = 0; i != num_cfs; ++i) {
s = jobs[i].Run(&logs_with_prep_tracker_, &file_meta[i]);
if (!s.ok()) {
break;
}
}
}
if (s.ok()) {
// Sync on all distinct output directories.
for (auto dir : distinct_output_dirs) {
if (dir != nullptr) {
s = dir->Fsync();
if (!s.ok()) {
break;
}
}
}
if (s.ok()) {
autovector<const autovector<MemTable*>*> mems_list;
for (int i = 0; i != num_cfs; ++i) {
const auto& mems = jobs[i].GetMemTables();
mems_list.emplace_back(&mems);
}
autovector<ColumnFamilyData*> all_cfds;
autovector<MemTableList*> imm_lists;
autovector<const MutableCFOptions*> mutable_cf_options_list;
for (auto cfd : *versions_->GetColumnFamilySet()) {
all_cfds.emplace_back(cfd);
imm_lists.emplace_back(cfd->imm());
mutable_cf_options_list.emplace_back(cfd->GetLatestMutableCFOptions());
}
s = MemTableList::TryInstallMemtableFlushResults(
imm_lists, all_cfds, mutable_cf_options_list, mems_list,
&atomic_flush_commit_in_progress_, &logs_with_prep_tracker_,
versions_.get(), &mutex_, file_meta, &job_context->memtables_to_free,
directories_.GetDbDir(), log_buffer);
}
}
if (s.ok()) {
assert(num_cfs ==
static_cast<int>(job_context->superversion_contexts.size()));
for (int i = 0; i != num_cfs; ++i) {
InstallSuperVersionAndScheduleWork(cfds[i],
&job_context->superversion_contexts[i],
*cfds[i]->GetLatestMutableCFOptions());
VersionStorageInfo::LevelSummaryStorage tmp;
ROCKS_LOG_BUFFER(log_buffer, "[%s] Level summary: %s\n",
cfds[i]->GetName().c_str(),
cfds[i]->current()->storage_info()->LevelSummary(&tmp));
}
if (made_progress) {
*made_progress = true;
}
#ifndef ROCKSDB_LITE
auto sfm = static_cast<SstFileManagerImpl*>(
immutable_db_options_.sst_file_manager.get());
for (int i = 0; i != num_cfs; ++i) {
NotifyOnFlushCompleted(cfds[i], &file_meta[i],
*cfds[i]->GetLatestMutableCFOptions(),
job_context->job_id, jobs[i].GetTableProperties());
if (sfm) {
std::string file_path = MakeTableFileName(
cfds[i]->ioptions()->cf_paths[0].path, file_meta[i].fd.GetNumber());
sfm->OnAddFile(file_path);
if (sfm->IsMaxAllowedSpaceReached() &&
error_handler_.GetBGError().ok()) {
Status new_bg_error =
Status::SpaceLimit("Max allowed space was reached");
error_handler_.SetBGError(new_bg_error,
BackgroundErrorReason::kFlush);
}
}
}
#endif // ROCKSDB_LITE
}
if (!s.ok()) {
for (int i = 0; i != num_cfs; ++i) {
auto& mems = jobs[i].GetMemTables();
cfds[i]->imm()->RollbackMemtableFlush(mems, file_meta[i].fd.GetNumber());
jobs[i].Cancel();
}
if (!s.IsShutdownInProgress()) {
Status new_bg_error = s;
error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush);
}
}
return s;
}
void DBImpl::NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta,
const MutableCFOptions& mutable_cf_options,
int job_id, TableProperties prop) {
......@@ -983,7 +1173,6 @@ Status DBImpl::Flush(const FlushOptions& flush_options,
return s;
}
Status DBImpl::FlushAllCFs(FlushReason flush_reason) {
Status s;
WriteContext context;
......
......@@ -85,11 +85,11 @@ const char* GetFlushReasonString (FlushReason flush_reason) {
}
}
FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd,
const ImmutableDBOptions& db_options,
const MutableCFOptions& mutable_cf_options,
const EnvOptions env_options, VersionSet* versions,
const uint64_t* max_memtable_id,
const EnvOptions& env_options, VersionSet* versions,
InstrumentedMutex* db_mutex,
std::atomic<bool>* shutting_down,
std::vector<SequenceNumber> existing_snapshots,
......@@ -98,11 +98,13 @@ FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd,
LogBuffer* log_buffer, Directory* db_directory,
Directory* output_file_directory,
CompressionType output_compression, Statistics* stats,
EventLogger* event_logger, bool measure_io_stats)
EventLogger* event_logger, bool measure_io_stats,
const bool sync_output_directory, const bool write_manifest)
: dbname_(dbname),
cfd_(cfd),
db_options_(db_options),
mutable_cf_options_(mutable_cf_options),
max_memtable_id_(max_memtable_id),
env_options_(env_options),
versions_(versions),
db_mutex_(db_mutex),
......@@ -118,6 +120,8 @@ FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd,
stats_(stats),
event_logger_(event_logger),
measure_io_stats_(measure_io_stats),
sync_output_directory_(sync_output_directory),
write_manifest_(write_manifest),
edit_(nullptr),
base_(nullptr),
pick_memtable_called(false) {
......@@ -162,7 +166,7 @@ void FlushJob::PickMemTable() {
assert(!pick_memtable_called);
pick_memtable_called = true;
// Save the contents of the earliest memtable as a new Table
cfd_->imm()->PickMemtablesToFlush(&mems_);
cfd_->imm()->PickMemtablesToFlush(max_memtable_id_, &mems_);
if (mems_.empty()) {
return;
}
......@@ -226,7 +230,7 @@ Status FlushJob::Run(LogsWithPrepTracker* prep_tracker,
if (!s.ok()) {
cfd_->imm()->RollbackMemtableFlush(mems_, meta_.fd.GetNumber());
} else {
} else if (write_manifest_) {
TEST_SYNC_POINT("FlushJob::InstallResults");
// Replace immutable memtable with the generated Table
s = cfd_->imm()->TryInstallMemtableFlushResults(
......@@ -373,7 +377,7 @@ Status FlushJob::WriteLevel0Table() {
s.ToString().c_str(),
meta_.marked_for_compaction ? " (needs compaction)" : "");
if (s.ok() && output_file_directory_ != nullptr) {
if (s.ok() && output_file_directory_ != nullptr && sync_output_directory_) {
s = output_file_directory_->Fsync();
}
TEST_SYNC_POINT("FlushJob::WriteLevel0Table");
......
......@@ -59,14 +59,16 @@ class FlushJob {
FlushJob(const std::string& dbname, ColumnFamilyData* cfd,
const ImmutableDBOptions& db_options,
const MutableCFOptions& mutable_cf_options,
const EnvOptions env_options, VersionSet* versions,
InstrumentedMutex* db_mutex, std::atomic<bool>* shutting_down,
const uint64_t* max_memtable_id, const EnvOptions& env_options,
VersionSet* versions, InstrumentedMutex* db_mutex,
std::atomic<bool>* shutting_down,
std::vector<SequenceNumber> existing_snapshots,
SequenceNumber earliest_write_conflict_snapshot,
SnapshotChecker* snapshot_checker, JobContext* job_context,
LogBuffer* log_buffer, Directory* db_directory,
Directory* output_file_directory, CompressionType output_compression,
Statistics* stats, EventLogger* event_logger, bool measure_io_stats);
Statistics* stats, EventLogger* event_logger, bool measure_io_stats,
const bool sync_output_directory, const bool write_manifest);
~FlushJob();
......@@ -77,16 +79,24 @@ class FlushJob {
FileMetaData* file_meta = nullptr);
void Cancel();
TableProperties GetTableProperties() const { return table_properties_; }
const autovector<MemTable*>& GetMemTables() const { return mems_; }
private:
void ReportStartedFlush();
void ReportFlushInputSize(const autovector<MemTable*>& mems);
void RecordFlushIOStats();
Status WriteLevel0Table();
const std::string& dbname_;
ColumnFamilyData* cfd_;
const ImmutableDBOptions& db_options_;
const MutableCFOptions& mutable_cf_options_;
// Pointer to a variable storing the largest memtable id to flush in this
// flush job. RocksDB uses this variable to select the memtables to flush in
// this job. All memtables in this column family with an ID smaller than or
// equal to *max_memtable_id_ will be selected for flush. If null, then all
// memtables in the column family will be selected.
const uint64_t* max_memtable_id_;
const EnvOptions env_options_;
VersionSet* versions_;
InstrumentedMutex* db_mutex_;
......@@ -103,6 +113,23 @@ class FlushJob {
EventLogger* event_logger_;
TableProperties table_properties_;
bool measure_io_stats_;
// True if this flush job should call fsync on the output directory. False
// otherwise.
// Usually sync_output_directory_ is true. A flush job needs to call sync on
// the output directory before committing to the MANIFEST.
// However, an individual flush job does not have to call sync on the output
// directory if it is part of an atomic flush. After all flush jobs in the
// atomic flush succeed, call sync once on each distinct output directory.
const bool sync_output_directory_;
// True if this flush job should write to MANIFEST after successfully
// flushing memtables. False otherwise.
// Usually write_manifest_ is true. A flush job commits to the MANIFEST after
// flushing the memtables.
// However, an individual flush job cannot rashly write to the MANIFEST
// immediately after it finishes the flush if it is part of an atomic flush.
// In this case, only after all flush jobs succeed in flush can RocksDB
// commit to the MANIFEST.
const bool write_manifest_;
// Variables below are set by PickMemTable():
FileMetaData meta_;
......
......@@ -30,6 +30,7 @@ class FlushJobTest : public testing::Test {
dbname_(test::PerThreadDBPath("flush_job_test")),
options_(),
db_options_(options_),
column_family_names_({kDefaultColumnFamilyName, "foo", "bar"}),
table_cache_(NewLRUCache(50000, 16)),
write_buffer_manager_(db_options_.db_write_buffer_size),
versions_(new VersionSet(dbname_, &db_options_, env_options_,
......@@ -45,7 +46,9 @@ class FlushJobTest : public testing::Test {
NewDB();
std::vector<ColumnFamilyDescriptor> column_families;
cf_options_.table_factory = mock_table_factory_;
column_families.emplace_back(kDefaultColumnFamilyName, cf_options_);
for (const auto& cf_name : column_family_names_) {
column_families.emplace_back(cf_name, cf_options_);
}
EXPECT_OK(versions_->Recover(column_families, false));
}
......@@ -56,6 +59,19 @@ class FlushJobTest : public testing::Test {
new_db.SetNextFile(2);
new_db.SetLastSequence(0);
autovector<VersionEdit> new_cfs;
SequenceNumber last_seq = 1;
uint32_t cf_id = 1;
for (size_t i = 1; i != column_family_names_.size(); ++i) {
VersionEdit new_cf;
new_cf.AddColumnFamily(column_family_names_[i]);
new_cf.SetColumnFamily(cf_id++);
new_cf.SetLogNumber(0);
new_cf.SetNextFile(2);
new_cf.SetLastSequence(last_seq++);
new_cfs.emplace_back(new_cf);
}
const std::string manifest = DescriptorFileName(dbname_, 1);
unique_ptr<WritableFile> file;
Status s = env_->NewWritableFile(
......@@ -68,6 +84,13 @@ class FlushJobTest : public testing::Test {
std::string record;
new_db.EncodeTo(&record);
s = log.AddRecord(record);
for (const auto& e : new_cfs) {
record.clear();
e.EncodeTo(&record);
s = log.AddRecord(record);
ASSERT_OK(s);
}
}
ASSERT_OK(s);
// Make "CURRENT" file that points to the new manifest file.
......@@ -79,6 +102,7 @@ class FlushJobTest : public testing::Test {
EnvOptions env_options_;
Options options_;
ImmutableDBOptions db_options_;
const std::vector<std::string> column_family_names_;
std::shared_ptr<Cache> table_cache_;
WriteController write_controller_;
WriteBufferManager write_buffer_manager_;
......@@ -96,9 +120,11 @@ TEST_F(FlushJobTest, Empty) {
SnapshotChecker* snapshot_checker = nullptr; // not relavant
FlushJob flush_job(
dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_,
*cfd->GetLatestMutableCFOptions(), env_options_, versions_.get(), &mutex_,
&shutting_down_, {}, kMaxSequenceNumber, snapshot_checker, &job_context,
nullptr, nullptr, nullptr, kNoCompression, nullptr, &event_logger, false);
*cfd->GetLatestMutableCFOptions(), nullptr /* memtable_id */,
env_options_, versions_.get(), &mutex_, &shutting_down_, {},
kMaxSequenceNumber, snapshot_checker, &job_context, nullptr, nullptr,
nullptr, kNoCompression, nullptr, &event_logger, false,
true /* sync_output_directory */, true /* write_manifest */);
{
InstrumentedMutexLock l(&mutex_);
flush_job.PickMemTable();
......@@ -139,12 +165,13 @@ TEST_F(FlushJobTest, NonEmpty) {
EventLogger event_logger(db_options_.info_log.get());
SnapshotChecker* snapshot_checker = nullptr; // not relavant
FlushJob flush_job(dbname_, versions_->GetColumnFamilySet()->GetDefault(),
db_options_, *cfd->GetLatestMutableCFOptions(),
env_options_, versions_.get(), &mutex_, &shutting_down_,
{}, kMaxSequenceNumber, snapshot_checker, &job_context,
nullptr, nullptr, nullptr, kNoCompression,
db_options_.statistics.get(), &event_logger, true);
FlushJob flush_job(
dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_,
*cfd->GetLatestMutableCFOptions(), nullptr /* memtable_id */,
env_options_, versions_.get(), &mutex_, &shutting_down_, {},
kMaxSequenceNumber, snapshot_checker, &job_context, nullptr, nullptr,
nullptr, kNoCompression, db_options_.statistics.get(), &event_logger,
true, true /* sync_output_directory */, true /* write_manifest */);
HistogramData hist;
FileMetaData file_meta;
......@@ -165,6 +192,178 @@ TEST_F(FlushJobTest, NonEmpty) {
job_context.Clean();
}
TEST_F(FlushJobTest, FlushMemTablesSingleColumnFamily) {
const size_t num_mems = 2;
const size_t num_mems_to_flush = 1;
const size_t num_keys_per_table = 100;
JobContext job_context(0);
ColumnFamilyData* cfd = versions_->GetColumnFamilySet()->GetDefault();
std::vector<uint64_t> memtable_ids;
std::vector<MemTable*> new_mems;
for (size_t i = 0; i != num_mems; ++i) {
MemTable* mem = cfd->ConstructNewMemtable(*cfd->GetLatestMutableCFOptions(),
kMaxSequenceNumber);
mem->SetID(i);
mem->Ref();
new_mems.emplace_back(mem);
memtable_ids.push_back(mem->GetID());
for (size_t j = 0; j < num_keys_per_table; ++j) {
std::string key(ToString(j + i * num_keys_per_table));
std::string value("value" + key);
mem->Add(SequenceNumber(j + i * num_keys_per_table), kTypeValue, key,
value);
}
}
autovector<MemTable*> to_delete;
for (auto mem : new_mems) {
cfd->imm()->Add(mem, &to_delete);
}
EventLogger event_logger(db_options_.info_log.get());
SnapshotChecker* snapshot_checker = nullptr; // not relavant
assert(memtable_ids.size() == num_mems);
uint64_t smallest_memtable_id = memtable_ids.front();
uint64_t flush_memtable_id = smallest_memtable_id + num_mems_to_flush - 1;
FlushJob flush_job(
dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_,
*cfd->GetLatestMutableCFOptions(), &flush_memtable_id, env_options_,
versions_.get(), &mutex_, &shutting_down_, {}, kMaxSequenceNumber,
snapshot_checker, &job_context, nullptr, nullptr, nullptr, kNoCompression,
db_options_.statistics.get(), &event_logger, true,
true /* sync_output_directory */, true /* write_manifest */);
HistogramData hist;
FileMetaData file_meta;
mutex_.Lock();
flush_job.PickMemTable();
ASSERT_OK(flush_job.Run(nullptr /* prep_tracker */, &file_meta));
mutex_.Unlock();
db_options_.statistics->histogramData(FLUSH_TIME, &hist);
ASSERT_GT(hist.average, 0.0);
ASSERT_EQ(ToString(0), file_meta.smallest.user_key().ToString());
ASSERT_EQ("99", file_meta.largest.user_key().ToString());
ASSERT_EQ(0, file_meta.fd.smallest_seqno);
ASSERT_EQ(SequenceNumber(num_mems_to_flush * num_keys_per_table - 1),
file_meta.fd.largest_seqno);
for (auto m : to_delete) {
delete m;
}
to_delete.clear();
job_context.Clean();
}
TEST_F(FlushJobTest, FlushMemtablesMultipleColumnFamilies) {
autovector<ColumnFamilyData*> all_cfds;
for (auto cfd : *versions_->GetColumnFamilySet()) {
all_cfds.push_back(cfd);
}
const std::vector<size_t> num_memtables = {2, 1, 3};
assert(num_memtables.size() == column_family_names_.size());
const size_t num_keys_per_memtable = 1000;
JobContext job_context(0);
std::vector<uint64_t> memtable_ids;
std::vector<SequenceNumber> smallest_seqs;
std::vector<SequenceNumber> largest_seqs;
autovector<MemTable*> to_delete;
SequenceNumber curr_seqno = 0;
size_t k = 0;
for (auto cfd : all_cfds) {
smallest_seqs.push_back(curr_seqno);
for (size_t i = 0; i != num_memtables[k]; ++i) {
MemTable* mem = cfd->ConstructNewMemtable(
*cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber);
mem->SetID(i);
mem->Ref();
mem->TEST_AtomicFlushSequenceNumber() = 123;
for (size_t j = 0; j != num_keys_per_memtable; ++j) {
std::string key(ToString(j + i * num_keys_per_memtable));
std::string value("value" + key);
mem->Add(curr_seqno++, kTypeValue, key, value);
}
cfd->imm()->Add(mem, &to_delete);
}
largest_seqs.push_back(curr_seqno - 1);
memtable_ids.push_back(num_memtables[k++] - 1);
}
EventLogger event_logger(db_options_.info_log.get());
SnapshotChecker* snapshot_checker = nullptr; // not relevant
std::vector<FlushJob> flush_jobs;
k = 0;
for (auto cfd : all_cfds) {
std::vector<SequenceNumber> snapshot_seqs;
flush_jobs.emplace_back(
dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(),
&memtable_ids[k], env_options_, versions_.get(), &mutex_,
&shutting_down_, snapshot_seqs, kMaxSequenceNumber, snapshot_checker,
&job_context, nullptr, nullptr, nullptr, kNoCompression,
db_options_.statistics.get(), &event_logger, true,
false /* sync_output_directory */, false /* write_manifest */);
k++;
}
HistogramData hist;
autovector<FileMetaData> file_metas;
mutex_.Lock();
for (auto& job : flush_jobs) {
job.PickMemTable();
}
for (auto& job : flush_jobs) {
FileMetaData meta;
// Run will release and re-acquire mutex
ASSERT_OK(job.Run(nullptr /**/, &meta));
file_metas.emplace_back(meta);
}
autovector<const autovector<MemTable*>*> mems_list;
for (size_t i = 0; i != all_cfds.size(); ++i) {
const auto& mems = flush_jobs[i].GetMemTables();
mems_list.push_back(&mems);
}
autovector<MemTableList*> imm_lists;
autovector<const MutableCFOptions*> mutable_cf_options_list;
for (auto cfd : all_cfds) {
imm_lists.push_back(cfd->imm());
mutable_cf_options_list.push_back(cfd->GetLatestMutableCFOptions());
}
bool atomic_flush_commit_in_progress = false;
Status s = MemTableList::TryInstallMemtableFlushResults(
imm_lists, all_cfds, mutable_cf_options_list, mems_list,
&atomic_flush_commit_in_progress, nullptr /* logs_prep_tracker */,
versions_.get(), &mutex_, file_metas, &job_context.memtables_to_free,
nullptr /* db_directory */, nullptr /* log_buffer */);
ASSERT_OK(s);
mutex_.Unlock();
db_options_.statistics->histogramData(FLUSH_TIME, &hist);
ASSERT_GT(hist.average, 0.0);
k = 0;
for (const auto& file_meta : file_metas) {
ASSERT_EQ(ToString(0), file_meta.smallest.user_key().ToString());
ASSERT_EQ("999", file_meta.largest.user_key()
.ToString()); // max key by bytewise comparator
ASSERT_EQ(smallest_seqs[k], file_meta.fd.smallest_seqno);
ASSERT_EQ(largest_seqs[k], file_meta.fd.largest_seqno);
// Verify that imm is empty
ASSERT_EQ(std::numeric_limits<uint64_t>::max(),
all_cfds[k]->imm()->GetEarliestMemTableID());
ASSERT_EQ(0, all_cfds[k]->imm()->GetLatestMemTableID());
++k;
}
for (auto m : to_delete) {
delete m;
}
to_delete.clear();
job_context.Clean();
}
TEST_F(FlushJobTest, Snapshots) {
JobContext job_context(0);
auto cfd = versions_->GetColumnFamilySet()->GetDefault();
......@@ -213,12 +412,13 @@ TEST_F(FlushJobTest, Snapshots) {
EventLogger event_logger(db_options_.info_log.get());
SnapshotChecker* snapshot_checker = nullptr; // not relavant
FlushJob flush_job(dbname_, versions_->GetColumnFamilySet()->GetDefault(),
db_options_, *cfd->GetLatestMutableCFOptions(),
env_options_, versions_.get(), &mutex_, &shutting_down_,
snapshots, kMaxSequenceNumber, snapshot_checker,
&job_context, nullptr, nullptr, nullptr, kNoCompression,
db_options_.statistics.get(), &event_logger, true);
FlushJob flush_job(
dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_,
*cfd->GetLatestMutableCFOptions(), nullptr /* memtable_id */,
env_options_, versions_.get(), &mutex_, &shutting_down_, snapshots,
kMaxSequenceNumber, snapshot_checker, &job_context, nullptr, nullptr,
nullptr, kNoCompression, db_options_.statistics.get(), &event_logger,
true, true /* sync_output_directory */, true /* write_manifest */);
mutex_.Lock();
flush_job.PickMemTable();
ASSERT_OK(flush_job.Run());
......
......@@ -101,7 +101,8 @@ MemTable::MemTable(const InternalKeyComparator& cmp,
env_(ioptions.env),
insert_with_hint_prefix_extractor_(
ioptions.memtable_insert_with_hint_prefix_extractor),
oldest_key_time_(std::numeric_limits<uint64_t>::max()) {
oldest_key_time_(std::numeric_limits<uint64_t>::max()),
atomic_flush_seqno_(kMaxSequenceNumber) {
UpdateFlushState();
// something went wrong if we need to flush before inserting anything
assert(!ShouldScheduleFlush());
......@@ -640,7 +641,7 @@ static bool SaveValue(void* arg, const char* entry) {
*(s->found_final_value) = true;
return false;
}
FALLTHROUGH_INTENDED;
FALLTHROUGH_INTENDED;
case kTypeValue: {
if (s->inplace_update_support) {
s->mem->GetLock(s->key->user_key())->ReadLock();
......
......@@ -383,6 +383,14 @@ class MemTable {
uint64_t GetID() const { return id_; }
SequenceNumber& TEST_AtomicFlushSequenceNumber() {
return atomic_flush_seqno_;
}
void TEST_SetFlushCompleted(bool completed) { flush_completed_ = completed; }
void TEST_SetFileNumber(uint64_t file_num) { file_number_ = file_num; }
private:
enum FlushStateEnum { FLUSH_NOT_REQUESTED, FLUSH_REQUESTED, FLUSH_SCHEDULED };
......@@ -455,6 +463,12 @@ class MemTable {
// Memtable id to track flush.
uint64_t id_ = 0;
// Sequence number of the atomic flush that is responsible for this memtable.
// The sequence number of atomic flush is a seq, such that no writes with
// sequence numbers greater than or equal to seq are flushed, while all
// writes with sequence number smaller than seq are flushed.
SequenceNumber atomic_flush_seqno_;
// Returns a heuristic flush decision
bool ShouldFlushNow() const;
......
......@@ -11,6 +11,7 @@
#include <inttypes.h>
#include <limits>
#include <queue>
#include <string>
#include "db/db_impl.h"
#include "db/memtable.h"
......@@ -269,10 +270,232 @@ void MemTableListVersion::TrimHistory(autovector<MemTable*>* to_delete) {
}
}
// Try to record multiple successful flush to the MANIFEST as an atomic unit.
// This function may just return Status::OK if there has already been
// a concurrent thread performing actual recording.
Status MemTableList::TryInstallMemtableFlushResults(
autovector<MemTableList*>& imm_lists,
const autovector<ColumnFamilyData*>& cfds,
const autovector<const MutableCFOptions*>& mutable_cf_options_list,
const autovector<const autovector<MemTable*>*>& mems_list,
bool* atomic_flush_commit_in_progress, LogsWithPrepTracker* prep_tracker,
VersionSet* vset, InstrumentedMutex* mu,
const autovector<FileMetaData>& file_metas,
autovector<MemTable*>* to_delete, Directory* db_directory,
LogBuffer* log_buffer) {
AutoThreadOperationStageUpdater stage_updater(
ThreadStatus::STAGE_MEMTABLE_INSTALL_FLUSH_RESULTS);
mu->AssertHeld();
for (size_t k = 0; k != mems_list.size(); ++k) {
for (size_t i = 0; i != mems_list[k]->size(); ++i) {
assert(i == 0 || (*mems_list[k])[i]->GetEdits()->NumEntries() == 0);
(*mems_list[k])[i]->flush_completed_ = true;
(*mems_list[k])[i]->file_number_ = file_metas[k].fd.GetNumber();
}
}
assert(atomic_flush_commit_in_progress != nullptr);
Status s;
if (*atomic_flush_commit_in_progress) {
// If the function reaches here, there must be a concurrent thread that
// have already started recording to MANIFEST. Therefore we should just
// return Status::OK and let the othe thread finish writing to MANIFEST on
// our behalf.
return s;
}
// If the function reaches here, the current thread will start writing to
// MANIFEST. It may record to MANIFEST the flush results of other flushes.
*atomic_flush_commit_in_progress = true;
auto comp = [&imm_lists](size_t lh, size_t rh) {
const auto& memlist1 = imm_lists[lh]->current_->memlist_;
const auto& memlist2 = imm_lists[rh]->current_->memlist_;
auto it1 = memlist1.rbegin();
auto it2 = memlist2.rbegin();
return (*it1)->atomic_flush_seqno_ > (*it2)->atomic_flush_seqno_;
};
// The top of the heap is the memtable with smallest atomic_flush_seqno_.
std::priority_queue<size_t, std::vector<size_t>, decltype(comp)> heap(comp);
// Sequence number of the oldest unfinished atomic flush.
SequenceNumber min_unfinished_seqno = kMaxSequenceNumber;
// Populate the heap with first element of each imm iff. it has been
// flushed to storage, i.e. flush_completed_ is true.
size_t num = imm_lists.size();
assert(num == cfds.size());
for (size_t i = 0; i != num; ++i) {
std::list<MemTable*>& memlist = imm_lists[i]->current_->memlist_;
if (memlist.empty()) {
continue;
}
auto it = memlist.rbegin();
if ((*it)->flush_completed_) {
heap.emplace(i);
} else if (min_unfinished_seqno > (*it)->atomic_flush_seqno_) {
min_unfinished_seqno = (*it)->atomic_flush_seqno_;
}
}
while (s.ok() && !heap.empty()) {
autovector<size_t> batch;
SequenceNumber seqno = kMaxSequenceNumber;
// Pop from the heap the memtables that belong to the same atomic flush,
// namely their atomic_flush_seqno_ are equal.
do {
size_t pos = heap.top();
const auto& memlist = imm_lists[pos]->current_->memlist_;
MemTable* mem = *(memlist.rbegin());
if (seqno == kMaxSequenceNumber) {
// First mem in this batch.
seqno = mem->atomic_flush_seqno_;
batch.emplace_back(pos);
heap.pop();
} else if (mem->atomic_flush_seqno_ == seqno) {
// mem has the same atomic_flush_seqno_, thus in the same atomic flush.
batch.emplace_back(pos);
heap.pop();
} else if (mem->atomic_flush_seqno_ > seqno) {
// mem belongs to another atomic flush with higher seqno, break the
// loop.
break;
}
} while (!heap.empty());
if (seqno >= min_unfinished_seqno) {
// If there is an older, unfinished atomic flush, then we should not
// proceed.
TEST_SYNC_POINT_CALLBACK(
"MemTableList::TryInstallMemtableFlushResults:"
"HasOlderUnfinishedAtomicFlush:0",
nullptr);
break;
}
// Found the earliest, complete atomic flush. No earlier atomic flush is
// pending. Therefore ready to record it to the MANIFEST.
uint32_t num_entries = 0;
autovector<ColumnFamilyData*> tmp_cfds;
autovector<const MutableCFOptions*> tmp_mutable_cf_options_list;
std::vector<autovector<MemTable*>> memtables_to_flush;
autovector<autovector<VersionEdit*>> edit_lists;
for (auto pos : batch) {
tmp_cfds.emplace_back(cfds[pos]);
tmp_mutable_cf_options_list.emplace_back(mutable_cf_options_list[pos]);
const auto& memlist = imm_lists[pos]->current_->memlist_;
uint64_t batch_file_number = 0;
autovector<MemTable*> tmp_mems;
autovector<VersionEdit*> edits;
for (auto it = memlist.rbegin(); it != memlist.rend(); ++it) {
MemTable* m = *it;
if (!m->flush_completed_ ||
(it != memlist.rbegin() && m->file_number_ != batch_file_number)) {
break;
}
if (it == memlist.rbegin()) {
batch_file_number = m->file_number_;
edits.push_back(m->GetEdits());
++num_entries;
}
tmp_mems.push_back(m);
}
edit_lists.push_back(edits);
memtables_to_flush.push_back(tmp_mems);
}
TEST_SYNC_POINT_CALLBACK(
"MemTableList::TryInstallMemtableFlushResults:FoundBatchToCommit:0",
&num_entries);
// Mark the version edits as an atomic group
uint32_t remaining = num_entries;
for (auto& edit_list : edit_lists) {
assert(edit_list.size() == 1);
edit_list[0]->MarkAtomicGroup(--remaining);
}
assert(remaining == 0);
size_t batch_sz = batch.size();
assert(batch_sz > 0);
assert(batch_sz == memtables_to_flush.size());
assert(batch_sz == tmp_cfds.size());
assert(batch_sz == edit_lists.size());
if (vset->db_options()->allow_2pc) {
for (size_t i = 0; i != batch_sz; ++i) {
auto& edit_list = edit_lists[i];
assert(!edit_list.empty());
edit_list.back()->SetMinLogNumberToKeep(
PrecomputeMinLogNumberToKeep(vset, *tmp_cfds[i], edit_list,
memtables_to_flush[i], prep_tracker));
}
}
// this can release and reacquire the mutex.
s = vset->LogAndApply(tmp_cfds, tmp_mutable_cf_options_list, edit_lists, mu,
db_directory);
for (const auto pos : batch) {
imm_lists[pos]->InstallNewVersion();
}
if (s.ok()) {
for (size_t i = 0; i != batch_sz; ++i) {
if (tmp_cfds[i]->IsDropped()) {
continue;
}
size_t pos = batch[i];
for (auto m : memtables_to_flush[i]) {
assert(m->file_number_ > 0);
uint64_t mem_id = m->GetID();
ROCKS_LOG_BUFFER(log_buffer,
"[%s] Level-0 commit table #%" PRIu64
": memtable #%" PRIu64 " done",
tmp_cfds[i]->GetName().c_str(), m->file_number_,
mem_id);
imm_lists[pos]->current_->Remove(m, to_delete);
}
}
} else {
for (size_t i = 0; i != batch_sz; ++i) {
size_t pos = batch[i];
for (auto m : memtables_to_flush[i]) {
uint64_t mem_id = m->GetID();
ROCKS_LOG_BUFFER(log_buffer,
"[%s] Level-0 commit table #%" PRIu64
": memtable #%" PRIu64 " failed",
tmp_cfds[i]->GetName().c_str(), m->file_number_,
mem_id);
m->flush_completed_ = false;
m->flush_in_progress_ = false;
m->edit_.Clear();
m->file_number_ = 0;
imm_lists[pos]->num_flush_not_started_++;
}
imm_lists[pos]->imm_flush_needed.store(true, std::memory_order_release);
}
}
// Adjust the heap AFTER installing new MemTableListVersions because the
// compare function 'comp' needs to capture the most up-to-date state of
// imm_lists.
for (auto pos : batch) {
const auto& memlist = imm_lists[pos]->current_->memlist_;
if (!memlist.empty()) {
MemTable* mem = *(memlist.rbegin());
if (mem->flush_completed_) {
heap.emplace(pos);
} else if (min_unfinished_seqno > mem->atomic_flush_seqno_) {
min_unfinished_seqno = mem->atomic_flush_seqno_;
}
}
}
}
*atomic_flush_commit_in_progress = false;
return s;
}
// Returns true if there is at least one memtable on which flush has
// not yet started.
bool MemTableList::IsFlushPending() const {
if ((flush_requested_ && num_flush_not_started_ >= 1) ||
if ((flush_requested_ && num_flush_not_started_ > 0) ||
(num_flush_not_started_ >= min_write_buffer_number_to_merge_)) {
assert(imm_flush_needed.load(std::memory_order_relaxed));
return true;
......@@ -281,12 +504,16 @@ bool MemTableList::IsFlushPending() const {
}
// Returns the memtables that need to be flushed.
void MemTableList::PickMemtablesToFlush(autovector<MemTable*>* ret) {
void MemTableList::PickMemtablesToFlush(const uint64_t* max_memtable_id,
autovector<MemTable*>* ret) {
AutoThreadOperationStageUpdater stage_updater(
ThreadStatus::STAGE_PICK_MEMTABLES_TO_FLUSH);
const auto& memlist = current_->memlist_;
for (auto it = memlist.rbegin(); it != memlist.rend(); ++it) {
MemTable* m = *it;
if (max_memtable_id != nullptr && m->GetID() > *max_memtable_id) {
break;
}
if (!m->flush_in_progress_) {
assert(!m->flush_completed_);
num_flush_not_started_--;
......
......@@ -163,6 +163,18 @@ class MemTableListVersion {
// write thread.)
class MemTableList {
public:
// Commit a successful atomic flush in the manifest file
static Status TryInstallMemtableFlushResults(
autovector<MemTableList*>& imm_lists,
const autovector<ColumnFamilyData*>& cfds,
const autovector<const MutableCFOptions*>& mutable_cf_options_list,
const autovector<const autovector<MemTable*>*>& mems_list,
bool* atomic_flush_commit_in_progress, LogsWithPrepTracker* prep_tracker,
VersionSet* vset, InstrumentedMutex* mu,
const autovector<FileMetaData>& file_meta,
autovector<MemTable*>* to_delete, Directory* db_directory,
LogBuffer* log_buffer);
// A list of memtables.
explicit MemTableList(int min_write_buffer_number_to_merge,
int max_write_buffer_number_to_maintain)
......@@ -201,7 +213,8 @@ class MemTableList {
// Returns the earliest memtables that needs to be flushed. The returned
// memtables are guaranteed to be in the ascending order of created time.
void PickMemtablesToFlush(autovector<MemTable*>* mems);
void PickMemtablesToFlush(const uint64_t* max_memtable_id,
autovector<MemTable*>* mems);
// Reset status of the given memtable list back to pending state so that
// they can get picked up again on the next round of flush.
......@@ -281,7 +294,8 @@ class MemTableList {
// committing in progress
bool commit_in_progress_;
// Requested a flush of all memtables to storage
// Requested a flush of memtables to storage. It's possible to request that
// a subset of memtables be flushed.
bool flush_requested_;
// The current memory usage.
......
此差异已折叠。
......@@ -3064,9 +3064,9 @@ Status VersionSet::ProcessManifestWrites(
// 'datas' is gramatically incorrect. We still use this notation is to indicate
// that this variable represents a collection of column_family_data.
Status VersionSet::LogAndApply(
const std::vector<ColumnFamilyData*>& column_family_datas,
const std::vector<MutableCFOptions>& mutable_cf_options_list,
const std::vector<autovector<VersionEdit*>>& edit_lists,
const autovector<ColumnFamilyData*>& column_family_datas,
const autovector<const MutableCFOptions*>& mutable_cf_options_list,
const autovector<autovector<VersionEdit*>>& edit_lists,
InstrumentedMutex* mu, Directory* db_directory, bool new_descriptor_log,
const ColumnFamilyOptions* new_cf_options) {
mu->AssertHeld();
......@@ -3098,8 +3098,8 @@ Status VersionSet::LogAndApply(
assert(static_cast<size_t>(num_cfds) == edit_lists.size());
}
for (int i = 0; i < num_cfds; ++i) {
writers.emplace_back(mu, column_family_datas[i], mutable_cf_options_list[i],
edit_lists[i]);
writers.emplace_back(mu, column_family_datas[i],
*mutable_cf_options_list[i], edit_lists[i]);
manifest_writers_.push_back(&writers[i]);
}
assert(!writers.empty());
......
......@@ -752,10 +752,14 @@ class VersionSet {
InstrumentedMutex* mu, Directory* db_directory = nullptr,
bool new_descriptor_log = false,
const ColumnFamilyOptions* column_family_options = nullptr) {
std::vector<ColumnFamilyData*> cfds(1, column_family_data);
std::vector<MutableCFOptions> mutable_cf_options_list(1,
mutable_cf_options);
std::vector<autovector<VersionEdit*>> edit_lists(1, {edit});
autovector<ColumnFamilyData*> cfds;
cfds.emplace_back(column_family_data);
autovector<const MutableCFOptions*> mutable_cf_options_list;
mutable_cf_options_list.emplace_back(&mutable_cf_options);
autovector<autovector<VersionEdit*>> edit_lists;
autovector<VersionEdit*> edit_list;
edit_list.emplace_back(edit);
edit_lists.emplace_back(edit_list);
return LogAndApply(cfds, mutable_cf_options_list, edit_lists, mu,
db_directory, new_descriptor_log, column_family_options);
}
......@@ -767,10 +771,12 @@ class VersionSet {
const autovector<VersionEdit*>& edit_list, InstrumentedMutex* mu,
Directory* db_directory = nullptr, bool new_descriptor_log = false,
const ColumnFamilyOptions* column_family_options = nullptr) {
std::vector<ColumnFamilyData*> cfds(1, column_family_data);
std::vector<MutableCFOptions> mutable_cf_options_list(1,
mutable_cf_options);
std::vector<autovector<VersionEdit*>> edit_lists(1, edit_list);
autovector<ColumnFamilyData*> cfds;
cfds.emplace_back(column_family_data);
autovector<const MutableCFOptions*> mutable_cf_options_list;
mutable_cf_options_list.emplace_back(&mutable_cf_options);
autovector<autovector<VersionEdit*>> edit_lists;
edit_lists.emplace_back(edit_list);
return LogAndApply(cfds, mutable_cf_options_list, edit_lists, mu,
db_directory, new_descriptor_log, column_family_options);
}
......@@ -778,12 +784,13 @@ class VersionSet {
// The across-multi-cf batch version. If edit_lists contain more than
// 1 version edits, caller must ensure that no edit in the []list is column
// family manipulation.
Status LogAndApply(const std::vector<ColumnFamilyData*>& cfds,
const std::vector<MutableCFOptions>& mutable_cf_options,
const std::vector<autovector<VersionEdit*>>& edit_lists,
InstrumentedMutex* mu, Directory* db_directory = nullptr,
bool new_descriptor_log = false,
const ColumnFamilyOptions* new_cf_options = nullptr);
Status LogAndApply(
const autovector<ColumnFamilyData*>& cfds,
const autovector<const MutableCFOptions*>& mutable_cf_options_list,
const autovector<autovector<VersionEdit*>>& edit_lists,
InstrumentedMutex* mu, Directory* db_directory = nullptr,
bool new_descriptor_log = false,
const ColumnFamilyOptions* new_cf_options = nullptr);
// Recover the last saved descriptor from persistent storage.
// If read_only == true, Recover() will not complain if some column families
......
......@@ -615,13 +615,19 @@ class ManifestWriterTest : public testing::Test {
TEST_F(ManifestWriterTest, SameColumnFamilyGroupCommit) {
NewDB();
const int kGroupSize = 5;
std::vector<VersionEdit> edits(kGroupSize);
std::vector<ColumnFamilyData*> cfds(kGroupSize, cfds_[0]);
std::vector<MutableCFOptions> all_mutable_cf_options(kGroupSize,
mutable_cf_options_);
std::vector<autovector<VersionEdit*>> edit_lists(kGroupSize);
autovector<VersionEdit> edits;
for (int i = 0; i != kGroupSize; ++i) {
edit_lists[i].emplace_back(&edits[i]);
edits.emplace_back(VersionEdit());
}
autovector<ColumnFamilyData*> cfds;
autovector<const MutableCFOptions*> all_mutable_cf_options;
autovector<autovector<VersionEdit*>> edit_lists;
for (int i = 0; i != kGroupSize; ++i) {
cfds.emplace_back(cfds_[0]);
all_mutable_cf_options.emplace_back(&mutable_cf_options_);
autovector<VersionEdit*> edit_list;
edit_list.emplace_back(&edits[i]);
edit_lists.emplace_back(edit_list);
}
int count = 0;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册