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
	
	 Igor Canadi
						Igor Canadi