Skip to content

Commit

Permalink
Refactor, clean up, fixes, and more testing for SeqnoToTimeMapping (f…
Browse files Browse the repository at this point in the history
…acebook#11905)

Summary:
This change is before a planned DBImpl change to ensure all sufficiently recent sequence numbers since Open are covered by SeqnoToTimeMapping (bug fix with existing test work-arounds). **Intended follow-up**

However, I found enough issues with SeqnoToTimeMapping to warrant this PR first, including very small fixes in DB implementation related to API contract of SeqnoToTimeMapping.

Functional fixes / changes:
* This fixes some mishandling of boundary cases. For example, if the user decides to stop writing to DB, the last written sequence number would perpetually have its write time updated to "now" and would always be ineligible for migration to cold tier. Part of the problem is that the SeqnoToTimeMapping would return a seqno known to have been written before (immediately or otherwise) the requested time, but compaction_job.cc would include that seqno in the preserve/exclude set. That is fixed (in part) by adding one in compaction_job.cc
* That problem was worse because a whole range of seqnos could be updated perpetually with new times in SeqnoToTimeMapping::Append (if no writes to DB). That logic was apparently optimized for GetOldestApproximateTime (now GetProximalTimeBeforeSeqno), which is not used in production, to the detriment of GetOldestSequenceNum (now GetProximalSeqnoBeforeTime), which is used in production. (Perhaps plans changed during development?) This is fixed in Append to optimize for accuracy of GetProximalSeqnoBeforeTime. (Unit tests added and updated.)
* Related: SeqnoToTimeMapping did not have a clear contract about the relationships between seqnos and times, just the idea of a rough correspondence. Now the class description makes it clear that the write time of each recorded seqno comes before or at the associated time, to support getting best results for GetProximalSeqnoBeforeTime. And this makes it easier to make clear the contract of each API function.
  * Update `DBImpl::RecordSeqnoToTimeMapping()` to follow this ordering in gathering samples.

Some part of these changes has required an expanded test work-around for the problem (see intended follow-up above) that the DB does not immediately ensure recent seqnos are covered by its mapping. These work-arounds will be removed with that planned work.

An apparent compaction bug is revealed in
PrecludeLastLevelTest::RangeDelsCauseFileEndpointsToOverlap, so that test is disabled. Filed GitHub issue facebook#11909

Cosmetic / code safety things (not exhaustive):
* Fix some confusing names.
  * `seqno_time_mapping` was used inconsistently in places. Now just `seqno_to_time_mapping` to correspond to class name.
  * Rename confusing `GetOldestSequenceNum` -> `GetProximalSeqnoBeforeTime` and `GetOldestApproximateTime` -> `GetProximalTimeBeforeSeqno`. Part of the motivation is that our times and seqnos here have the same underlying type, so we want to be clear about which is expected where to avoid mixing.
  * Rename `kUnknownSeqnoTime` to `kUnknownTimeBeforeAll` because the value is a bad choice for unknown if we ever add ProximalAfterBlah functions.
  * Arithmetic on SeqnoTimePair doesn't make sense except for delta encoding, so use better names / APIs with that in mind.
  * (OMG) Don't allow direct comparison between SeqnoTimePair and SequenceNumber. (There is no checking that it isn't compared against time by accident.)
  * A field name essentially matching the containing class name is a confusing pattern (`seqno_time_mapping_`).
  * Wrap calls to confusing (but useful) upper_bound and lower_bound functions to have clearer names and more code reuse.

Pull Request resolved: facebook#11905

Test Plan: GetOldestSequenceNum (now GetProximalSeqnoBeforeTime) and TruncateOldEntries were lacking unit tests, despite both being used in production (experimental feature). Added those and expanded others.

Reviewed By: jowlyzhang

Differential Revision: D49755592

Pulled By: pdillinger

fbshipit-source-id: f72a3baac74d24b963c77e538bba89a7fc8dce51
  • Loading branch information
