Fix assertion failure in bg flush (#7362)

Summary:
https://github.com/facebook/rocksdb/issues/7340 reports and reproduces an assertion failure caused by a combination of the following:
- atomic flush is disabled.
- a column family can appear multiple times in the flush queue at the same time. This behavior was introduced in release 5.17.

Consequently, it is possible that two flushes race with each other. One bg flush thread flushes all memtables. The other thread calls `FlushMemTableToOutputFile()` afterwards, and hits the assertion error below.

```
  assert(cfd->imm()->NumNotFlushed() != 0);
  assert(cfd->imm()->IsFlushPending());
```

Fix this by reverting the behavior. In non-atomic-flush case, a column family can appear in the flush queue at most once at the same time.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/7362

Test Plan:
make check
Also run stress test successfully for 10 times.
```
make crash_test
```

Reviewed By: ajkr

Differential Revision: D25172996

Pulled By: riversand963

fbshipit-source-id: f1559b6366cc609e961e3fc83fae548f1fad08ce
This commit is contained in:
Yanqin Jin 2020-12-02 09:29:50 -08:00 committed by Facebook GitHub Bot
parent 9e1640403a
commit e062a719cc
10 changed files with 169 additions and 112 deletions

View File

@ -159,10 +159,10 @@ Status DBImpl::FlushMemTableToOutputFile(
FlushJob flush_job( FlushJob flush_job(
dbname_, cfd, immutable_db_options_, mutable_cf_options, dbname_, cfd, immutable_db_options_, mutable_cf_options,
nullptr /* memtable_id */, file_options_for_compaction_, versions_.get(), port::kMaxUint64 /* memtable_id */, file_options_for_compaction_,
&mutex_, &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot, versions_.get(), &mutex_, &shutting_down_, snapshot_seqs,
snapshot_checker, job_context, log_buffer, directories_.GetDbDir(), earliest_write_conflict_snapshot, snapshot_checker, job_context,
GetDataDir(cfd, 0U), log_buffer, directories_.GetDbDir(), GetDataDir(cfd, 0U),
GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_, 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 */, thread_pri, true /* sync_output_directory */, true /* write_manifest */, thread_pri,
@ -313,30 +313,22 @@ Status DBImpl::FlushMemTablesToOutputFiles(
return AtomicFlushMemTablesToOutputFiles( return AtomicFlushMemTablesToOutputFiles(
bg_flush_args, made_progress, job_context, log_buffer, thread_pri); bg_flush_args, made_progress, job_context, log_buffer, thread_pri);
} }
assert(bg_flush_args.size() == 1);
std::vector<SequenceNumber> snapshot_seqs; std::vector<SequenceNumber> snapshot_seqs;
SequenceNumber earliest_write_conflict_snapshot; SequenceNumber earliest_write_conflict_snapshot;
SnapshotChecker* snapshot_checker; SnapshotChecker* snapshot_checker;
GetSnapshotContext(job_context, &snapshot_seqs, GetSnapshotContext(job_context, &snapshot_seqs,
&earliest_write_conflict_snapshot, &snapshot_checker); &earliest_write_conflict_snapshot, &snapshot_checker);
Status status; const auto& bg_flush_arg = bg_flush_args[0];
for (auto& arg : bg_flush_args) { ColumnFamilyData* cfd = bg_flush_arg.cfd_;
ColumnFamilyData* cfd = arg.cfd_; MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions(); SuperVersionContext* superversion_context =
SuperVersionContext* superversion_context = arg.superversion_context_; bg_flush_arg.superversion_context_;
Status s = FlushMemTableToOutputFile( Status s = FlushMemTableToOutputFile(
cfd, mutable_cf_options, made_progress, job_context, cfd, mutable_cf_options, made_progress, job_context, superversion_context,
superversion_context, snapshot_seqs, earliest_write_conflict_snapshot, snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker,
snapshot_checker, log_buffer, thread_pri); log_buffer, thread_pri);
if (!s.ok()) { return s;
status = s;
if (!s.IsShutdownInProgress() && !s.IsColumnFamilyDropped()) {
// At this point, DB is not shutting down, nor is cfd dropped.
// Something is wrong, thus we break out of the loop.
break;
}
}
}
return status;
} }
/* /*
@ -399,7 +391,7 @@ Status DBImpl::AtomicFlushMemTablesToOutputFiles(
all_mutable_cf_options.emplace_back(*cfd->GetLatestMutableCFOptions()); all_mutable_cf_options.emplace_back(*cfd->GetLatestMutableCFOptions());
const MutableCFOptions& mutable_cf_options = all_mutable_cf_options.back(); const MutableCFOptions& mutable_cf_options = all_mutable_cf_options.back();
const uint64_t* max_memtable_id = &(bg_flush_args[i].max_memtable_id_); uint64_t max_memtable_id = bg_flush_args[i].max_memtable_id_;
jobs.emplace_back(new FlushJob( jobs.emplace_back(new FlushJob(
dbname_, cfd, immutable_db_options_, mutable_cf_options, dbname_, cfd, immutable_db_options_, mutable_cf_options,
max_memtable_id, file_options_for_compaction_, versions_.get(), &mutex_, max_memtable_id, file_options_for_compaction_, versions_.get(), &mutex_,
@ -1697,8 +1689,9 @@ void DBImpl::GenerateFlushRequest(const autovector<ColumnFamilyData*>& cfds,
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd, Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
const FlushOptions& flush_options, const FlushOptions& flush_options,
FlushReason flush_reason, bool writes_stopped) { FlushReason flush_reason, bool writes_stopped) {
// This method should not be called if atomic_flush is true.
assert(!immutable_db_options_.atomic_flush);
Status s; Status s;
uint64_t flush_memtable_id = 0;
if (!flush_options.allow_write_stall) { if (!flush_options.allow_write_stall) {
bool flush_needed = true; bool flush_needed = true;
s = WaitUntilFlushWouldNotStallWrites(cfd, &flush_needed); s = WaitUntilFlushWouldNotStallWrites(cfd, &flush_needed);
@ -1708,7 +1701,8 @@ Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
} }
} }
FlushRequest flush_req; autovector<FlushRequest> flush_reqs;
autovector<uint64_t> memtable_ids_to_wait;
{ {
WriteContext context; WriteContext context;
InstrumentedMutexLock guard_lock(&mutex_); InstrumentedMutexLock guard_lock(&mutex_);
@ -1730,11 +1724,13 @@ Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
assert(cfd->imm()->NumNotFlushed() > 0); assert(cfd->imm()->NumNotFlushed() > 0);
} }
} }
const uint64_t flush_memtable_id = port::kMaxUint64;
if (s.ok()) { if (s.ok()) {
if (cfd->imm()->NumNotFlushed() != 0 || !cfd->mem()->IsEmpty() || if (cfd->imm()->NumNotFlushed() != 0 || !cfd->mem()->IsEmpty() ||
!cached_recoverable_state_empty_.load()) { !cached_recoverable_state_empty_.load()) {
flush_memtable_id = cfd->imm()->GetLatestMemTableID(); FlushRequest req{{cfd, flush_memtable_id}};
flush_req.emplace_back(cfd, flush_memtable_id); flush_reqs.emplace_back(std::move(req));
memtable_ids_to_wait.emplace_back(cfd->imm()->GetLatestMemTableID());
} }
if (immutable_db_options_.persist_stats_to_disk && if (immutable_db_options_.persist_stats_to_disk &&
flush_reason != FlushReason::kErrorRecoveryRetryFlush) { flush_reason != FlushReason::kErrorRecoveryRetryFlush) {
@ -1760,15 +1756,19 @@ Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
"to avoid holding old logs", "to avoid holding old logs",
cfd->GetName().c_str()); cfd->GetName().c_str());
s = SwitchMemtable(cfd_stats, &context); s = SwitchMemtable(cfd_stats, &context);
flush_memtable_id = cfd_stats->imm()->GetLatestMemTableID(); FlushRequest req{{cfd_stats, flush_memtable_id}};
flush_req.emplace_back(cfd_stats, flush_memtable_id); flush_reqs.emplace_back(std::move(req));
memtable_ids_to_wait.emplace_back(
cfd->imm()->GetLatestMemTableID());
} }
} }
} }
} }
if (s.ok() && !flush_req.empty()) {
for (auto& elem : flush_req) { if (s.ok() && !flush_reqs.empty()) {
ColumnFamilyData* loop_cfd = elem.first; for (const auto& req : flush_reqs) {
assert(req.size() == 1);
ColumnFamilyData* loop_cfd = req[0].first;
loop_cfd->imm()->FlushRequested(); loop_cfd->imm()->FlushRequested();
} }
// If the caller wants to wait for this flush to complete, it indicates // If the caller wants to wait for this flush to complete, it indicates
@ -1776,12 +1776,15 @@ Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
// other threads which may drop the column family concurrently. // other threads which may drop the column family concurrently.
// Therefore, we increase the cfd's ref count. // Therefore, we increase the cfd's ref count.
if (flush_options.wait) { if (flush_options.wait) {
for (auto& elem : flush_req) { for (const auto& req : flush_reqs) {
ColumnFamilyData* loop_cfd = elem.first; assert(req.size() == 1);
ColumnFamilyData* loop_cfd = req[0].first;
loop_cfd->Ref(); loop_cfd->Ref();
} }
} }
SchedulePendingFlush(flush_req, flush_reason); for (const auto& req : flush_reqs) {
SchedulePendingFlush(req, flush_reason);
}
MaybeScheduleFlushOrCompaction(); MaybeScheduleFlushOrCompaction();
} }
@ -1797,9 +1800,11 @@ Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
if (s.ok() && flush_options.wait) { if (s.ok() && flush_options.wait) {
autovector<ColumnFamilyData*> cfds; autovector<ColumnFamilyData*> cfds;
autovector<const uint64_t*> flush_memtable_ids; autovector<const uint64_t*> flush_memtable_ids;
for (auto& iter : flush_req) { assert(flush_reqs.size() == memtable_ids_to_wait.size());
cfds.push_back(iter.first); for (size_t i = 0; i < flush_reqs.size(); ++i) {
flush_memtable_ids.push_back(&(iter.second)); assert(flush_reqs[i].size() == 1);
cfds.push_back(flush_reqs[i][0].first);
flush_memtable_ids.push_back(&(memtable_ids_to_wait[i]));
} }
s = WaitForFlushMemTables( s = WaitForFlushMemTables(
cfds, flush_memtable_ids, cfds, flush_memtable_ids,
@ -2224,6 +2229,17 @@ DBImpl::FlushRequest DBImpl::PopFirstFromFlushQueue() {
assert(!flush_queue_.empty()); assert(!flush_queue_.empty());
FlushRequest flush_req = flush_queue_.front(); FlushRequest flush_req = flush_queue_.front();
flush_queue_.pop_front(); flush_queue_.pop_front();
if (!immutable_db_options_.atomic_flush) {
assert(flush_req.size() == 1);
}
for (const auto& elem : flush_req) {
if (!immutable_db_options_.atomic_flush) {
ColumnFamilyData* cfd = elem.first;
assert(cfd);
assert(cfd->queued_for_flush());
cfd->set_queued_for_flush(false);
}
}
// TODO: need to unset flush reason? // TODO: need to unset flush reason?
return flush_req; return flush_req;
} }
@ -2256,19 +2272,36 @@ ColumnFamilyData* DBImpl::PickCompactionFromQueue(
void DBImpl::SchedulePendingFlush(const FlushRequest& flush_req, void DBImpl::SchedulePendingFlush(const FlushRequest& flush_req,
FlushReason flush_reason) { FlushReason flush_reason) {
mutex_.AssertHeld();
if (flush_req.empty()) { if (flush_req.empty()) {
return; return;
} }
for (auto& iter : flush_req) { if (!immutable_db_options_.atomic_flush) {
ColumnFamilyData* cfd = iter.first; // For the non-atomic flush case, we never schedule multiple column
cfd->Ref(); // families in the same flush request.
cfd->SetFlushReason(flush_reason); assert(flush_req.size() == 1);
ColumnFamilyData* cfd = flush_req[0].first;
assert(cfd);
if (!cfd->queued_for_flush() && cfd->imm()->IsFlushPending()) {
cfd->Ref();
cfd->set_queued_for_flush(true);
cfd->SetFlushReason(flush_reason);
++unscheduled_flushes_;
flush_queue_.push_back(flush_req);
}
} else {
for (auto& iter : flush_req) {
ColumnFamilyData* cfd = iter.first;
cfd->Ref();
cfd->SetFlushReason(flush_reason);
}
++unscheduled_flushes_;
flush_queue_.push_back(flush_req);
} }
++unscheduled_flushes_;
flush_queue_.push_back(flush_req);
} }
void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) { void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) {
mutex_.AssertHeld();
if (!cfd->queued_for_compaction() && cfd->NeedsCompaction()) { if (!cfd->queued_for_compaction() && cfd->NeedsCompaction()) {
AddToCompactionQueue(cfd); AddToCompactionQueue(cfd);
++unscheduled_compactions_; ++unscheduled_compactions_;

View File

@ -1335,10 +1335,17 @@ Status DBImpl::SwitchWAL(WriteContext* write_context) {
} }
for (auto cfd : cfds) { for (auto cfd : cfds) {
cfd->imm()->FlushRequested(); cfd->imm()->FlushRequested();
if (!immutable_db_options_.atomic_flush) {
FlushRequest flush_req;
GenerateFlushRequest({cfd}, &flush_req);
SchedulePendingFlush(flush_req, FlushReason::kWriteBufferManager);
}
}
if (immutable_db_options_.atomic_flush) {
FlushRequest flush_req;
GenerateFlushRequest(cfds, &flush_req);
SchedulePendingFlush(flush_req, FlushReason::kWriteBufferManager);
} }
FlushRequest flush_req;
GenerateFlushRequest(cfds, &flush_req);
SchedulePendingFlush(flush_req, FlushReason::kWriteBufferManager);
MaybeScheduleFlushOrCompaction(); MaybeScheduleFlushOrCompaction();
} }
return status; return status;
@ -1414,10 +1421,17 @@ Status DBImpl::HandleWriteBufferFull(WriteContext* write_context) {
} }
for (const auto cfd : cfds) { for (const auto cfd : cfds) {
cfd->imm()->FlushRequested(); cfd->imm()->FlushRequested();
if (!immutable_db_options_.atomic_flush) {
FlushRequest flush_req;
GenerateFlushRequest({cfd}, &flush_req);
SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull);
}
}
if (immutable_db_options_.atomic_flush) {
FlushRequest flush_req;
GenerateFlushRequest(cfds, &flush_req);
SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull);
} }
FlushRequest flush_req;
GenerateFlushRequest(cfds, &flush_req);
SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull);
MaybeScheduleFlushOrCompaction(); MaybeScheduleFlushOrCompaction();
} }
return status; return status;
@ -1641,10 +1655,16 @@ Status DBImpl::ScheduleFlushes(WriteContext* context) {
if (status.ok()) { if (status.ok()) {
if (immutable_db_options_.atomic_flush) { if (immutable_db_options_.atomic_flush) {
AssignAtomicFlushSeq(cfds); AssignAtomicFlushSeq(cfds);
FlushRequest flush_req;
GenerateFlushRequest(cfds, &flush_req);
SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull);
} else {
for (auto* cfd : cfds) {
FlushRequest flush_req;
GenerateFlushRequest({cfd}, &flush_req);
SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull);
}
} }
FlushRequest flush_req;
GenerateFlushRequest(cfds, &flush_req);
SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull);
MaybeScheduleFlushOrCompaction(); MaybeScheduleFlushOrCompaction();
} }
return status; return status;

View File

@ -748,10 +748,12 @@ TEST_F(DBSecondaryTest, SwitchWALMultiColumnFamilies) {
} }
}; };
for (int k = 0; k != 8; ++k) { for (int k = 0; k != 8; ++k) {
ASSERT_OK( for (int j = 0; j < 2; ++j) {
Put(0 /*cf*/, "key" + std::to_string(k), "value" + std::to_string(k))); ASSERT_OK(Put(0 /*cf*/, "key" + std::to_string(k),
ASSERT_OK( "value" + std::to_string(k)));
Put(1 /*cf*/, "key" + std::to_string(k), "value" + std::to_string(k))); ASSERT_OK(Put(1 /*cf*/, "key" + std::to_string(k),
"value" + std::to_string(k)));
}
TEST_SYNC_POINT( TEST_SYNC_POINT(
"DBSecondaryTest::SwitchWALMultipleColumnFamilies:BeforeCatchUp"); "DBSecondaryTest::SwitchWALMultipleColumnFamilies:BeforeCatchUp");
ASSERT_OK(db_secondary_->TryCatchUpWithPrimary()); ASSERT_OK(db_secondary_->TryCatchUpWithPrimary());

