Summary: Abstract out FlushProcess and take it out of DBImpl. This also includes taking DeletionState outside of DBImpl. Currently this diff is only doing the refactoring. Future work includes: 1. Decoupling flush_process.cc, make it depend on less state 2. Write flush_process_test, which will mock out everything that FlushProcess depends on and test it in isolation Test Plan: make check Reviewers: rven, yhchiang, sdong, ljin Reviewed By: ljin Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D27561main
parent
efa2fb33b0
commit
a39e931e50
@ -0,0 +1,223 @@ |
|||||||
|
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
||||||
|
// This source code is licensed under the BSD-style license found in the
|
||||||
|
// LICENSE file in the root directory of this source tree. An additional grant
|
||||||
|
// of patent rights can be found in the PATENTS file in the same directory.
|
||||||
|
//
|
||||||
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||||
|
// Use of this source code is governed by a BSD-style license that can be
|
||||||
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||||
|
|
||||||
|
#include "db/flush_job.h" |
||||||
|
|
||||||
|
#ifndef __STDC_FORMAT_MACROS |
||||||
|
#define __STDC_FORMAT_MACROS |
||||||
|
#endif |
||||||
|
|
||||||
|
#include <inttypes.h> |
||||||
|
#include <algorithm> |
||||||
|
#include <vector> |
||||||
|
|
||||||
|
#include "db/builder.h" |
||||||
|
#include "db/db_iter.h" |
||||||
|
#include "db/dbformat.h" |
||||||
|
#include "db/filename.h" |
||||||
|
#include "db/log_reader.h" |
||||||
|
#include "db/log_writer.h" |
||||||
|
#include "db/memtable.h" |
||||||
|
#include "db/memtable_list.h" |
||||||
|
#include "db/merge_context.h" |
||||||
|
#include "db/version_set.h" |
||||||
|
#include "port/port.h" |
||||||
|
#include "port/likely.h" |
||||||
|
#include "rocksdb/db.h" |
||||||
|
#include "rocksdb/env.h" |
||||||
|
#include "rocksdb/statistics.h" |
||||||
|
#include "rocksdb/status.h" |
||||||
|
#include "rocksdb/table.h" |
||||||
|
#include "table/block.h" |
||||||
|
#include "table/block_based_table_factory.h" |
||||||
|
#include "table/merger.h" |
||||||
|
#include "table/table_builder.h" |
||||||
|
#include "table/two_level_iterator.h" |
||||||
|
#include "util/coding.h" |
||||||
|
#include "util/logging.h" |
||||||
|
#include "util/log_buffer.h" |
||||||
|
#include "util/mutexlock.h" |
||||||
|
#include "util/perf_context_imp.h" |
||||||
|
#include "util/iostats_context_imp.h" |
||||||
|
#include "util/stop_watch.h" |
||||||
|
#include "util/sync_point.h" |
||||||
|
|
||||||
|
namespace rocksdb { |
||||||
|
|
||||||
|
FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd, |
||||||
|
const DBOptions& db_options, |
||||||
|
const MutableCFOptions& mutable_cf_options, |
||||||
|
const EnvOptions& env_options, VersionSet* versions, |
||||||
|
port::Mutex* db_mutex, std::atomic<bool>* shutting_down, |
||||||
|
FileNumToPathIdMap* pending_outputs, |
||||||
|
SequenceNumber newest_snapshot, JobContext* job_context, |
||||||
|
LogBuffer* log_buffer, Directory* db_directory, |
||||||
|
CompressionType output_compression, Statistics* stats) |
||||||
|
: dbname_(dbname), |
||||||
|
cfd_(cfd), |
||||||
|
db_options_(db_options), |
||||||
|
mutable_cf_options_(mutable_cf_options), |
||||||
|
env_options_(env_options), |
||||||
|
versions_(versions), |
||||||
|
db_mutex_(db_mutex), |
||||||
|
shutting_down_(shutting_down), |
||||||
|
pending_outputs_(pending_outputs), |
||||||
|
newest_snapshot_(newest_snapshot), |
||||||
|
job_context_(job_context), |
||||||
|
log_buffer_(log_buffer), |
||||||
|
db_directory_(db_directory), |
||||||
|
output_compression_(output_compression), |
||||||
|
stats_(stats) {} |
||||||
|
|
||||||
|
Status FlushJob::Run() { |
||||||
|
// Save the contents of the earliest memtable as a new Table
|
||||||
|
uint64_t file_number; |
||||||
|
autovector<MemTable*> mems; |
||||||
|
cfd_->imm()->PickMemtablesToFlush(&mems); |
||||||
|
if (mems.empty()) { |
||||||
|
LogToBuffer(log_buffer_, "[%s] Nothing in memtable to flush", |
||||||
|
cfd_->GetName().c_str()); |
||||||
|
return Status::OK(); |
||||||
|
} |
||||||
|
|
||||||
|
// entries mems are (implicitly) sorted in ascending order by their created
|
||||||
|
// time. We will use the first memtable's `edit` to keep the meta info for
|
||||||
|
// this flush.
|
||||||
|
MemTable* m = mems[0]; |
||||||
|
VersionEdit* edit = m->GetEdits(); |
||||||
|
edit->SetPrevLogNumber(0); |
||||||
|
// SetLogNumber(log_num) indicates logs with number smaller than log_num
|
||||||
|
// will no longer be picked up for recovery.
|
||||||
|
edit->SetLogNumber(mems.back()->GetNextLogNumber()); |
||||||
|
edit->SetColumnFamily(cfd_->GetID()); |
||||||
|
|
||||||
|
// This will release and re-acquire the mutex.
|
||||||
|
Status s = WriteLevel0Table(mems, edit, &file_number); |
||||||
|
|
||||||
|
if (s.ok() && |
||||||
|
(shutting_down_->load(std::memory_order_acquire) || cfd_->IsDropped())) { |
||||||
|
s = Status::ShutdownInProgress( |
||||||
|
"Database shutdown or Column family drop during flush"); |
||||||
|
} |
||||||
|
|
||||||
|
if (!s.ok()) { |
||||||
|
cfd_->imm()->RollbackMemtableFlush(mems, file_number, pending_outputs_); |
||||||
|
} else { |
||||||
|
// Replace immutable memtable with the generated Table
|
||||||
|
s = cfd_->imm()->InstallMemtableFlushResults( |
||||||
|
cfd_, mutable_cf_options_, mems, versions_, db_mutex_, file_number, |
||||||
|
pending_outputs_, &job_context_->memtables_to_free, db_directory_, |
||||||
|
log_buffer_); |
||||||
|
} |
||||||
|
|
||||||
|
return s; |
||||||
|
} |
||||||
|
|
||||||
|
Status FlushJob::WriteLevel0Table(const autovector<MemTable*>& mems, |
||||||
|
VersionEdit* edit, uint64_t* filenumber) { |
||||||
|
db_mutex_->AssertHeld(); |
||||||
|
const uint64_t start_micros = db_options_.env->NowMicros(); |
||||||
|
FileMetaData meta; |
||||||
|
|
||||||
|
meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0); |
||||||
|
*filenumber = meta.fd.GetNumber(); |
||||||
|
// path 0 for level 0 file.
|
||||||
|
pending_outputs_->insert({meta.fd.GetNumber(), 0}); |
||||||
|
|
||||||
|
const SequenceNumber earliest_seqno_in_memtable = |
||||||
|
mems[0]->GetFirstSequenceNumber(); |
||||||
|
Version* base = cfd_->current(); |
||||||
|
base->Ref(); // it is likely that we do not need this reference
|
||||||
|
Status s; |
||||||
|
{ |
||||||
|
db_mutex_->Unlock(); |
||||||
|
if (log_buffer_) { |
||||||
|
log_buffer_->FlushBufferToLog(); |
||||||
|
} |
||||||
|
std::vector<Iterator*> memtables; |
||||||
|
ReadOptions ro; |
||||||
|
ro.total_order_seek = true; |
||||||
|
Arena arena; |
||||||
|
for (MemTable* m : mems) { |
||||||
|
Log(db_options_.info_log, |
||||||
|
"[%s] Flushing memtable with next log file: %" PRIu64 "\n", |
||||||
|
cfd_->GetName().c_str(), m->GetNextLogNumber()); |
||||||
|
memtables.push_back(m->NewIterator(ro, &arena)); |
||||||
|
} |
||||||
|
{ |
||||||
|
ScopedArenaIterator iter(NewMergingIterator(&cfd_->internal_comparator(), |
||||||
|
&memtables[0], |
||||||
|
memtables.size(), &arena)); |
||||||
|
Log(db_options_.info_log, |
||||||
|
"[%s] Level-0 flush table #%" PRIu64 ": started", |
||||||
|
cfd_->GetName().c_str(), meta.fd.GetNumber()); |
||||||
|
|
||||||
|
s = BuildTable(dbname_, db_options_.env, *cfd_->ioptions(), env_options_, |
||||||
|
cfd_->table_cache(), iter.get(), &meta, |
||||||
|
cfd_->internal_comparator(), newest_snapshot_, |
||||||
|
earliest_seqno_in_memtable, output_compression_, |
||||||
|
cfd_->ioptions()->compression_opts, Env::IO_HIGH); |
||||||
|
LogFlush(db_options_.info_log); |
||||||
|
} |
||||||
|
Log(db_options_.info_log, |
||||||
|
"[%s] Level-0 flush table #%" PRIu64 ": %" PRIu64 " bytes %s", |
||||||
|
cfd_->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(), |
||||||
|
s.ToString().c_str()); |
||||||
|
|
||||||
|
if (!db_options_.disableDataSync && db_directory_ != nullptr) { |
||||||
|
db_directory_->Fsync(); |
||||||
|
} |
||||||
|
db_mutex_->Lock(); |
||||||
|
} |
||||||
|
base->Unref(); |
||||||
|
|
||||||
|
// re-acquire the most current version
|
||||||
|
base = cfd_->current(); |
||||||
|
|
||||||
|
// There could be multiple threads writing to its own level-0 file.
|
||||||
|
// The pending_outputs cannot be cleared here, otherwise this newly
|
||||||
|
// created file might not be considered as a live-file by another
|
||||||
|
// compaction thread that is concurrently deleting obselete files.
|
||||||
|
// The pending_outputs can be cleared only after the new version is
|
||||||
|
// committed so that other threads can recognize this file as a
|
||||||
|
// valid one.
|
||||||
|
// pending_outputs_.erase(meta.number);
|
||||||
|
|
||||||
|
// Note that if file_size is zero, the file has been deleted and
|
||||||
|
// should not be added to the manifest.
|
||||||
|
int level = 0; |
||||||
|
if (s.ok() && meta.fd.GetFileSize() > 0) { |
||||||
|
const Slice min_user_key = meta.smallest.user_key(); |
||||||
|
const Slice max_user_key = meta.largest.user_key(); |
||||||
|
// if we have more than 1 background thread, then we cannot
|
||||||
|
// insert files directly into higher levels because some other
|
||||||
|
// threads could be concurrently producing compacted files for
|
||||||
|
// that key range.
|
||||||
|
if (base != nullptr && db_options_.max_background_compactions <= 1 && |
||||||
|
db_options_.max_background_flushes == 0 && |
||||||
|
cfd_->ioptions()->compaction_style == kCompactionStyleLevel) { |
||||||
|
level = base->PickLevelForMemTableOutput(mutable_cf_options_, |
||||||
|
min_user_key, max_user_key); |
||||||
|
} |
||||||
|
edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(), |
||||||
|
meta.fd.GetFileSize(), meta.smallest, meta.largest, |
||||||
|
meta.smallest_seqno, meta.largest_seqno); |
||||||
|
} |
||||||
|
|
||||||
|
InternalStats::CompactionStats stats(1); |
||||||
|
stats.micros = db_options_.env->NowMicros() - start_micros; |
||||||
|
stats.bytes_written = meta.fd.GetFileSize(); |
||||||
|
cfd_->internal_stats()->AddCompactionStats(level, stats); |
||||||
|
cfd_->internal_stats()->AddCFStats(InternalStats::BYTES_FLUSHED, |
||||||
|
meta.fd.GetFileSize()); |
||||||
|
RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize()); |
||||||
|
return s; |
||||||
|
} |
||||||
|
|
||||||
|
} // namespace rocksdb
|
@ -0,0 +1,86 @@ |
|||||||
|
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
||||||
|
// This source code is licensed under the BSD-style license found in the
|
||||||
|
// LICENSE file in the root directory of this source tree. An additional grant
|
||||||
|
// of patent rights can be found in the PATENTS file in the same directory.
|
||||||
|
//
|
||||||
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||||
|
// Use of this source code is governed by a BSD-style license that can be
|
||||||
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||||
|
#pragma once |
||||||
|
|
||||||
|
#include <atomic> |
||||||
|
#include <deque> |
||||||
|
#include <limits> |
||||||
|
#include <set> |
||||||
|
#include <utility> |
||||||
|
#include <vector> |
||||||
|
#include <string> |
||||||
|
|
||||||
|
#include "db/dbformat.h" |
||||||
|
#include "db/log_writer.h" |
||||||
|
#include "db/snapshot.h" |
||||||
|
#include "db/column_family.h" |
||||||
|
#include "db/version_edit.h" |
||||||
|
#include "db/memtable_list.h" |
||||||
|
#include "port/port.h" |
||||||
|
#include "rocksdb/db.h" |
||||||
|
#include "rocksdb/env.h" |
||||||
|
#include "rocksdb/memtablerep.h" |
||||||
|
#include "rocksdb/transaction_log.h" |
||||||
|
#include "util/autovector.h" |
||||||
|
#include "util/stop_watch.h" |
||||||
|
#include "util/thread_local.h" |
||||||
|
#include "util/scoped_arena_iterator.h" |
||||||
|
#include "db/internal_stats.h" |
||||||
|
#include "db/write_controller.h" |
||||||
|
#include "db/flush_scheduler.h" |
||||||
|
#include "db/write_thread.h" |
||||||
|
#include "db/job_context.h" |
||||||
|
|
||||||
|
namespace rocksdb { |
||||||
|
|
||||||
|
class MemTable; |
||||||
|
class TableCache; |
||||||
|
class Version; |
||||||
|
class VersionEdit; |
||||||
|
class VersionSet; |
||||||
|
class Arena; |
||||||
|
|
||||||
|
class FlushJob { |
||||||
|
public: |
||||||
|
// TODO(icanadi) make effort to reduce number of parameters here
|
||||||
|
// IMPORTANT: mutable_cf_options needs to be alive while FlushJob is alive
|
||||||
|
FlushJob(const std::string& dbname, ColumnFamilyData* cfd, |
||||||
|
const DBOptions& db_options, |
||||||
|
const MutableCFOptions& mutable_cf_options, |
||||||
|
const EnvOptions& env_options, VersionSet* versions, |
||||||
|
port::Mutex* db_mutex, std::atomic<bool>* shutting_down, |
||||||
|
FileNumToPathIdMap* pending_outputs, SequenceNumber newest_snapshot, |
||||||
|
JobContext* job_context, LogBuffer* log_buffer, |
||||||
|
Directory* db_directory, CompressionType output_compression, |
||||||
|
Statistics* stats); |
||||||
|
~FlushJob() {} |
||||||
|
|
||||||
|
Status Run(); |
||||||
|
|
||||||
|
private: |
||||||
|
Status WriteLevel0Table(const autovector<MemTable*>& mems, VersionEdit* edit, |
||||||
|
uint64_t* filenumber); |
||||||
|
const std::string& dbname_; |
||||||
|
ColumnFamilyData* cfd_; |
||||||
|
const DBOptions& db_options_; |
||||||
|
const MutableCFOptions& mutable_cf_options_; |
||||||
|
const EnvOptions& env_options_; |
||||||
|
VersionSet* versions_; |
||||||
|
port::Mutex* db_mutex_; |
||||||
|
std::atomic<bool>* shutting_down_; |
||||||
|
FileNumToPathIdMap* pending_outputs_; |
||||||
|
SequenceNumber newest_snapshot_; |
||||||
|
JobContext* job_context_; |
||||||
|
LogBuffer* log_buffer_; |
||||||
|
Directory* db_directory_; |
||||||
|
CompressionType output_compression_; |
||||||
|
Statistics* stats_; |
||||||
|
}; |
||||||
|
|
||||||
|
} // namespace rocksdb
|
@ -0,0 +1,113 @@ |
|||||||
|
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
||||||
|
// This source code is licensed under the BSD-style license found in the
|
||||||
|
// LICENSE file in the root directory of this source tree. An additional grant
|
||||||
|
// of patent rights can be found in the PATENTS file in the same directory.
|
||||||
|
|
||||||
|
#include "db/flush_job.h" |
||||||
|
#include "db/column_family.h" |
||||||
|
#include "db/version_set.h" |
||||||
|
#include "rocksdb/cache.h" |
||||||
|
#include "util/testharness.h" |
||||||
|
#include "util/testutil.h" |
||||||
|
|
||||||
|
namespace rocksdb { |
||||||
|
|
||||||
|
// TODO(icanadi) Mock out everything else:
|
||||||
|
// 1. VersionSet
|
||||||
|
// 2. TableBuilder
|
||||||
|
// 3. Memtable
|
||||||
|
class FlushJobTest { |
||||||
|
public: |
||||||
|
FlushJobTest() |
||||||
|
: env_(Env::Default()), |
||||||
|
dbname_(test::TmpDir() + "/flush_job_test"), |
||||||
|
table_cache_(NewLRUCache(50000, 16, 8)), |
||||||
|
versions_(new VersionSet(dbname_, &db_options_, env_options_, |
||||||
|
table_cache_.get(), &write_controller_)), |
||||||
|
shutting_down_(false) { |
||||||
|
ASSERT_OK(env_->CreateDirIfMissing(dbname_)); |
||||||
|
db_options_.db_paths.emplace_back(dbname_, |
||||||
|
std::numeric_limits<uint64_t>::max()); |
||||||
|
// TODO(icanadi) Remove this once we mock out VersionSet
|
||||||
|
NewDB(); |
||||||
|
std::vector<ColumnFamilyDescriptor> column_families; |
||||||
|
column_families.emplace_back(); |
||||||
|
|
||||||
|
ASSERT_OK(versions_->Recover(column_families, false)); |
||||||
|
} |
||||||
|
|
||||||
|
void NewDB() { |
||||||
|
VersionEdit new_db; |
||||||
|
new_db.SetLogNumber(0); |
||||||
|
new_db.SetNextFile(2); |
||||||
|
new_db.SetLastSequence(0); |
||||||
|
|
||||||
|
const std::string manifest = DescriptorFileName(dbname_, 1); |
||||||
|
unique_ptr<WritableFile> file; |
||||||
|
Status s = env_->NewWritableFile( |
||||||
|
manifest, &file, env_->OptimizeForManifestWrite(env_options_)); |
||||||
|
ASSERT_OK(s); |
||||||
|
{ |
||||||
|
log::Writer log(std::move(file)); |
||||||
|
std::string record; |
||||||
|
new_db.EncodeTo(&record); |
||||||
|
s = log.AddRecord(record); |
||||||
|
} |
||||||
|
ASSERT_OK(s); |
||||||
|
// Make "CURRENT" file that points to the new manifest file.
|
||||||
|
s = SetCurrentFile(env_, dbname_, 1, nullptr); |
||||||
|
} |
||||||
|
|
||||||
|
Env* env_; |
||||||
|
std::string dbname_; |
||||||
|
EnvOptions env_options_; |
||||||
|
std::shared_ptr<Cache> table_cache_; |
||||||
|
WriteController write_controller_; |
||||||
|
DBOptions db_options_; |
||||||
|
ColumnFamilyOptions cf_options_; |
||||||
|
std::unique_ptr<VersionSet> versions_; |
||||||
|
port::Mutex mutex_; |
||||||
|
std::atomic<bool> shutting_down_; |
||||||
|
FileNumToPathIdMap pending_outputs_; |
||||||
|
}; |
||||||
|
|
||||||
|
TEST(FlushJobTest, Empty) { |
||||||
|
JobContext job_context; |
||||||
|
auto cfd = versions_->GetColumnFamilySet()->GetDefault(); |
||||||
|
FlushJob flush_job(dbname_, versions_->GetColumnFamilySet()->GetDefault(), |
||||||
|
db_options_, *cfd->GetLatestMutableCFOptions(), |
||||||
|
env_options_, versions_.get(), &mutex_, &shutting_down_, |
||||||
|
&pending_outputs_, SequenceNumber(), &job_context, nullptr, |
||||||
|
nullptr, kNoCompression, nullptr); |
||||||
|
ASSERT_OK(flush_job.Run()); |
||||||
|
} |
||||||
|
|
||||||
|
TEST(FlushJobTest, NonEmpty) { |
||||||
|
JobContext job_context; |
||||||
|
auto cfd = versions_->GetColumnFamilySet()->GetDefault(); |
||||||
|
|
||||||
|
auto new_mem = new MemTable(cfd->internal_comparator(), *cfd->ioptions(), |
||||||
|
*cfd->GetLatestMutableCFOptions()); |
||||||
|
new_mem->Ref(); |
||||||
|
for (int i = 1; i < 10000; ++i) { |
||||||
|
std::string key(std::to_string(i)); |
||||||
|
std::string value("value" + std::to_string(i)); |
||||||
|
new_mem->Add(SequenceNumber(i), kTypeValue, key, value); |
||||||
|
} |
||||||
|
cfd->imm()->Add(new_mem); |
||||||
|
|
||||||
|
FlushJob flush_job(dbname_, versions_->GetColumnFamilySet()->GetDefault(), |
||||||
|
db_options_, *cfd->GetLatestMutableCFOptions(), |
||||||
|
env_options_, versions_.get(), &mutex_, &shutting_down_, |
||||||
|
&pending_outputs_, SequenceNumber(), &job_context, nullptr, |
||||||
|
nullptr, kNoCompression, nullptr); |
||||||
|
mutex_.Lock(); |
||||||
|
ASSERT_OK(flush_job.Run()); |
||||||
|
mutex_.Unlock(); |
||||||
|
// TODO(icanadi) once you have TableMock, verify that key-values are as
|
||||||
|
// expected
|
||||||
|
} |
||||||
|
|
||||||
|
} // namespace rocksdb
|
||||||
|
|
||||||
|
int main(int argc, char** argv) { return rocksdb::test::RunAllTests(); } |
@ -0,0 +1,87 @@ |
|||||||
|
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
||||||
|
// This source code is licensed under the BSD-style license found in the
|
||||||
|
// LICENSE file in the root directory of this source tree. An additional grant
|
||||||
|
// of patent rights can be found in the PATENTS file in the same directory.
|
||||||
|
//
|
||||||
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||||
|
// Use of this source code is governed by a BSD-style license that can be
|
||||||
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||||
|
|
||||||
|
#pragma once |
||||||
|
|
||||||
|
#include <string> |
||||||
|
#include <vector> |
||||||
|
|
||||||
|
#include "db/column_family.h" |
||||||
|
|
||||||
|
namespace rocksdb { |
||||||
|
|
||||||
|
class MemTable; |
||||||
|
|
||||||
|
struct JobContext { |
||||||
|
inline bool HaveSomethingToDelete() const { |
||||||
|
return candidate_files.size() || sst_delete_files.size() || |
||||||
|
log_delete_files.size(); |
||||||
|
} |
||||||
|
|
||||||
|
// Structure to store information for candidate files to delete.
|
||||||
|
struct CandidateFileInfo { |
||||||
|
std::string file_name; |
||||||
|
uint32_t path_id; |
||||||
|
CandidateFileInfo(std::string name, uint32_t path) |
||||||
|
: file_name(std::move(name)), path_id(path) {} |
||||||
|
bool operator==(const CandidateFileInfo& other) const { |
||||||
|
return file_name == other.file_name && path_id == other.path_id; |
||||||
|
} |
||||||
|
}; |
||||||
|
|
||||||
|
// a list of all files that we'll consider deleting
|
||||||
|
// (every once in a while this is filled up with all files
|
||||||
|
// in the DB directory)
|
||||||
|
std::vector<CandidateFileInfo> candidate_files; |
||||||
|
|
||||||
|
// the list of all live sst files that cannot be deleted
|
||||||
|
std::vector<FileDescriptor> sst_live; |
||||||
|
|
||||||
|
// a list of sst files that we need to delete
|
||||||
|
std::vector<FileMetaData*> sst_delete_files; |
||||||
|
|
||||||
|
// a list of log files that we need to delete
|
||||||
|
std::vector<uint64_t> log_delete_files; |
||||||
|
|
||||||
|
// a list of memtables to be free
|
||||||
|
autovector<MemTable*> memtables_to_free; |
||||||
|
|
||||||
|
autovector<SuperVersion*> superversions_to_free; |
||||||
|
|
||||||
|
SuperVersion* new_superversion; // if nullptr no new superversion
|
||||||
|
|
||||||
|
// the current manifest_file_number, log_number and prev_log_number
|
||||||
|
// that corresponds to the set of files in 'live'.
|
||||||
|
uint64_t manifest_file_number, pending_manifest_file_number, log_number, |
||||||
|
prev_log_number; |
||||||
|
|
||||||
|
explicit JobContext(bool create_superversion = false) { |
||||||
|
manifest_file_number = 0; |
||||||
|
pending_manifest_file_number = 0; |
||||||
|
log_number = 0; |
||||||
|
prev_log_number = 0; |
||||||
|
new_superversion = create_superversion ? new SuperVersion() : nullptr; |
||||||
|
} |
||||||
|
|
||||||
|
~JobContext() { |
||||||
|
// free pending memtables
|
||||||
|
for (auto m : memtables_to_free) { |
||||||
|
delete m; |
||||||
|
} |
||||||
|
// free superversions
|
||||||
|
for (auto s : superversions_to_free) { |
||||||
|
delete s; |
||||||
|
} |
||||||
|
// if new_superversion was not used, it will be non-nullptr and needs
|
||||||
|
// to be freed here
|
||||||
|
delete new_superversion; |
||||||
|
} |
||||||
|
}; |
||||||
|
|
||||||
|
} // namespace rocksdb
|
Loading…
Reference in new issue