pdillinger authored and facebook-github-bot committed Sep 29, 2023
1 parent 2cfe53e commit 02443dd
Show file tree
Hide file tree
Showing 17 changed files with 451 additions and 203 deletions.
6 changes: 3 additions & 3 deletions db/builder.cc
Original file line number Diff line number Diff line change
Expand Up @@ -294,12 +294,12 @@ Status BuildTable(
if (!s.ok() || empty) {
builder->Abandon();
} else {
std::string seqno_time_mapping_str;
std::string seqno_to_time_mapping_str;
seqno_to_time_mapping.Encode(
seqno_time_mapping_str, meta->fd.smallest_seqno,
seqno_to_time_mapping_str, meta->fd.smallest_seqno,
meta->fd.largest_seqno, meta->file_creation_time);
builder->SetSeqnoTimeTableProperties(
seqno_time_mapping_str,
seqno_to_time_mapping_str,
ioptions.compaction_style == CompactionStyle::kCompactionStyleFIFO
? meta->file_creation_time
: meta->oldest_ancester_time);
Expand Down
26 changes: 16 additions & 10 deletions db/compaction/compaction_job.cc
Original file line number Diff line number Diff line change
Expand Up @@ -288,23 +288,23 @@ void CompactionJob::Prepare() {

if (preserve_time_duration > 0) {
const ReadOptions read_options(Env::IOActivity::kCompaction);
// setup seqno_time_mapping_
seqno_time_mapping_.SetMaxTimeDuration(preserve_time_duration);
// setup seqno_to_time_mapping_
seqno_to_time_mapping_.SetMaxTimeDuration(preserve_time_duration);
for (const auto& each_level : *c->inputs()) {
for (const auto& fmd : each_level.files) {
std::shared_ptr<const TableProperties> tp;
Status s =
cfd->current()->GetTableProperties(read_options, &tp, fmd, nullptr);
if (s.ok()) {
seqno_time_mapping_.Add(tp->seqno_to_time_mapping)
seqno_to_time_mapping_.Add(tp->seqno_to_time_mapping)
.PermitUncheckedError();
seqno_time_mapping_.Add(fmd->fd.smallest_seqno,
fmd->oldest_ancester_time);
seqno_to_time_mapping_.Add(fmd->fd.smallest_seqno,
fmd->oldest_ancester_time);
}
}
}

auto status = seqno_time_mapping_.Sort();
auto status = seqno_to_time_mapping_.Sort();
if (!status.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Invalid sequence number to time mapping: Status: %s",
Expand All @@ -320,13 +320,17 @@ void CompactionJob::Prepare() {
preserve_time_min_seqno_ = 0;
preclude_last_level_min_seqno_ = 0;
} else {
seqno_time_mapping_.TruncateOldEntries(_current_time);
seqno_to_time_mapping_.TruncateOldEntries(_current_time);
uint64_t preserve_time =
static_cast<uint64_t>(_current_time) > preserve_time_duration
? _current_time - preserve_time_duration
: 0;
// GetProximalSeqnoBeforeTime tells us the last seqno known to have been
// written at or before the given time. + 1 to get the minimum we should
// preserve without excluding anything that might have been written on or
// after the given time.
preserve_time_min_seqno_ =
seqno_time_mapping_.GetOldestSequenceNum(preserve_time);
seqno_to_time_mapping_.GetProximalSeqnoBeforeTime(preserve_time) + 1;
if (c->immutable_options()->preclude_last_level_data_seconds > 0) {
uint64_t preclude_last_level_time =
static_cast<uint64_t>(_current_time) >
Expand All @@ -335,7 +339,9 @@ void CompactionJob::Prepare() {
c->immutable_options()->preclude_last_level_data_seconds
: 0;
preclude_last_level_min_seqno_ =
seqno_time_mapping_.GetOldestSequenceNum(preclude_last_level_time);
seqno_to_time_mapping_.GetProximalSeqnoBeforeTime(
preclude_last_level_time) +
1;
}
}
}
Expand Down Expand Up @@ -1570,7 +1576,7 @@ Status CompactionJob::FinishCompactionOutputFile(

const uint64_t current_entries = outputs.NumEntries();

s = outputs.Finish(s, seqno_time_mapping_);
s = outputs.Finish(s, seqno_to_time_mapping_);

if (s.ok()) {
// With accurate smallest and largest key, we can get a slightly more
Expand Down
2 changes: 1 addition & 1 deletion db/compaction/compaction_job.h
Original file line number Diff line number Diff line change
Expand Up @@ -350,7 +350,7 @@ class CompactionJob {

// Stores the sequence number to time mapping gathered from all input files
// it also collects the smallest_seqno -> oldest_ancester_time from the SST.
SeqnoToTimeMapping seqno_time_mapping_;
SeqnoToTimeMapping seqno_to_time_mapping_;

// Minimal sequence number for preserving the time information. The time info
// older than this sequence number won't be preserved after the compaction and
Expand Down
14 changes: 8 additions & 6 deletions db/compaction/compaction_outputs.cc
Original file line number Diff line number Diff line change
Expand Up @@ -18,16 +18,18 @@ void CompactionOutputs::NewBuilder(const TableBuilderOptions& tboptions) {
builder_.reset(NewTableBuilder(tboptions, file_writer_.get()));
}

Status CompactionOutputs::Finish(const Status& intput_status,
const SeqnoToTimeMapping& seqno_time_mapping) {
Status CompactionOutputs::Finish(
const Status& intput_status,
const SeqnoToTimeMapping& seqno_to_time_mapping) {
FileMetaData* meta = GetMetaData();
assert(meta != nullptr);
Status s = intput_status;
if (s.ok()) {
std::string seqno_time_mapping_str;
seqno_time_mapping.Encode(seqno_time_mapping_str, meta->fd.smallest_seqno,
meta->fd.largest_seqno, meta->file_creation_time);
builder_->SetSeqnoTimeTableProperties(seqno_time_mapping_str,
std::string seqno_to_time_mapping_str;
seqno_to_time_mapping.Encode(
seqno_to_time_mapping_str, meta->fd.smallest_seqno,
meta->fd.largest_seqno, meta->file_creation_time);
builder_->SetSeqnoTimeTableProperties(seqno_to_time_mapping_str,
meta->oldest_ancester_time);
s = builder_->Finish();

Expand Down
2 changes: 1 addition & 1 deletion db/compaction/compaction_outputs.h
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ class CompactionOutputs {

// Finish the current output file
Status Finish(const Status& intput_status,
const SeqnoToTimeMapping& seqno_time_mapping);
const SeqnoToTimeMapping& seqno_to_time_mapping);

// Update output table properties from table builder
void UpdateTableProperties() {
Expand Down
44 changes: 35 additions & 9 deletions db/compaction/tiered_compaction_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1249,8 +1249,11 @@ TEST_F(PrecludeLastLevelTest, MigrationFromPreserveTimeManualCompaction) {
options.num_levels = kNumLevels;
DestroyAndReopen(options);

// bootstrap DB sequence numbers (FIXME: make these steps unnecessary)
ASSERT_OK(Put("foo", "bar"));
ASSERT_OK(SingleDelete("foo"));
// pass some time first, otherwise the first a few keys write time are going
// to be zero, and internally zero has special meaning: kUnknownSeqnoTime
// to be zero, and internally zero has special meaning: kUnknownTimeBeforeAll
dbfull()->TEST_WaitForPeriodicTaskRun(
[&] { mock_clock_->MockSleepForSeconds(static_cast<int>(kKeyPerSec)); });

Expand Down Expand Up @@ -1311,8 +1314,11 @@ TEST_F(PrecludeLastLevelTest, MigrationFromPreserveTimeAutoCompaction) {
options.num_levels = kNumLevels;
DestroyAndReopen(options);

// bootstrap DB sequence numbers (FIXME: make these steps unnecessary)
ASSERT_OK(Put("foo", "bar"));
ASSERT_OK(SingleDelete("foo"));
// pass some time first, otherwise the first a few keys write time are going
// to be zero, and internally zero has special meaning: kUnknownSeqnoTime
// to be zero, and internally zero has special meaning: kUnknownTimeBeforeAll
dbfull()->TEST_WaitForPeriodicTaskRun(
[&] { mock_clock_->MockSleepForSeconds(static_cast<int>(kKeyPerSec)); });

Expand Down Expand Up @@ -1387,8 +1393,11 @@ TEST_F(PrecludeLastLevelTest, MigrationFromPreserveTimePartial) {
options.num_levels = kNumLevels;
DestroyAndReopen(options);

// bootstrap DB sequence numbers (FIXME: make these steps unnecessary)
ASSERT_OK(Put("foo", "bar"));
ASSERT_OK(SingleDelete("foo"));
// pass some time first, otherwise the first a few keys write time are going
// to be zero, and internally zero has special meaning: kUnknownSeqnoTime
// to be zero, and internally zero has special meaning: kUnknownTimeBeforeAll
dbfull()->TEST_WaitForPeriodicTaskRun(
[&] { mock_clock_->MockSleepForSeconds(static_cast<int>(kKeyPerSec)); });

Expand Down Expand Up @@ -1514,8 +1523,11 @@ TEST_F(PrecludeLastLevelTest, LastLevelOnlyCompactionPartial) {
options.num_levels = kNumLevels;
DestroyAndReopen(options);

// bootstrap DB sequence numbers (FIXME: make these steps unnecessary)
ASSERT_OK(Put("foo", "bar"));
ASSERT_OK(SingleDelete("foo"));
// pass some time first, otherwise the first a few keys write time are going
// to be zero, and internally zero has special meaning: kUnknownSeqnoTime
// to be zero, and internally zero has special meaning: kUnknownTimeBeforeAll
dbfull()->TEST_WaitForPeriodicTaskRun(
[&] { mock_clock_->MockSleepForSeconds(static_cast<int>(kKeyPerSec)); });

Expand Down Expand Up @@ -1592,8 +1604,11 @@ TEST_P(PrecludeLastLevelTestWithParms, LastLevelOnlyCompactionNoPreclude) {
options.num_levels = kNumLevels;
DestroyAndReopen(options);

// bootstrap DB sequence numbers (FIXME: make these steps unnecessary)
ASSERT_OK(Put("foo", "bar"));
ASSERT_OK(SingleDelete("foo"));
// pass some time first, otherwise the first a few keys write time are going
// to be zero, and internally zero has special meaning: kUnknownSeqnoTime
// to be zero, and internally zero has special meaning: kUnknownTimeBeforeAll
dbfull()->TEST_WaitForPeriodicTaskRun(
[&] { mock_clock_->MockSleepForSeconds(static_cast<int>(kKeyPerSec)); });

Expand Down Expand Up @@ -1906,8 +1921,11 @@ TEST_F(PrecludeLastLevelTest, PartialPenultimateLevelCompaction) {
options.num_levels = kNumLevels;
DestroyAndReopen(options);

// bootstrap DB sequence numbers (FIXME: make these steps unnecessary)
ASSERT_OK(Put("foo", "bar"));
ASSERT_OK(SingleDelete("foo"));
// pass some time first, otherwise the first a few keys write time are going
// to be zero, and internally zero has special meaning: kUnknownSeqnoTime
// to be zero, and internally zero has special meaning: kUnknownTimeBeforeAll
dbfull()->TEST_WaitForPeriodicTaskRun(
[&] { mock_clock_->MockSleepForSeconds(static_cast<int>(10)); });

Expand Down Expand Up @@ -1996,7 +2014,13 @@ TEST_F(PrecludeLastLevelTest, PartialPenultimateLevelCompaction) {
Close();
}

TEST_F(PrecludeLastLevelTest, RangeDelsCauseFileEndpointsToOverlap) {
// FIXME broken test:
// dbfull()->TEST_WaitForCompact()
// Corruption: force_consistency_checks(DEBUG): VersionBuilder: L5 has
// overlapping ranges:
// file #14 largest key: '6B6579303030303134' seq:32, type:1 vs.
// file #19 smallest key: '6B6579303030303130' seq:10, type:1
TEST_F(PrecludeLastLevelTest, DISABLED_RangeDelsCauseFileEndpointsToOverlap) {
const int kNumLevels = 7;
const int kSecondsPerKey = 10;
const int kNumFiles = 3;
Expand All @@ -2017,8 +2041,11 @@ TEST_F(PrecludeLastLevelTest, RangeDelsCauseFileEndpointsToOverlap) {
options.target_file_size_base = kFileBytes;
DestroyAndReopen(options);

// bootstrap DB sequence numbers (FIXME: make these steps unnecessary)
ASSERT_OK(Put("foo", "bar"));
ASSERT_OK(SingleDelete("foo"));
// pass some time first, otherwise the first a few keys write time are going
// to be zero, and internally zero has special meaning: kUnknownSeqnoTime
// to be zero, and internally zero has special meaning: kUnknownTimeBeforeAll
dbfull()->TEST_WaitForPeriodicTaskRun([&] {
mock_clock_->MockSleepForSeconds(static_cast<int>(kSecondsPerKey));
});
Expand Down Expand Up @@ -2139,7 +2166,6 @@ TEST_F(PrecludeLastLevelTest, RangeDelsCauseFileEndpointsToOverlap) {
Close();
}


} // namespace ROCKSDB_NAMESPACE

int main(int argc, char** argv) {
Expand Down
10 changes: 6 additions & 4 deletions db/db_impl/db_impl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -830,9 +830,9 @@ Status DBImpl::RegisterRecordSeqnoTimeWorker() {
}
}
if (min_time_duration == std::numeric_limits<uint64_t>::max()) {
seqno_time_mapping_.Resize(0, 0);
seqno_to_time_mapping_.Resize(0, 0);
} else {
seqno_time_mapping_.Resize(min_time_duration, max_time_duration);
seqno_to_time_mapping_.Resize(min_time_duration, max_time_duration);
}
}

Expand Down Expand Up @@ -6371,16 +6371,18 @@ Status DBImpl::GetCreationTimeOfOldestFile(uint64_t* creation_time) {
}

void DBImpl::RecordSeqnoToTimeMapping() {
// TECHNICALITY: Sample last sequence number *before* time, as prescribed
// for SeqnoToTimeMapping
SequenceNumber seqno = GetLatestSequenceNumber();
// Get time first then sequence number, so the actual time of seqno is <=
// unix_time recorded
int64_t unix_time = 0;
immutable_db_options_.clock->GetCurrentTime(&unix_time)
.PermitUncheckedError(); // Ignore error
SequenceNumber seqno = GetLatestSequenceNumber();
bool appended = false;
{
InstrumentedMutexLock l(&mutex_);
appended = seqno_time_mapping_.Append(seqno, unix_time);
appended = seqno_to_time_mapping_.Append(seqno, unix_time);
}
if (!appended) {
ROCKS_LOG_WARN(immutable_db_options_.info_log,
Expand Down
4 changes: 2 additions & 2 deletions db/db_impl/db_impl.h
Original file line number Diff line number Diff line change
Expand Up @@ -2748,9 +2748,9 @@ class DBImpl : public DB {
// Pointer to WriteBufferManager stalling interface.
std::unique_ptr<StallInterface> wbm_stall_;

// seqno_time_mapping_ stores the sequence number to time mapping, it's not
// seqno_to_time_mapping_ stores the sequence number to time mapping, it's not
// thread safe, both read and write need db mutex hold.
SeqnoToTimeMapping seqno_time_mapping_;
SeqnoToTimeMapping seqno_to_time_mapping_;

// Stop write token that is acquired when first LockWAL() is called.
// Destroyed when last UnlockWAL() is called. Controlled by DB mutex.
Expand Down
4 changes: 2 additions & 2 deletions db/db_impl/db_impl_compaction_flush.cc
Original file line number Diff line number Diff line change
Expand Up @@ -250,7 +250,7 @@ Status DBImpl::FlushMemTableToOutputFile(
GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_,
&event_logger_, mutable_cf_options.report_bg_io_stats,
true /* sync_output_directory */, true /* write_manifest */, thread_pri,
io_tracer_, seqno_time_mapping_, db_id_, db_session_id_,
io_tracer_, seqno_to_time_mapping_, db_id_, db_session_id_,
cfd->GetFullHistoryTsLow(), &blob_callback_);
FileMetaData file_meta;

Expand Down Expand Up @@ -522,7 +522,7 @@ Status DBImpl::AtomicFlushMemTablesToOutputFiles(
GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_,
&event_logger_, mutable_cf_options.report_bg_io_stats,
false /* sync_output_directory */, false /* write_manifest */,
thread_pri, io_tracer_, seqno_time_mapping_, db_id_, db_session_id_,
thread_pri, io_tracer_, seqno_to_time_mapping_, db_id_, db_session_id_,
cfd->GetFullHistoryTsLow(), &blob_callback_));
}

Expand Down
2 changes: 1 addition & 1 deletion db/db_impl/db_impl_debug.cc
Original file line number Diff line number Diff line change
Expand Up @@ -306,7 +306,7 @@ const PeriodicTaskScheduler& DBImpl::TEST_GetPeriodicTaskScheduler() const {

SeqnoToTimeMapping DBImpl::TEST_GetSeqnoToTimeMapping() const {
InstrumentedMutexLock l(&mutex_);
return seqno_time_mapping_;
return seqno_to_time_mapping_;
}


Expand Down
4 changes: 2 additions & 2 deletions db/db_impl/db_impl_open.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1649,7 +1649,7 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
TableFileCreationReason::kRecovery, 0 /* oldest_key_time */,
0 /* file_creation_time */, db_id_, db_session_id_,
0 /* target_file_size */, meta.fd.GetNumber());
SeqnoToTimeMapping empty_seqno_time_mapping;
SeqnoToTimeMapping empty_seqno_to_time_mapping;
Version* version = cfd->current();
version->Ref();
const ReadOptions read_option(Env::IOActivity::kDBOpen);
Expand All @@ -1661,7 +1661,7 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
snapshot_seqs, earliest_write_conflict_snapshot, kMaxSequenceNumber,
snapshot_checker, paranoid_file_checks, cfd->internal_stats(), &io_s,
io_tracer_, BlobFileCreationReason::kRecovery,
empty_seqno_time_mapping, &event_logger_, job_id, Env::IO_HIGH,
empty_seqno_to_time_mapping, &event_logger_, job_id, Env::IO_HIGH,
nullptr /* table_properties */, write_hint,
nullptr /*full_history_ts_low*/, &blob_callback_, version,
&num_input_entries);
Expand Down
13 changes: 7 additions & 6 deletions db/flush_job.cc
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ FlushJob::FlushJob(
Statistics* stats, EventLogger* event_logger, bool measure_io_stats,
const bool sync_output_directory, const bool write_manifest,
Env::Priority thread_pri, const std::shared_ptr<IOTracer>& io_tracer,
const SeqnoToTimeMapping& seqno_time_mapping, const std::string& db_id,
const SeqnoToTimeMapping& seqno_to_time_mapping, const std::string& db_id,
const std::string& db_session_id, std::string full_history_ts_low,
BlobFileCompletionCallback* blob_callback)
: dbname_(dbname),
Expand Down Expand Up @@ -136,7 +136,7 @@ FlushJob::FlushJob(
clock_(db_options_.clock),
full_history_ts_low_(std::move(full_history_ts_low)),
blob_callback_(blob_callback),
db_impl_seqno_time_mapping_(seqno_time_mapping) {
db_impl_seqno_to_time_mapping_(seqno_to_time_mapping) {
// Update the thread status to indicate flush.
ReportStartedFlush();
TEST_SYNC_POINT("FlushJob::FlushJob()");
Expand Down Expand Up @@ -851,10 +851,11 @@ Status FlushJob::WriteLevel0Table() {
Status s;

SequenceNumber smallest_seqno = mems_.front()->GetEarliestSequenceNumber();
if (!db_impl_seqno_time_mapping_.Empty()) {
// make a local copy, as the seqno_time_mapping from db_impl is not thread
// safe, which will be used while not holding the db_mutex.
seqno_to_time_mapping_ = db_impl_seqno_time_mapping_.Copy(smallest_seqno);
if (!db_impl_seqno_to_time_mapping_.Empty()) {
// make a local copy, as the seqno_to_time_mapping from db_impl is not
// thread safe, which will be used while not holding the db_mutex.
seqno_to_time_mapping_ =
db_impl_seqno_to_time_mapping_.Copy(smallest_seqno);
}

std::vector<BlobFileAddition> blob_file_additions;
Expand Down
6 changes: 3 additions & 3 deletions db/flush_job.h
Original file line number Diff line number Diff line change
Expand Up @@ -210,9 +210,9 @@ class FlushJob {
const std::string full_history_ts_low_;
BlobFileCompletionCallback* blob_callback_;

// reference to the seqno_time_mapping_ in db_impl.h, not safe to read without
// db mutex
const SeqnoToTimeMapping& db_impl_seqno_time_mapping_;
// reference to the seqno_to_time_mapping_ in db_impl.h, not safe to read
// without db mutex
const SeqnoToTimeMapping& db_impl_seqno_to_time_mapping_;
SeqnoToTimeMapping seqno_to_time_mapping_;

// Keeps track of the newest user-defined timestamp for this flush job if
Expand Down
Loading

0 comments on commit 02443dd

Please sign in to comment.