View File

@ -1646,7 +1646,7 @@ TEST_F(DBWALTest, RestoreTotalLogSizeAfterRecoverWithoutFlush) {
void OnFlushBegin(DB* /*db*/, const FlushJobInfo& flush_job_info) override { void OnFlushBegin(DB* /*db*/, const FlushJobInfo& flush_job_info) override {
count++; count++;
assert(FlushReason::kWriteBufferManager == flush_job_info.flush_reason); ASSERT_EQ(FlushReason::kWriteBufferManager, flush_job_info.flush_reason);
} }
}; };
std::shared_ptr<TestFlushListener> test_listener = std::shared_ptr<TestFlushListener> test_listener =
@ -1690,7 +1690,9 @@ TEST_F(DBWALTest, RestoreTotalLogSizeAfterRecoverWithoutFlush) {
1 * kMB); 1 * kMB);
// Write one more key to trigger flush. // Write one more key to trigger flush.
ASSERT_OK(Put(0, "foo", "v2")); ASSERT_OK(Put(0, "foo", "v2"));
dbfull()->TEST_WaitForFlushMemTable(); for (auto* h : handles_) {
ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable(h));
}
// Flushed two column families. // Flushed two column families.
ASSERT_EQ(2, test_listener->count.load()); ASSERT_EQ(2, test_listener->count.load());
} }

