提交 a39e931e 编写于 作者: I Igor Canadi

FlushProcess

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/D27561
上级 efa2fb33
...@@ -143,7 +143,8 @@ TESTS = \ ...@@ -143,7 +143,8 @@ TESTS = \
cuckoo_table_builder_test \ cuckoo_table_builder_test \
cuckoo_table_reader_test \ cuckoo_table_reader_test \
cuckoo_table_db_test \ cuckoo_table_db_test \
write_batch_with_index_test write_batch_with_index_test \
flush_job_test
TOOLS = \ TOOLS = \
sst_dump \ sst_dump \
...@@ -412,6 +413,9 @@ ttl_test: utilities/ttl/ttl_test.o $(LIBOBJECTS) $(TESTHARNESS) ...@@ -412,6 +413,9 @@ ttl_test: utilities/ttl/ttl_test.o $(LIBOBJECTS) $(TESTHARNESS)
write_batch_with_index_test: utilities/write_batch_with_index/write_batch_with_index_test.o $(LIBOBJECTS) $(TESTHARNESS) write_batch_with_index_test: utilities/write_batch_with_index/write_batch_with_index_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(CXX) utilities/write_batch_with_index/write_batch_with_index_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) $(CXX) utilities/write_batch_with_index/write_batch_with_index_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS)
flush_job_test: db/flush_job_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(CXX) db/flush_job_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS)
dbformat_test: db/dbformat_test.o $(LIBOBJECTS) $(TESTHARNESS) dbformat_test: db/dbformat_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(CXX) db/dbformat_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) $(CXX) db/dbformat_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS)
......
...@@ -20,6 +20,7 @@ ...@@ -20,6 +20,7 @@
#include <limits> #include <limits>
#include "db/db_impl.h" #include "db/db_impl.h"
#include "db/job_context.h"
#include "db/version_set.h" #include "db/version_set.h"
#include "db/internal_stats.h" #include "db/internal_stats.h"
#include "db/compaction_picker.h" #include "db/compaction_picker.h"
...@@ -71,15 +72,15 @@ ColumnFamilyHandleImpl::ColumnFamilyHandleImpl(ColumnFamilyData* cfd, ...@@ -71,15 +72,15 @@ ColumnFamilyHandleImpl::ColumnFamilyHandleImpl(ColumnFamilyData* cfd,
ColumnFamilyHandleImpl::~ColumnFamilyHandleImpl() { ColumnFamilyHandleImpl::~ColumnFamilyHandleImpl() {
if (cfd_ != nullptr) { if (cfd_ != nullptr) {
DBImpl::DeletionState deletion_state; JobContext job_context;
mutex_->Lock(); mutex_->Lock();
if (cfd_->Unref()) { if (cfd_->Unref()) {
delete cfd_; delete cfd_;
} }
db_->FindObsoleteFiles(deletion_state, false, true); db_->FindObsoleteFiles(&job_context, false, true);
mutex_->Unlock(); mutex_->Unlock();
if (deletion_state.HaveSomethingToDelete()) { if (job_context.HaveSomethingToDelete()) {
db_->PurgeObsoleteFiles(deletion_state); db_->PurgeObsoleteFiles(job_context);
} }
} }
} }
......
...@@ -42,7 +42,7 @@ Status DBImpl::DisableFileDeletions() { ...@@ -42,7 +42,7 @@ Status DBImpl::DisableFileDeletions() {
} }
Status DBImpl::EnableFileDeletions(bool force) { Status DBImpl::EnableFileDeletions(bool force) {
DeletionState deletion_state; JobContext job_context;
bool should_purge_files = false; bool should_purge_files = false;
{ {
MutexLock l(&mutex_); MutexLock l(&mutex_);
...@@ -55,7 +55,7 @@ Status DBImpl::EnableFileDeletions(bool force) { ...@@ -55,7 +55,7 @@ Status DBImpl::EnableFileDeletions(bool force) {
if (disable_delete_obsolete_files_ == 0) { if (disable_delete_obsolete_files_ == 0) {
Log(db_options_.info_log, "File Deletions Enabled"); Log(db_options_.info_log, "File Deletions Enabled");
should_purge_files = true; should_purge_files = true;
FindObsoleteFiles(deletion_state, true); FindObsoleteFiles(&job_context, true);
} else { } else {
Log(db_options_.info_log, Log(db_options_.info_log,
"File Deletions Enable, but not really enabled. Counter: %d", "File Deletions Enable, but not really enabled. Counter: %d",
...@@ -63,7 +63,7 @@ Status DBImpl::EnableFileDeletions(bool force) { ...@@ -63,7 +63,7 @@ Status DBImpl::EnableFileDeletions(bool force) {
} }
} }
if (should_purge_files) { if (should_purge_files) {
PurgeObsoleteFiles(deletion_state); PurgeObsoleteFiles(job_context);
} }
LogFlush(db_options_.info_log); LogFlush(db_options_.info_log);
return Status::OK(); return Status::OK();
......
此差异已折叠。
...@@ -35,6 +35,7 @@ ...@@ -35,6 +35,7 @@
#include "db/write_controller.h" #include "db/write_controller.h"
#include "db/flush_scheduler.h" #include "db/flush_scheduler.h"
#include "db/write_thread.h" #include "db/write_thread.h"
#include "db/job_context.h"
namespace rocksdb { namespace rocksdb {
...@@ -223,88 +224,19 @@ class DBImpl : public DB { ...@@ -223,88 +224,19 @@ class DBImpl : public DB {
void TEST_EndWrite(void* w); void TEST_EndWrite(void* w);
#endif // NDEBUG #endif // NDEBUG
// 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(name), path_id(path) {}
bool operator==(const CandidateFileInfo& other) const {
return file_name == other.file_name && path_id == other.path_id;
}
};
// needed for CleanupIteratorState
struct DeletionState {
inline bool HaveSomethingToDelete() const {
return candidate_files.size() ||
sst_delete_files.size() ||
log_delete_files.size();
}
// 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 DeletionState(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;
}
~DeletionState() {
// 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;
}
};
// Returns the list of live files in 'live' and the list // Returns the list of live files in 'live' and the list
// of all files in the filesystem in 'candidate_files'. // of all files in the filesystem in 'candidate_files'.
// If force == false and the last call was less than // If force == false and the last call was less than
// db_options_.delete_obsolete_files_period_micros microseconds ago, // db_options_.delete_obsolete_files_period_micros microseconds ago,
// it will not fill up the deletion_state // it will not fill up the job_context
void FindObsoleteFiles(DeletionState& deletion_state, void FindObsoleteFiles(JobContext* job_context, bool force,
bool force,
bool no_full_scan = false); bool no_full_scan = false);
// Diffs the files listed in filenames and those that do not // Diffs the files listed in filenames and those that do not
// belong to live files are posibly removed. Also, removes all the // belong to live files are posibly removed. Also, removes all the
// files in sst_delete_files and log_delete_files. // files in sst_delete_files and log_delete_files.
// It is not necessary to hold the mutex when invoking this method. // It is not necessary to hold the mutex when invoking this method.
void PurgeObsoleteFiles(DeletionState& deletion_state); void PurgeObsoleteFiles(const JobContext& background_contet);
ColumnFamilyHandle* DefaultColumnFamily() const; ColumnFamilyHandle* DefaultColumnFamily() const;
...@@ -347,9 +279,10 @@ class DBImpl : public DB { ...@@ -347,9 +279,10 @@ class DBImpl : public DB {
// Flush the in-memory write buffer to storage. Switches to a new // Flush the in-memory write buffer to storage. Switches to a new
// log-file/memtable and writes a new descriptor iff successful. // log-file/memtable and writes a new descriptor iff successful.
Status FlushMemTableToOutputFile( Status FlushMemTableToOutputFile(ColumnFamilyData* cfd,
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, const MutableCFOptions& mutable_cf_options,
bool* madeProgress, DeletionState& deletion_state, LogBuffer* log_buffer); bool* madeProgress, JobContext* job_context,
LogBuffer* log_buffer);
// REQUIRES: log_numbers are sorted in ascending order // REQUIRES: log_numbers are sorted in ascending order
Status RecoverLogFiles(const std::vector<uint64_t>& log_numbers, Status RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
...@@ -362,11 +295,6 @@ class DBImpl : public DB { ...@@ -362,11 +295,6 @@ class DBImpl : public DB {
// concurrent flush memtables to storage. // concurrent flush memtables to storage.
Status WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem, Status WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem,
VersionEdit* edit); VersionEdit* edit);
Status WriteLevel0Table(ColumnFamilyData* cfd,
const MutableCFOptions& mutable_cf_options,
const autovector<MemTable*>& mems,
VersionEdit* edit, uint64_t* filenumber, LogBuffer* log_buffer);
Status DelayWrite(uint64_t expiration_time); Status DelayWrite(uint64_t expiration_time);
Status ScheduleFlushes(WriteContext* context); Status ScheduleFlushes(WriteContext* context);
...@@ -388,39 +316,32 @@ class DBImpl : public DB { ...@@ -388,39 +316,32 @@ class DBImpl : public DB {
static void BGWorkFlush(void* db); static void BGWorkFlush(void* db);
void BackgroundCallCompaction(); void BackgroundCallCompaction();
void BackgroundCallFlush(); void BackgroundCallFlush();
Status BackgroundCompaction(bool* madeProgress, DeletionState& deletion_state, Status BackgroundCompaction(bool* madeProgress, JobContext* job_context,
LogBuffer* log_buffer); LogBuffer* log_buffer);
Status BackgroundFlush(bool* madeProgress, DeletionState& deletion_state, Status BackgroundFlush(bool* madeProgress, JobContext* job_context,
LogBuffer* log_buffer); LogBuffer* log_buffer);
void CleanupCompaction(CompactionState* compact, Status status); void CleanupCompaction(CompactionState* compact, Status status);
Status DoCompactionWork(CompactionState* compact, Status DoCompactionWork(CompactionState* compact,
const MutableCFOptions& mutable_cf_options, const MutableCFOptions& mutable_cf_options,
DeletionState& deletion_state, JobContext* job_context, LogBuffer* log_buffer);
LogBuffer* log_buffer);
// This function is called as part of compaction. It enables Flush process to // This function is called as part of compaction. It enables Flush process to
// preempt compaction, since it's higher prioirty // preempt compaction, since it's higher prioirty
// Returns: micros spent executing // Returns: micros spent executing
uint64_t CallFlushDuringCompaction(ColumnFamilyData* cfd, uint64_t CallFlushDuringCompaction(ColumnFamilyData* cfd,
const MutableCFOptions& mutable_cf_options, DeletionState& deletion_state, const MutableCFOptions& mutable_cf_options,
LogBuffer* log_buffer); JobContext* job_context,
LogBuffer* log_buffer);
// Call compaction filter if is_compaction_v2 is not true. Then iterate // Call compaction filter if is_compaction_v2 is not true. Then iterate
// through input and compact the kv-pairs // through input and compact the kv-pairs
Status ProcessKeyValueCompaction( Status ProcessKeyValueCompaction(
const MutableCFOptions& mutable_cf_options, const MutableCFOptions& mutable_cf_options, bool is_snapshot_supported,
bool is_snapshot_supported, SequenceNumber visible_at_tip, SequenceNumber earliest_snapshot,
SequenceNumber visible_at_tip, SequenceNumber latest_snapshot, JobContext* job_context,
SequenceNumber earliest_snapshot, bool bottommost_level, int64_t* imm_micros, Iterator* input,
SequenceNumber latest_snapshot, CompactionState* compact, bool is_compaction_v2, int* num_output_records,
DeletionState& deletion_state, LogBuffer* log_buffer);
bool bottommost_level,
int64_t& imm_micros,
Iterator* input,
CompactionState* compact,
bool is_compaction_v2,
int* num_output_records,
LogBuffer* log_buffer);
// Call compaction_filter_v2->Filter() on kv-pairs in compact // Call compaction_filter_v2->Filter() on kv-pairs in compact
void CallCompactionFilterV2(CompactionState* compact, void CallCompactionFilterV2(CompactionState* compact,
...@@ -624,11 +545,12 @@ class DBImpl : public DB { ...@@ -624,11 +545,12 @@ class DBImpl : public DB {
SequenceNumber* prev_snapshot); SequenceNumber* prev_snapshot);
// Background threads call this function, which is just a wrapper around // Background threads call this function, which is just a wrapper around
// the cfd->InstallSuperVersion() function. Background threads carry // the InstallSuperVersion() function. Background threads carry
// deletion_state which can have new_superversion already allocated. // job_context which can have new_superversion already
void InstallSuperVersion(ColumnFamilyData* cfd, // allocated.
DeletionState& deletion_state, void InstallSuperVersionBackground(
const MutableCFOptions& mutable_cf_options); ColumnFamilyData* cfd, JobContext* job_context,
const MutableCFOptions& mutable_cf_options);
SuperVersion* InstallSuperVersion( SuperVersion* InstallSuperVersion(
ColumnFamilyData* cfd, SuperVersion* new_sv, ColumnFamilyData* cfd, SuperVersion* new_sv,
......
// 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
// 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
// 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(); }
...@@ -10,6 +10,7 @@ ...@@ -10,6 +10,7 @@
#include <string> #include <string>
#include <utility> #include <utility>
#include "db/job_context.h"
#include "db/db_impl.h" #include "db/db_impl.h"
#include "db/db_iter.h" #include "db/db_iter.h"
#include "db/column_family.h" #include "db/column_family.h"
...@@ -155,14 +156,14 @@ void ForwardIterator::Cleanup(bool release_sv) { ...@@ -155,14 +156,14 @@ void ForwardIterator::Cleanup(bool release_sv) {
if (release_sv) { if (release_sv) {
if (sv_ != nullptr && sv_->Unref()) { if (sv_ != nullptr && sv_->Unref()) {
DBImpl::DeletionState deletion_state; JobContext job_context;
db_->mutex_.Lock(); db_->mutex_.Lock();
sv_->Cleanup(); sv_->Cleanup();
db_->FindObsoleteFiles(deletion_state, false, true); db_->FindObsoleteFiles(&job_context, false, true);
db_->mutex_.Unlock(); db_->mutex_.Unlock();
delete sv_; delete sv_;
if (deletion_state.HaveSomethingToDelete()) { if (job_context.HaveSomethingToDelete()) {
db_->PurgeObsoleteFiles(deletion_state); db_->PurgeObsoleteFiles(job_context);
} }
} }
} }
......
// 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
...@@ -5,6 +5,11 @@ ...@@ -5,6 +5,11 @@
// //
#include "db/memtable_list.h" #include "db/memtable_list.h"
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
#endif
#include <inttypes.h>
#include <string> #include <string>
#include "rocksdb/db.h" #include "rocksdb/db.h"
#include "db/memtable.h" #include "db/memtable.h"
...@@ -161,10 +166,10 @@ void MemTableList::RollbackMemtableFlush(const autovector<MemTable*>& mems, ...@@ -161,10 +166,10 @@ void MemTableList::RollbackMemtableFlush(const autovector<MemTable*>& mems,
// Record a successful flush in the manifest file // Record a successful flush in the manifest file
Status MemTableList::InstallMemtableFlushResults( Status MemTableList::InstallMemtableFlushResults(
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
const autovector<MemTable*>& mems, VersionSet* vset, const autovector<MemTable*>& mems, VersionSet* vset, port::Mutex* mu,
port::Mutex* mu, Logger* info_log, uint64_t file_number, uint64_t file_number, FileNumToPathIdMap* pending_outputs,
FileNumToPathIdMap* pending_outputs, autovector<MemTable*>* to_delete, autovector<MemTable*>* to_delete, Directory* db_directory,
Directory* db_directory, LogBuffer* log_buffer) { LogBuffer* log_buffer) {
mu->AssertHeld(); mu->AssertHeld();
// flush was sucessful // flush was sucessful
...@@ -194,8 +199,8 @@ Status MemTableList::InstallMemtableFlushResults( ...@@ -194,8 +199,8 @@ Status MemTableList::InstallMemtableFlushResults(
break; break;
} }
LogToBuffer(log_buffer, "[%s] Level-0 commit table #%lu started", LogToBuffer(log_buffer, "[%s] Level-0 commit table #%" PRIu64 " started",
cfd->GetName().c_str(), (unsigned long)m->file_number_); cfd->GetName().c_str(), m->file_number_);
// this can release and reacquire the mutex. // this can release and reacquire the mutex.
s = vset->LogAndApply(cfd, mutable_cf_options, &m->edit_, mu, db_directory); s = vset->LogAndApply(cfd, mutable_cf_options, &m->edit_, mu, db_directory);
...@@ -209,10 +214,9 @@ Status MemTableList::InstallMemtableFlushResults( ...@@ -209,10 +214,9 @@ Status MemTableList::InstallMemtableFlushResults(
uint64_t mem_id = 1; // how many memtables has been flushed. uint64_t mem_id = 1; // how many memtables has been flushed.
do { do {
if (s.ok()) { // commit new state if (s.ok()) { // commit new state
LogToBuffer(log_buffer, LogToBuffer(log_buffer, "[%s] Level-0 commit table #%" PRIu64
"[%s] Level-0 commit table #%lu: memtable #%lu done", ": memtable #%" PRIu64 " done",
cfd->GetName().c_str(), (unsigned long)m->file_number_, cfd->GetName().c_str(), m->file_number_, mem_id);
(unsigned long)mem_id);
current_->Remove(m); current_->Remove(m);
assert(m->file_number_ > 0); assert(m->file_number_ > 0);
...@@ -226,10 +230,9 @@ Status MemTableList::InstallMemtableFlushResults( ...@@ -226,10 +230,9 @@ Status MemTableList::InstallMemtableFlushResults(
} }
} else { } else {
//commit failed. setup state so that we can flush again. //commit failed. setup state so that we can flush again.
Log(info_log, LogToBuffer(log_buffer, "Level-0 commit table #%" PRIu64
"Level-0 commit table #%lu: memtable #%lu failed", ": memtable #%" PRIu64 " failed",
(unsigned long)m->file_number_, m->file_number_, mem_id);
(unsigned long)mem_id);
m->flush_completed_ = false; m->flush_completed_ = false;
m->flush_in_progress_ = false; m->flush_in_progress_ = false;
m->edit_.Clear(); m->edit_.Clear();
......
...@@ -114,10 +114,10 @@ class MemTableList { ...@@ -114,10 +114,10 @@ class MemTableList {
// Commit a successful flush in the manifest file // Commit a successful flush in the manifest file
Status InstallMemtableFlushResults( Status InstallMemtableFlushResults(
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
const autovector<MemTable*>& m, VersionSet* vset, const autovector<MemTable*>& m, VersionSet* vset, port::Mutex* mu,
port::Mutex* mu, Logger* info_log, uint64_t file_number, uint64_t file_number, FileNumToPathIdMap* pending_outputs,
FileNumToPathIdMap* pending_outputs, autovector<MemTable*>* to_delete, autovector<MemTable*>* to_delete, Directory* db_directory,
Directory* db_directory, LogBuffer* log_buffer); LogBuffer* log_buffer);
// New memtables are inserted at the front of the list. // New memtables are inserted at the front of the list.
// Takes ownership of the referenced held on *m by the caller of Add(). // Takes ownership of the referenced held on *m by the caller of Add().
......
...@@ -74,7 +74,7 @@ void PrintStackTraceLine(const char* symbol, void* frame) { ...@@ -74,7 +74,7 @@ void PrintStackTraceLine(const char* symbol, void* frame) {
// out source to atos, for the address translation // out source to atos, for the address translation
const int kLineMax = 256; const int kLineMax = 256;
char cmd[kLineMax]; char cmd[kLineMax];
snprintf(cmd, kLineMax, "xcrun atos %p -p %d 2>&1", frame, pid); snprintf(cmd, kLineMax, "xcrun atos -d %p -p %d 2>&1", frame, pid);
auto f = popen(cmd, "r"); auto f = popen(cmd, "r");
if (f) { if (f) {
char line[kLineMax]; char line[kLineMax];
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册