View File

@ -80,22 +80,24 @@ const char* GetFlushReasonString (FlushReason flush_reason) {
} }
} }
FlushJob::FlushJob( FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd,
const std::string& dbname, ColumnFamilyData* cfd, const ImmutableDBOptions& db_options,
const ImmutableDBOptions& db_options, const MutableCFOptions& mutable_cf_options,
const MutableCFOptions& mutable_cf_options, const uint64_t* max_memtable_id, uint64_t max_memtable_id, const FileOptions& file_options,
const FileOptions& file_options, VersionSet* versions, VersionSet* versions, InstrumentedMutex* db_mutex,
InstrumentedMutex* db_mutex, std::atomic<bool>* shutting_down, std::atomic<bool>* shutting_down,
std::vector<SequenceNumber> existing_snapshots, std::vector<SequenceNumber> existing_snapshots,
SequenceNumber earliest_write_conflict_snapshot, SequenceNumber earliest_write_conflict_snapshot,
SnapshotChecker* snapshot_checker, JobContext* job_context, SnapshotChecker* snapshot_checker, JobContext* job_context,
LogBuffer* log_buffer, FSDirectory* db_directory, LogBuffer* log_buffer, FSDirectory* db_directory,
FSDirectory* output_file_directory, CompressionType output_compression, FSDirectory* output_file_directory,
Statistics* stats, EventLogger* event_logger, bool measure_io_stats, CompressionType output_compression, Statistics* stats,
const bool sync_output_directory, const bool write_manifest, EventLogger* event_logger, bool measure_io_stats,
Env::Priority thread_pri, const std::shared_ptr<IOTracer>& io_tracer, const bool sync_output_directory, const bool write_manifest,
const std::string& db_id, const std::string& db_session_id, Env::Priority thread_pri,
std::string full_history_ts_low) const std::shared_ptr<IOTracer>& io_tracer,
const std::string& db_id, const std::string& db_session_id,
std::string full_history_ts_low)
: dbname_(dbname), : dbname_(dbname),
db_id_(db_id), db_id_(db_id),
db_session_id_(db_session_id), db_session_id_(db_session_id),

View File

@ -60,10 +60,9 @@ class FlushJob {
// IMPORTANT: mutable_cf_options needs to be alive while FlushJob is alive // IMPORTANT: mutable_cf_options needs to be alive while FlushJob is alive
FlushJob(const std::string& dbname, ColumnFamilyData* cfd, FlushJob(const std::string& dbname, ColumnFamilyData* cfd,
const ImmutableDBOptions& db_options, const ImmutableDBOptions& db_options,
const MutableCFOptions& mutable_cf_options, const MutableCFOptions& mutable_cf_options, uint64_t max_memtable_id,
const uint64_t* max_memtable_id, const FileOptions& file_options, const FileOptions& file_options, VersionSet* versions,
VersionSet* versions, InstrumentedMutex* db_mutex, InstrumentedMutex* db_mutex, std::atomic<bool>* shutting_down,
std::atomic<bool>* shutting_down,
std::vector<SequenceNumber> existing_snapshots, std::vector<SequenceNumber> existing_snapshots,
SequenceNumber earliest_write_conflict_snapshot, SequenceNumber earliest_write_conflict_snapshot,
SnapshotChecker* snapshot_checker, JobContext* job_context, SnapshotChecker* snapshot_checker, JobContext* job_context,
@ -110,12 +109,11 @@ class FlushJob {
ColumnFamilyData* cfd_; ColumnFamilyData* cfd_;
const ImmutableDBOptions& db_options_; const ImmutableDBOptions& db_options_;
const MutableCFOptions& mutable_cf_options_; const MutableCFOptions& mutable_cf_options_;
// Pointer to a variable storing the largest memtable id to flush in this // A variable storing the largest memtable id to flush in this
// flush job. RocksDB uses this variable to select the memtables to flush in // 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 // 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 // equal to max_memtable_id_ will be selected for flush.
// memtables in the column family will be selected. uint64_t max_memtable_id_;
const uint64_t* max_memtable_id_;
const FileOptions file_options_; const FileOptions file_options_;
VersionSet* versions_; VersionSet* versions_;
InstrumentedMutex* db_mutex_; InstrumentedMutex* db_mutex_;

View File

@ -158,7 +158,7 @@ TEST_F(FlushJobTest, Empty) {
SnapshotChecker* snapshot_checker = nullptr; // not relavant SnapshotChecker* snapshot_checker = nullptr; // not relavant
FlushJob flush_job( FlushJob flush_job(
dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_,
*cfd->GetLatestMutableCFOptions(), nullptr /* memtable_id */, *cfd->GetLatestMutableCFOptions(), port::kMaxUint64 /* memtable_id */,
env_options_, versions_.get(), &mutex_, &shutting_down_, {}, env_options_, versions_.get(), &mutex_, &shutting_down_, {},
kMaxSequenceNumber, snapshot_checker, &job_context, nullptr, nullptr, kMaxSequenceNumber, snapshot_checker, &job_context, nullptr, nullptr,
nullptr, kNoCompression, nullptr, &event_logger, false, nullptr, kNoCompression, nullptr, &event_logger, false,
@ -240,7 +240,7 @@ TEST_F(FlushJobTest, NonEmpty) {
SnapshotChecker* snapshot_checker = nullptr; // not relavant SnapshotChecker* snapshot_checker = nullptr; // not relavant
FlushJob flush_job( FlushJob flush_job(
dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_,
*cfd->GetLatestMutableCFOptions(), nullptr /* memtable_id */, *cfd->GetLatestMutableCFOptions(), port::kMaxUint64 /* memtable_id */,
env_options_, versions_.get(), &mutex_, &shutting_down_, {}, env_options_, versions_.get(), &mutex_, &shutting_down_, {},
kMaxSequenceNumber, snapshot_checker, &job_context, nullptr, nullptr, kMaxSequenceNumber, snapshot_checker, &job_context, nullptr, nullptr,
nullptr, kNoCompression, db_options_.statistics.get(), &event_logger, nullptr, kNoCompression, db_options_.statistics.get(), &event_logger,
@ -302,7 +302,7 @@ TEST_F(FlushJobTest, FlushMemTablesSingleColumnFamily) {
uint64_t flush_memtable_id = smallest_memtable_id + num_mems_to_flush - 1; uint64_t flush_memtable_id = smallest_memtable_id + num_mems_to_flush - 1;
FlushJob flush_job( FlushJob flush_job(
dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_,
*cfd->GetLatestMutableCFOptions(), &flush_memtable_id, env_options_, *cfd->GetLatestMutableCFOptions(), flush_memtable_id, env_options_,
versions_.get(), &mutex_, &shutting_down_, {}, kMaxSequenceNumber, versions_.get(), &mutex_, &shutting_down_, {}, kMaxSequenceNumber,
snapshot_checker, &job_context, nullptr, nullptr, nullptr, kNoCompression, snapshot_checker, &job_context, nullptr, nullptr, nullptr, kNoCompression,
db_options_.statistics.get(), &event_logger, true, db_options_.statistics.get(), &event_logger, true,
@ -374,7 +374,7 @@ TEST_F(FlushJobTest, FlushMemtablesMultipleColumnFamilies) {
std::vector<SequenceNumber> snapshot_seqs; std::vector<SequenceNumber> snapshot_seqs;
flush_jobs.emplace_back(new FlushJob( flush_jobs.emplace_back(new FlushJob(
dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(), dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(),
&memtable_ids[k], env_options_, versions_.get(), &mutex_, memtable_ids[k], env_options_, versions_.get(), &mutex_,
&shutting_down_, snapshot_seqs, kMaxSequenceNumber, snapshot_checker, &shutting_down_, snapshot_seqs, kMaxSequenceNumber, snapshot_checker,
&job_context, nullptr, nullptr, nullptr, kNoCompression, &job_context, nullptr, nullptr, nullptr, kNoCompression,
db_options_.statistics.get(), &event_logger, true, db_options_.statistics.get(), &event_logger, true,
@ -491,7 +491,7 @@ TEST_F(FlushJobTest, Snapshots) {
SnapshotChecker* snapshot_checker = nullptr; // not relavant SnapshotChecker* snapshot_checker = nullptr; // not relavant
FlushJob flush_job( FlushJob flush_job(
dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_,
*cfd->GetLatestMutableCFOptions(), nullptr /* memtable_id */, *cfd->GetLatestMutableCFOptions(), port::kMaxUint64 /* memtable_id */,
env_options_, versions_.get(), &mutex_, &shutting_down_, snapshots, env_options_, versions_.get(), &mutex_, &shutting_down_, snapshots,
kMaxSequenceNumber, snapshot_checker, &job_context, nullptr, nullptr, kMaxSequenceNumber, snapshot_checker, &job_context, nullptr, nullptr,
nullptr, kNoCompression, db_options_.statistics.get(), &event_logger, nullptr, kNoCompression, db_options_.statistics.get(), &event_logger,
@ -558,8 +558,8 @@ TEST_F(FlushJobTimestampTest, AllKeysExpired) {
PutFixed64(&full_history_ts_low, std::numeric_limits<uint64_t>::max()); PutFixed64(&full_history_ts_low, std::numeric_limits<uint64_t>::max());
FlushJob flush_job( FlushJob flush_job(
dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(), dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(),
nullptr /* memtable_id */, env_options_, versions_.get(), &mutex_, port::kMaxUint64 /* memtable_id */, env_options_, versions_.get(),
&shutting_down_, snapshots, kMaxSequenceNumber, snapshot_checker, &mutex_, &shutting_down_, snapshots, kMaxSequenceNumber, snapshot_checker,
&job_context, nullptr, nullptr, nullptr, kNoCompression, &job_context, nullptr, nullptr, nullptr, kNoCompression,
db_options_.statistics.get(), &event_logger, true, db_options_.statistics.get(), &event_logger, true,
true /* sync_output_directory */, true /* write_manifest */, true /* sync_output_directory */, true /* write_manifest */,
@ -609,8 +609,8 @@ TEST_F(FlushJobTimestampTest, NoKeyExpired) {
PutFixed64(&full_history_ts_low, 0); PutFixed64(&full_history_ts_low, 0);
FlushJob flush_job( FlushJob flush_job(
dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(), dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(),
nullptr /* memtable_id */, env_options_, versions_.get(), &mutex_, port::kMaxUint64 /* memtable_id */, env_options_, versions_.get(),
&shutting_down_, snapshots, kMaxSequenceNumber, snapshot_checker, &mutex_, &shutting_down_, snapshots, kMaxSequenceNumber, snapshot_checker,
&job_context, nullptr, nullptr, nullptr, kNoCompression, &job_context, nullptr, nullptr, nullptr, kNoCompression,
db_options_.statistics.get(), &event_logger, true, db_options_.statistics.get(), &event_logger, true,
true /* sync_output_directory */, true /* write_manifest */, true /* sync_output_directory */, true /* write_manifest */,

View File

@ -334,7 +334,7 @@ bool MemTableList::IsFlushPending() const {
} }
// Returns the memtables that need to be flushed. // Returns the memtables that need to be flushed.
void MemTableList::PickMemtablesToFlush(const uint64_t* max_memtable_id, void MemTableList::PickMemtablesToFlush(uint64_t max_memtable_id,
autovector<MemTable*>* ret) { autovector<MemTable*>* ret) {
AutoThreadOperationStageUpdater stage_updater( AutoThreadOperationStageUpdater stage_updater(
ThreadStatus::STAGE_PICK_MEMTABLES_TO_FLUSH); ThreadStatus::STAGE_PICK_MEMTABLES_TO_FLUSH);
@ -345,7 +345,7 @@ void MemTableList::PickMemtablesToFlush(const uint64_t* max_memtable_id,
if (!atomic_flush && m->atomic_flush_seqno_ != kMaxSequenceNumber) { if (!atomic_flush && m->atomic_flush_seqno_ != kMaxSequenceNumber) {
atomic_flush = true; atomic_flush = true;
} }
if (max_memtable_id != nullptr && m->GetID() > *max_memtable_id) { if (m->GetID() > max_memtable_id) {
break; break;
} }
if (!m->flush_in_progress_) { if (!m->flush_in_progress_) {

View File

@ -251,7 +251,7 @@ class MemTableList {
// Returns the earliest memtables that needs to be flushed. The returned // Returns the earliest memtables that needs to be flushed. The returned
// memtables are guaranteed to be in the ascending order of created time. // memtables are guaranteed to be in the ascending order of created time.
void PickMemtablesToFlush(const uint64_t* max_memtable_id, void PickMemtablesToFlush(uint64_t max_memtable_id,
autovector<MemTable*>* mems); autovector<MemTable*>* mems);
// Reset status of the given memtable list back to pending state so that // Reset status of the given memtable list back to pending state so that

View File

@ -199,7 +199,7 @@ TEST_F(MemTableListTest, Empty) {
ASSERT_FALSE(list.IsFlushPending()); ASSERT_FALSE(list.IsFlushPending());
autovector<MemTable*> mems; autovector<MemTable*> mems;
list.PickMemtablesToFlush(nullptr /* memtable_id */, &mems); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &mems);
ASSERT_EQ(0, mems.size()); ASSERT_EQ(0, mems.size());
autovector<MemTable*> to_delete; autovector<MemTable*> to_delete;
@ -399,7 +399,7 @@ TEST_F(MemTableListTest, GetFromHistoryTest) {
// Flush this memtable from the list. // Flush this memtable from the list.
// (It will then be a part of the memtable history). // (It will then be a part of the memtable history).
autovector<MemTable*> to_flush; autovector<MemTable*> to_flush;
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush);
ASSERT_EQ(1, to_flush.size()); ASSERT_EQ(1, to_flush.size());
MutableCFOptions mutable_cf_options(options); MutableCFOptions mutable_cf_options(options);
@ -451,7 +451,7 @@ TEST_F(MemTableListTest, GetFromHistoryTest) {
ASSERT_EQ(0, to_delete.size()); ASSERT_EQ(0, to_delete.size());
to_flush.clear(); to_flush.clear();
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush);
ASSERT_EQ(1, to_flush.size()); ASSERT_EQ(1, to_flush.size());
// Flush second memtable // Flush second memtable
@ -567,7 +567,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
ASSERT_FALSE(list.IsFlushPending()); ASSERT_FALSE(list.IsFlushPending());
ASSERT_FALSE(list.imm_flush_needed.load(std::memory_order_acquire)); ASSERT_FALSE(list.imm_flush_needed.load(std::memory_order_acquire));
autovector<MemTable*> to_flush; autovector<MemTable*> to_flush;
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush);
ASSERT_EQ(0, to_flush.size()); ASSERT_EQ(0, to_flush.size());
// Request a flush even though there is nothing to flush // Request a flush even though there is nothing to flush
@ -576,7 +576,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
ASSERT_FALSE(list.imm_flush_needed.load(std::memory_order_acquire)); ASSERT_FALSE(list.imm_flush_needed.load(std::memory_order_acquire));
// Attempt to 'flush' to clear request for flush // Attempt to 'flush' to clear request for flush
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush);
ASSERT_EQ(0, to_flush.size()); ASSERT_EQ(0, to_flush.size());
ASSERT_FALSE(list.IsFlushPending()); ASSERT_FALSE(list.IsFlushPending());
ASSERT_FALSE(list.imm_flush_needed.load(std::memory_order_acquire)); ASSERT_FALSE(list.imm_flush_needed.load(std::memory_order_acquire));
@ -600,7 +600,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
ASSERT_TRUE(list.imm_flush_needed.load(std::memory_order_acquire)); ASSERT_TRUE(list.imm_flush_needed.load(std::memory_order_acquire));
// Pick tables to flush // Pick tables to flush
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush);
ASSERT_EQ(2, to_flush.size()); ASSERT_EQ(2, to_flush.size());
ASSERT_EQ(2, list.NumNotFlushed()); ASSERT_EQ(2, list.NumNotFlushed());
ASSERT_FALSE(list.IsFlushPending()); ASSERT_FALSE(list.IsFlushPending());
@ -621,7 +621,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
ASSERT_EQ(0, to_delete.size()); ASSERT_EQ(0, to_delete.size());
// Pick tables to flush // Pick tables to flush
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush);
ASSERT_EQ(3, to_flush.size()); ASSERT_EQ(3, to_flush.size());
ASSERT_EQ(3, list.NumNotFlushed()); ASSERT_EQ(3, list.NumNotFlushed());
ASSERT_FALSE(list.IsFlushPending()); ASSERT_FALSE(list.IsFlushPending());
@ -629,7 +629,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
// Pick tables to flush again // Pick tables to flush again
autovector<MemTable*> to_flush2; autovector<MemTable*> to_flush2;
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush2); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush2);
ASSERT_EQ(0, to_flush2.size()); ASSERT_EQ(0, to_flush2.size());
ASSERT_EQ(3, list.NumNotFlushed()); ASSERT_EQ(3, list.NumNotFlushed());
ASSERT_FALSE(list.IsFlushPending()); ASSERT_FALSE(list.IsFlushPending());
@ -647,7 +647,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
ASSERT_TRUE(list.imm_flush_needed.load(std::memory_order_acquire)); ASSERT_TRUE(list.imm_flush_needed.load(std::memory_order_acquire));
// Pick tables to flush again // Pick tables to flush again
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush2); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush2);
ASSERT_EQ(1, to_flush2.size()); ASSERT_EQ(1, to_flush2.size());
ASSERT_EQ(4, list.NumNotFlushed()); ASSERT_EQ(4, list.NumNotFlushed());
ASSERT_FALSE(list.IsFlushPending()); ASSERT_FALSE(list.IsFlushPending());
@ -668,7 +668,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
ASSERT_EQ(0, to_delete.size()); ASSERT_EQ(0, to_delete.size());
// Pick tables to flush // Pick tables to flush
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush);
// Should pick 4 of 5 since 1 table has been picked in to_flush2 // Should pick 4 of 5 since 1 table has been picked in to_flush2
ASSERT_EQ(4, to_flush.size()); ASSERT_EQ(4, to_flush.size());
ASSERT_EQ(5, list.NumNotFlushed()); ASSERT_EQ(5, list.NumNotFlushed());
@ -677,7 +677,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
// Pick tables to flush again // Pick tables to flush again
autovector<MemTable*> to_flush3; autovector<MemTable*> to_flush3;
list.PickMemtablesToFlush(nullptr /* memtable_id */, &to_flush3); list.PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */, &to_flush3);
ASSERT_EQ(0, to_flush3.size()); // nothing not in progress of being flushed ASSERT_EQ(0, to_flush3.size()); // nothing not in progress of being flushed
ASSERT_EQ(5, list.NumNotFlushed()); ASSERT_EQ(5, list.NumNotFlushed());
ASSERT_FALSE(list.IsFlushPending()); ASSERT_FALSE(list.IsFlushPending());
@ -738,7 +738,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
autovector<MemTable*> to_flush4; autovector<MemTable*> to_flush4;
list.FlushRequested(); list.FlushRequested();
ASSERT_TRUE(list.HasFlushRequested()); ASSERT_TRUE(list.HasFlushRequested());
list.PickMemtablesToFlush(&memtable_id, &to_flush4); list.PickMemtablesToFlush(memtable_id, &to_flush4);
ASSERT_TRUE(to_flush4.empty()); ASSERT_TRUE(to_flush4.empty());
ASSERT_EQ(1, list.NumNotFlushed()); ASSERT_EQ(1, list.NumNotFlushed());
ASSERT_TRUE(list.imm_flush_needed.load(std::memory_order_acquire)); ASSERT_TRUE(list.imm_flush_needed.load(std::memory_order_acquire));
@ -749,7 +749,7 @@ TEST_F(MemTableListTest, FlushPendingTest) {
// equal to 5. Therefore, only tables[5] will be selected. // equal to 5. Therefore, only tables[5] will be selected.
memtable_id = 5; memtable_id = 5;
list.FlushRequested(); list.FlushRequested();
list.PickMemtablesToFlush(&memtable_id, &to_flush4); list.PickMemtablesToFlush(memtable_id, &to_flush4);
ASSERT_EQ(1, static_cast<int>(to_flush4.size())); ASSERT_EQ(1, static_cast<int>(to_flush4.size()));
ASSERT_EQ(1, list.NumNotFlushed()); ASSERT_EQ(1, list.NumNotFlushed());
ASSERT_FALSE(list.imm_flush_needed.load(std::memory_order_acquire)); ASSERT_FALSE(list.imm_flush_needed.load(std::memory_order_acquire));
@ -841,7 +841,8 @@ TEST_F(MemTableListTest, AtomicFlusTest) {
auto* list = lists[i]; auto* list = lists[i];
ASSERT_FALSE(list->IsFlushPending()); ASSERT_FALSE(list->IsFlushPending());
ASSERT_FALSE(list->imm_flush_needed.load(std::memory_order_acquire)); ASSERT_FALSE(list->imm_flush_needed.load(std::memory_order_acquire));
list->PickMemtablesToFlush(nullptr /* memtable_id */, &flush_candidates[i]); list->PickMemtablesToFlush(port::kMaxUint64 /* memtable_id */,
&flush_candidates[i]);
ASSERT_EQ(0, flush_candidates[i].size()); ASSERT_EQ(0, flush_candidates[i].size());
} }
// Request flush even though there is nothing to flush // Request flush even though there is nothing to flush
@ -871,8 +872,7 @@ TEST_F(MemTableListTest, AtomicFlusTest) {
// Pick memtables to flush // Pick memtables to flush
for (auto i = 0; i != num_cfs; ++i) { for (auto i = 0; i != num_cfs; ++i) {
flush_candidates[i].clear(); flush_candidates[i].clear();
lists[i]->PickMemtablesToFlush(&flush_memtable_ids[i], lists[i]->PickMemtablesToFlush(flush_memtable_ids[i], &flush_candidates[i]);
&flush_candidates[i]);
ASSERT_EQ(flush_memtable_ids[i] - 0 + 1, ASSERT_EQ(flush_memtable_ids[i] - 0 + 1,
static_cast<uint64_t>(flush_candidates[i].size())); static_cast<uint64_t>(flush_candidates[i].size()));
} }