You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/db/db_impl.h

675 lines
26 KiB

// 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 <list>
#include <utility>
#include <list>
#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/wal_manager.h"
#include "db/writebuffer.h"
#include "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 "util/hash.h"
#include "util/instrumented_mutex.h"
#include "db/internal_stats.h"
#include "db/write_controller.h"
#include "db/flush_scheduler.h"
#include "db/write_thread.h"
namespace rocksdb {
class MemTable;
class TableCache;
class Version;
class VersionEdit;
class VersionSet;
class CompactionFilterV2;
class Arena;
struct JobContext;
class DBImpl : public DB {
public:
DBImpl(const DBOptions& options, const std::string& dbname);
virtual ~DBImpl();
// Implementations of the DB interface
using DB::Put;
virtual Status Put(const WriteOptions& options,
ColumnFamilyHandle* column_family, const Slice& key,
const Slice& value);
using DB::Merge;
virtual Status Merge(const WriteOptions& options,
ColumnFamilyHandle* column_family, const Slice& key,
const Slice& value);
using DB::Delete;
virtual Status Delete(const WriteOptions& options,
ColumnFamilyHandle* column_family, const Slice& key);
using DB::Write;
virtual Status Write(const WriteOptions& options, WriteBatch* updates);
using DB::Get;
virtual Status Get(const ReadOptions& options,
ColumnFamilyHandle* column_family, const Slice& key,
std::string* value);
using DB::MultiGet;
virtual std::vector<Status> MultiGet(
const ReadOptions& options,
const std::vector<ColumnFamilyHandle*>& column_family,
const std::vector<Slice>& keys, std::vector<std::string>* values);
virtual Status CreateColumnFamily(const ColumnFamilyOptions& options,
const std::string& column_family,
ColumnFamilyHandle** handle);
virtual Status DropColumnFamily(ColumnFamilyHandle* column_family);
// Returns false if key doesn't exist in the database and true if it may.
// If value_found is not passed in as null, then return the value if found in
// memory. On return, if value was found, then value_found will be set to true
// , otherwise false.
using DB::KeyMayExist;
virtual bool KeyMayExist(const ReadOptions& options,
ColumnFamilyHandle* column_family, const Slice& key,
std::string* value, bool* value_found = nullptr);
using DB::NewIterator;
virtual Iterator* NewIterator(const ReadOptions& options,
ColumnFamilyHandle* column_family);
virtual Status NewIterators(
const ReadOptions& options,
const std::vector<ColumnFamilyHandle*>& column_families,
std::vector<Iterator*>* iterators);
virtual const Snapshot* GetSnapshot();
virtual void ReleaseSnapshot(const Snapshot* snapshot);
using DB::GetProperty;
virtual bool GetProperty(ColumnFamilyHandle* column_family,
const Slice& property, std::string* value);
using DB::GetIntProperty;
virtual bool GetIntProperty(ColumnFamilyHandle* column_family,
const Slice& property, uint64_t* value) override;
using DB::GetApproximateSizes;
virtual void GetApproximateSizes(ColumnFamilyHandle* column_family,
const Range* range, int n, uint64_t* sizes);
using DB::CompactRange;
virtual Status CompactRange(ColumnFamilyHandle* column_family,
const Slice* begin, const Slice* end,
bool reduce_level = false, int target_level = -1,
uint32_t target_path_id = 0);
using DB::CompactFiles;
virtual Status CompactFiles(
const CompactionOptions& compact_options,
ColumnFamilyHandle* column_family,
const std::vector<std::string>& input_file_names,
const int output_level, const int output_path_id = -1);
using DB::SetOptions;
Status SetOptions(ColumnFamilyHandle* column_family,
const std::unordered_map<std::string, std::string>& options_map);
using DB::NumberLevels;
virtual int NumberLevels(ColumnFamilyHandle* column_family);
using DB::MaxMemCompactionLevel;
virtual int MaxMemCompactionLevel(ColumnFamilyHandle* column_family);
using DB::Level0StopWriteTrigger;
virtual int Level0StopWriteTrigger(ColumnFamilyHandle* column_family);
[RocksDB] BackupableDB Summary: In this diff I present you BackupableDB v1. You can easily use it to backup your DB and it will do incremental snapshots for you. Let's first describe how you would use BackupableDB. It's inheriting StackableDB interface so you can easily construct it with your DB object -- it will add a method RollTheSnapshot() to the DB object. When you call RollTheSnapshot(), current snapshot of the DB will be stored in the backup dir. To restore, you can just call RestoreDBFromBackup() on a BackupableDB (which is a static method) and it will restore all files from the backup dir. In the next version, it will even support automatic backuping every X minutes. There are multiple things you can configure: 1. backup_env and db_env can be different, which is awesome because then you can easily backup to HDFS or wherever you feel like. 2. sync - if true, it *guarantees* backup consistency on machine reboot 3. number of snapshots to keep - this will keep last N snapshots around if you want, for some reason, be able to restore from an earlier snapshot. All the backuping is done in incremental fashion - if we already have 00010.sst, we will not copy it again. *IMPORTANT* -- This is based on assumption that 00010.sst never changes - two files named 00010.sst from the same DB will always be exactly the same. Is this true? I always copy manifest, current and log files. 4. You can decide if you want to flush the memtables before you backup, or you're fine with backing up the log files -- either way, you get a complete and consistent view of the database at a time of backup. 5. More things you can find in BackupableDBOptions Here is the directory structure I use: backup_dir/CURRENT_SNAPSHOT - just 4 bytes holding the latest snapshot 0, 1, 2, ... - files containing serialized version of each snapshot - containing a list of files files/*.sst - sst files shared between snapshots - if one snapshot references 00010.sst and another one needs to backup it from the DB, it will just reference the same file files/ 0/, 1/, 2/, ... - snapshot directories containing private snapshot files - current, manifest and log files All the files are ref counted and deleted immediatelly when they get out of scope. Some other stuff in this diff: 1. Added GetEnv() method to the DB. Discussed with @haobo and we agreed that it seems right thing to do. 2. Fixed StackableDB interface. The way it was set up before, I was not able to implement BackupableDB. Test Plan: I have a unittest, but please don't look at this yet. I just hacked it up to help me with debugging. I will write a lot of good tests and update the diff. Also, `make asan_check` Reviewers: dhruba, haobo, emayanke Reviewed By: dhruba CC: leveldb, haobo Differential Revision: https://reviews.facebook.net/D14295
11 years ago
virtual const std::string& GetName() const;
virtual Env* GetEnv() const;
using DB::GetOptions;
virtual const Options& GetOptions(ColumnFamilyHandle* column_family) const;
using DB::Flush;
virtual Status Flush(const FlushOptions& options,
ColumnFamilyHandle* column_family);
virtual SequenceNumber GetLatestSequenceNumber() const;
#ifndef ROCKSDB_LITE
virtual Status DisableFileDeletions();
virtual Status EnableFileDeletions(bool force);
virtual int IsFileDeletionsEnabled() const;
// All the returned filenames start with "/"
virtual Status GetLiveFiles(std::vector<std::string>&,
uint64_t* manifest_file_size,
bool flush_memtable = true);
virtual Status GetSortedWalFiles(VectorLogPtr& files);
virtual Status GetUpdatesSince(
SequenceNumber seq_number, unique_ptr<TransactionLogIterator>* iter,
const TransactionLogIterator::ReadOptions&
read_options = TransactionLogIterator::ReadOptions());
virtual Status DeleteFile(std::string name);
virtual void GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata);
// Obtains the meta data of the specified column family of the DB.
// Status::NotFound() will be returned if the current DB does not have
// any column family match the specified name.
// TODO(yhchiang): output parameter is placed in the end in this codebase.
virtual void GetColumnFamilyMetaData(
ColumnFamilyHandle* column_family,
ColumnFamilyMetaData* metadata) override;
#endif // ROCKSDB_LITE
// checks if all live files exist on file system and that their file sizes
// match to our in-memory records
virtual Status CheckConsistency();
virtual Status GetDbIdentity(std::string& identity);
Status RunManualCompaction(ColumnFamilyData* cfd, int input_level,
int output_level, uint32_t output_path_id,
const Slice* begin, const Slice* end);
#ifndef ROCKSDB_LITE
// Extra methods (for testing) that are not in the public DB interface
// Implemented in db_impl_debug.cc
// Compact any files in the named level that overlap [*begin, *end]
Status TEST_CompactRange(int level, const Slice* begin, const Slice* end,
ColumnFamilyHandle* column_family = nullptr);
// Force current memtable contents to be flushed.
Status TEST_FlushMemTable(bool wait = true);
// Wait for memtable compaction
Status TEST_WaitForFlushMemTable(ColumnFamilyHandle* column_family = nullptr);
// Wait for any compaction
Status TEST_WaitForCompact();
// Return an internal iterator over the current state of the database.
// The keys of this iterator are internal keys (see format.h).
// The returned iterator should be deleted when no longer needed.
Iterator* TEST_NewInternalIterator(
Arena* arena, ColumnFamilyHandle* column_family = nullptr);
// Return the maximum overlapping data (in bytes) at next level for any
// file at a level >= 1.
int64_t TEST_MaxNextLevelOverlappingBytes(ColumnFamilyHandle* column_family =
nullptr);
// Return the current manifest file no.
uint64_t TEST_Current_Manifest_FileNo();
// get total level0 file size. Only for testing.
uint64_t TEST_GetLevel0TotalSize();
void TEST_GetFilesMetaData(ColumnFamilyHandle* column_family,
std::vector<std::vector<FileMetaData>>* metadata);
void TEST_LockMutex();
void TEST_UnlockMutex();
// REQUIRES: mutex locked
void* TEST_BeginWrite();
// REQUIRES: mutex locked
// pass the pointer that you got from TEST_BeginWrite()
void TEST_EndWrite(void* w);
uint64_t TEST_max_total_in_memory_state() {
return max_total_in_memory_state_;
}
#endif // ROCKSDB_LITE
// Returns the list of live files in 'live' and the list
// of all files in the filesystem in 'candidate_files'.
// If force == false and the last call was less than
// db_options_.delete_obsolete_files_period_micros microseconds ago,
// it will not fill up the job_context
void FindObsoleteFiles(JobContext* job_context, bool force,
bool no_full_scan = false);
// Diffs the files listed in filenames and those that do not
// belong to live files are posibly removed. Also, removes all the
// files in sst_delete_files and log_delete_files.
// It is not necessary to hold the mutex when invoking this method.
void PurgeObsoleteFiles(const JobContext& background_contet);
ColumnFamilyHandle* DefaultColumnFamily() const;
const SnapshotList& snapshots() const { return snapshots_; }
protected:
Env* const env_;
const std::string dbname_;
unique_ptr<VersionSet> versions_;
const DBOptions db_options_;
Statistics* stats_;
Iterator* NewInternalIterator(const ReadOptions&, ColumnFamilyData* cfd,
SuperVersion* super_version, Arena* arena);
void NotifyOnFlushCompleted(ColumnFamilyData* cfd, uint64_t file_number,
const MutableCFOptions& mutable_cf_options);
void NotifyOnCompactionCompleted(ColumnFamilyData* cfd,
Compaction *c, const Status &st);
void NewThreadStatusCfInfo(ColumnFamilyData* cfd) const;
void EraseThreadStatusCfInfo(ColumnFamilyData* cfd) const;
void EraseThreadStatusDbInfo() const;
private:
friend class DB;
friend class InternalStats;
#ifndef ROCKSDB_LITE
friend class ForwardIterator;
#endif
friend struct SuperVersion;
friend class CompactedDBImpl;
struct CompactionState;
struct WriteContext;
Status NewDB();
// Recover the descriptor from persistent storage. May do a significant
// amount of work to recover recently logged updates. Any changes to
// be made to the descriptor are added to *edit.
Status Recover(const std::vector<ColumnFamilyDescriptor>& column_families,
bool read_only = false, bool error_if_log_file_exist = false);
void MaybeIgnoreError(Status* s) const;
const Status CreateArchivalDirectory();
// Delete any unneeded files and stale in-memory entries.
void DeleteObsoleteFiles();
// Background process needs to call
// auto x = CaptureCurrentFileNumberInPendingOutputs()
// <do something>
// ReleaseFileNumberFromPendingOutputs(x)
// This will protect any temporary files created while <do something> is
// executing from being deleted.
// -----------
// This function will capture current file number and append it to
// pending_outputs_. This will prevent any background process to delete any
// file created after this point.
std::list<uint64_t>::iterator CaptureCurrentFileNumberInPendingOutputs();
// This function should be called with the result of
// CaptureCurrentFileNumberInPendingOutputs(). It then marks that any file
// created between the calls CaptureCurrentFileNumberInPendingOutputs() and
// ReleaseFileNumberFromPendingOutputs() can now be deleted (if it's not live
// and blocked by any other pending_outputs_ calls)
void ReleaseFileNumberFromPendingOutputs(std::list<uint64_t>::iterator v);
// Flush the in-memory write buffer to storage. Switches to a new
// log-file/memtable and writes a new descriptor iff successful.
Status FlushMemTableToOutputFile(ColumnFamilyData* cfd,
const MutableCFOptions& mutable_cf_options,
bool* madeProgress, JobContext* job_context,
LogBuffer* log_buffer);
// REQUIRES: log_numbers are sorted in ascending order
Status RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
SequenceNumber* max_sequence, bool read_only);
// The following two methods are used to flush a memtable to
// storage. The first one is used atdatabase RecoveryTime (when the
// database is opened) and is heavyweight because it holds the mutex
// for the entire period. The second method WriteLevel0Table supports
// concurrent flush memtables to storage.
Status WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem,
VersionEdit* edit);
Status DelayWrite(uint64_t expiration_time);
Status ScheduleFlushes(WriteContext* context);
Status SetNewMemtableAndNewLogFile(ColumnFamilyData* cfd,
WriteContext* context);
// Force current memtable contents to be flushed.
Status FlushMemTable(ColumnFamilyData* cfd, const FlushOptions& options);
// Wait for memtable flushed
Status WaitForFlushMemTable(ColumnFamilyData* cfd);
void RecordFlushIOStats();
void RecordCompactionIOStats();
#ifndef ROCKSDB_LITE
Status CompactFilesImpl(
const CompactionOptions& compact_options, ColumnFamilyData* cfd,
Version* version, const std::vector<std::string>& input_file_names,
const int output_level, int output_path_id);
#endif // ROCKSDB_LITE
ColumnFamilyData* GetColumnFamilyDataByName(const std::string& cf_name);
void MaybeScheduleFlushOrCompaction();
Rewritten system for scheduling background work Summary: When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue. The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction. Here are the performance results: Command: ./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333 Before the patch: fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s After the patch: fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got: fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s Test Plan: make check two stress tests: Big number of compactions and flushes: ./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000 max_background_flushes=0, to verify that this case also works correctly ./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000 Reviewers: ljin, rven, yhchiang, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D30123
10 years ago
void SchedulePendingFlush(ColumnFamilyData* cfd);
void SchedulePendingCompaction(ColumnFamilyData* cfd);
static void BGWorkCompaction(void* db);
static void BGWorkFlush(void* db);
void BackgroundCallCompaction();
void BackgroundCallFlush();
Status BackgroundCompaction(bool* madeProgress, JobContext* job_context,
LogBuffer* log_buffer);
Status BackgroundFlush(bool* madeProgress, JobContext* job_context,
LogBuffer* log_buffer);
// This function is called as part of compaction. It enables Flush process to
// preempt compaction, since it's higher prioirty
uint64_t CallFlushDuringCompaction(ColumnFamilyData* cfd,
const MutableCFOptions& mutable_cf_options,
JobContext* job_context,
LogBuffer* log_buffer);
void PrintStatistics();
// dump rocksdb.stats to LOG
void MaybeDumpStats();
// Return the minimum empty level that could hold the total data in the
// input level. Return the input level, if such level could not be found.
int FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
const MutableCFOptions& mutable_cf_options, int level);
// Move the files in the input level to the target level.
// If target_level < 0, automatically calculate the minimum level that could
// hold the data set.
Status ReFitLevel(ColumnFamilyData* cfd, int level, int target_level = -1);
Rewritten system for scheduling background work Summary: When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue. The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction. Here are the performance results: Command: ./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333 Before the patch: fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s After the patch: fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got: fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s Test Plan: make check two stress tests: Big number of compactions and flushes: ./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000 max_background_flushes=0, to verify that this case also works correctly ./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000 Reviewers: ljin, rven, yhchiang, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D30123
10 years ago
// helper functions for adding and removing from flush & compaction queues
void AddToCompactionQueue(ColumnFamilyData* cfd);
ColumnFamilyData* PopFirstFromCompactionQueue();
void AddToFlushQueue(ColumnFamilyData* cfd);
ColumnFamilyData* PopFirstFromFlushQueue();
// table_cache_ provides its own synchronization
std::shared_ptr<Cache> table_cache_;
// Lock over the persistent DB state. Non-nullptr iff successfully acquired.
FileLock* db_lock_;
// State below is protected by mutex_
InstrumentedMutex mutex_;
std::atomic<bool> shutting_down_;
// This condition variable is signaled on these conditions:
// * whenever bg_compaction_scheduled_ goes down to 0
// * if bg_manual_only_ > 0, whenever a compaction finishes, even if it hasn't
// made any progress
// * whenever a compaction made any progress
// * whenever bg_flush_scheduled_ value decreases (i.e. whenever a flush is
// done, even if it didn't make any progress)
// * whenever there is an error in background flush or compaction
InstrumentedCondVar bg_cv_;
uint64_t logfile_number_;
unique_ptr<log::Writer> log_;
bool log_dir_synced_;
bool log_empty_;
ColumnFamilyHandleImpl* default_cf_handle_;
make internal stats independent of statistics Summary: also make it aware of column family output from db_bench ``` ** Compaction Stats [default] ** Level Files Size(MB) Score Read(GB) Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) RW-Amp W-Amp Rd(MB/s) Wr(MB/s) Rn(cnt) Rnp1(cnt) Wnp1(cnt) Wnew(cnt) Comp(sec) Comp(cnt) Avg(sec) Stall(sec) Stall(cnt) Avg(ms) ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ L0 14 956 0.9 0.0 0.0 0.0 2.7 2.7 0.0 0.0 0.0 111.6 0 0 0 0 24 40 0.612 75.20 492387 0.15 L1 21 2001 2.0 5.7 2.0 3.7 5.3 1.6 5.4 2.6 71.2 65.7 31 43 55 12 82 2 41.242 43.72 41183 1.06 L2 217 18974 1.9 16.5 2.0 14.4 15.1 0.7 15.6 7.4 70.1 64.3 17 182 185 3 241 16 15.052 0.00 0 0.00 L3 1641 188245 1.8 9.1 1.1 8.0 8.5 0.5 15.4 7.4 61.3 57.2 9 75 76 1 152 9 16.887 0.00 0 0.00 L4 4447 449025 0.4 13.4 4.8 8.6 9.1 0.5 4.7 1.9 77.8 52.7 38 79 100 21 176 38 4.639 0.00 0 0.00 Sum 6340 659201 0.0 44.7 10.0 34.7 40.6 6.0 32.0 15.2 67.7 61.6 95 379 416 37 676 105 6.439 118.91 533570 0.22 Int 0 0 0.0 1.2 0.4 0.8 1.3 0.5 5.2 2.7 59.1 65.6 3 7 9 2 20 10 2.003 0.00 0 0.00 Stalls(secs): 75.197 level0_slowdown, 0.000 level0_numfiles, 0.000 memtable_compaction, 43.717 leveln_slowdown Stalls(count): 492387 level0_slowdown, 0 level0_numfiles, 0 memtable_compaction, 41183 leveln_slowdown ** DB Stats ** Uptime(secs): 202.1 total, 13.5 interval Cumulative writes: 6291456 writes, 6291456 batches, 1.0 writes per batch, 4.90 ingest GB Cumulative WAL: 6291456 writes, 6291456 syncs, 1.00 writes per sync, 4.90 GB written Interval writes: 1048576 writes, 1048576 batches, 1.0 writes per batch, 836.0 ingest MB Interval WAL: 1048576 writes, 1048576 syncs, 1.00 writes per sync, 0.82 MB written Test Plan: ran it Reviewers: sdong, yhchiang, igor Reviewed By: igor Subscribers: leveldb Differential Revision: https://reviews.facebook.net/D19917
10 years ago
InternalStats* default_cf_internal_stats_;
unique_ptr<ColumnFamilyMemTablesImpl> column_family_memtables_;
struct LogFileNumberSize {
explicit LogFileNumberSize(uint64_t _number)
: number(_number), size(0), getting_flushed(false) {}
void AddSize(uint64_t new_size) { size += new_size; }
uint64_t number;
uint64_t size;
bool getting_flushed;
};
std::deque<LogFileNumberSize> alive_log_files_;
uint64_t total_log_size_;
// only used for dynamically adjusting max_total_wal_size. it is a sum of
// [write_buffer_size * max_write_buffer_number] over all column families
uint64_t max_total_in_memory_state_;
// If true, we have only one (default) column family. We use this to optimize
// some code-paths
bool single_column_family_mode_;
bool is_snapshot_supported_;
// Class to maintain directories for all database paths other than main one.
class Directories {
public:
Status SetDirectories(Env* env, const std::string& dbname,
const std::string& wal_dir,
const std::vector<DbPath>& data_paths);
Directory* GetDataDir(size_t path_id);
Directory* GetWalDir() {
if (wal_dir_) {
return wal_dir_.get();
}
return db_dir_.get();
}
Directory* GetDbDir() { return db_dir_.get(); }
private:
std::unique_ptr<Directory> db_dir_;
std::vector<std::unique_ptr<Directory>> data_dirs_;
std::unique_ptr<Directory> wal_dir_;
Status CreateAndNewDirectory(Env* env, const std::string& dirname,
std::unique_ptr<Directory>* directory) const;
};
Directories directories_;
WriteBuffer write_buffer_;
WriteThread write_thread_;
WriteBatch tmp_batch_;
WriteController write_controller_;
FlushScheduler flush_scheduler_;
SnapshotList snapshots_;
// For each background job, pending_outputs_ keeps the current file number at
// the time that background job started.
// FindObsoleteFiles()/PurgeObsoleteFiles() never deletes any file that has
// number bigger than any of the file number in pending_outputs_. Since file
// numbers grow monotonically, this also means that pending_outputs_ is always
// sorted. After a background job is done executing, its file number is
// deleted from pending_outputs_, which allows PurgeObsoleteFiles() to clean
// it up.
// State is protected with db mutex.
std::list<uint64_t> pending_outputs_;
Rewritten system for scheduling background work Summary: When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue. The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction. Here are the performance results: Command: ./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333 Before the patch: fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s After the patch: fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got: fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s Test Plan: make check two stress tests: Big number of compactions and flushes: ./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000 max_background_flushes=0, to verify that this case also works correctly ./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000 Reviewers: ljin, rven, yhchiang, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D30123
10 years ago
// flush_queue_ and compaction_queue_ hold column families that we need to
// flush and compact, respectively.
// A column family is inserted into flush_queue_ when it satisfies condition
// cfd->imm()->IsFlushPending()
// A column family is inserted into compaction_queue_ when it satisfied
// condition cfd->NeedsCompaction()
// Column families in this list are all Ref()-erenced
// TODO(icanadi) Provide some kind of ReferencedColumnFamily class that will
// do RAII on ColumnFamilyData
// Column families are in this queue when they need to be flushed or
// compacted. Consumers of these queues are flush and compaction threads. When
// column family is put on this queue, we increase unscheduled_flushes_ and
// unscheduled_compactions_. When these variables are bigger than zero, that
// means we need to schedule background threads for compaction and thread.
// Once the background threads are scheduled, we decrease unscheduled_flushes_
// and unscheduled_compactions_. That way we keep track of number of
// compaction and flush threads we need to schedule. This scheduling is done
// in MaybeScheduleFlushOrCompaction()
// invariant(column family present in flush_queue_ <==>
// ColumnFamilyData::pending_flush_ == true)
std::deque<ColumnFamilyData*> flush_queue_;
// invariant(column family present in compaction_queue_ <==>
// ColumnFamilyData::pending_compaction_ == true)
std::deque<ColumnFamilyData*> compaction_queue_;
int unscheduled_flushes_;
int unscheduled_compactions_;
Fix a deadlock in CompactRange() Summary: The way DBImpl::TEST_CompactRange() throttles down the number of bg compactions can cause it to deadlock when CompactRange() is called concurrently from multiple threads. Imagine a following scenario with only two threads (max_background_compactions is 10 and bg_compaction_scheduled_ is initially 0): 1. Thread #1 increments bg_compaction_scheduled_ (to LargeNumber), sets bg_compaction_scheduled_ to 9 (newvalue), schedules the compaction (bg_compaction_scheduled_ is now 10) and waits for it to complete. 2. Thread #2 calls TEST_CompactRange(), increments bg_compaction_scheduled_ (now LargeNumber + 10) and waits on a cv for bg_compaction_scheduled_ to drop to LargeNumber. 3. BG thread completes the first manual compaction, decrements bg_compaction_scheduled_ and wakes up all threads waiting on bg_cv_. Thread #1 runs, increments bg_compaction_scheduled_ by LargeNumber again (now 2*LargeNumber + 9). Since that's more than LargeNumber + newvalue, thread #2 also goes to sleep (waiting on bg_cv_), without resetting bg_compaction_scheduled_. This diff attempts to address the problem by introducing a new counter bg_manual_only_ (when positive, MaybeScheduleFlushOrCompaction() will only schedule manual compactions). Test Plan: I could pretty much consistently reproduce the deadlock with a program that calls CompactRange(nullptr, nullptr) immediately after Write() from multiple threads. This no longer happens with this patch. Tests (make check) pass. Reviewers: dhruba, igor, sdong, haobo Reviewed By: igor CC: leveldb Differential Revision: https://reviews.facebook.net/D14799
11 years ago
// count how many background compactions are running or have been scheduled
int bg_compaction_scheduled_;
Fix a deadlock in CompactRange() Summary: The way DBImpl::TEST_CompactRange() throttles down the number of bg compactions can cause it to deadlock when CompactRange() is called concurrently from multiple threads. Imagine a following scenario with only two threads (max_background_compactions is 10 and bg_compaction_scheduled_ is initially 0): 1. Thread #1 increments bg_compaction_scheduled_ (to LargeNumber), sets bg_compaction_scheduled_ to 9 (newvalue), schedules the compaction (bg_compaction_scheduled_ is now 10) and waits for it to complete. 2. Thread #2 calls TEST_CompactRange(), increments bg_compaction_scheduled_ (now LargeNumber + 10) and waits on a cv for bg_compaction_scheduled_ to drop to LargeNumber. 3. BG thread completes the first manual compaction, decrements bg_compaction_scheduled_ and wakes up all threads waiting on bg_cv_. Thread #1 runs, increments bg_compaction_scheduled_ by LargeNumber again (now 2*LargeNumber + 9). Since that's more than LargeNumber + newvalue, thread #2 also goes to sleep (waiting on bg_cv_), without resetting bg_compaction_scheduled_. This diff attempts to address the problem by introducing a new counter bg_manual_only_ (when positive, MaybeScheduleFlushOrCompaction() will only schedule manual compactions). Test Plan: I could pretty much consistently reproduce the deadlock with a program that calls CompactRange(nullptr, nullptr) immediately after Write() from multiple threads. This no longer happens with this patch. Tests (make check) pass. Reviewers: dhruba, igor, sdong, haobo Reviewed By: igor CC: leveldb Differential Revision: https://reviews.facebook.net/D14799
11 years ago
// If non-zero, MaybeScheduleFlushOrCompaction() will only schedule manual
// compactions (if manual_compaction_ is not null). This mechanism enables
// manual compactions to wait until all other compactions are finished.
int bg_manual_only_;
// number of background memtable flush jobs, submitted to the HIGH pool
int bg_flush_scheduled_;
// Information for a manual compaction
struct ManualCompaction {
ColumnFamilyData* cfd;
int input_level;
int output_level;
uint32_t output_path_id;
bool done;
Status status;
bool in_progress; // compaction request being processed?
const InternalKey* begin; // nullptr means beginning of key range
const InternalKey* end; // nullptr means end of key range
InternalKey tmp_storage; // Used to keep track of compaction progress
};
ManualCompaction* manual_compaction_;
// Have we encountered a background error in paranoid mode?
Status bg_error_;
// shall we disable deletion of obsolete files
// if 0 the deletion is enabled.
// if non-zero, files will not be getting deleted
// This enables two different threads to call
// EnableFileDeletions() and DisableFileDeletions()
// without any synchronization
int disable_delete_obsolete_files_;
Speed up FindObsoleteFiles() Summary: There are two versions of FindObsoleteFiles(): * full scan, which is executed every 6 hours (and it's terribly slow) * no full scan, which is executed every time a background process finishes and iterator is deleted This diff is optimizing the second case (no full scan). Here's what we do before the diff: * Get the list of obsolete files (files with ref==0). Some files in obsolete_files set might actually be live. * Get the list of live files to avoid deleting files that are live. * Delete files that are in obsolete_files and not in live_files. After this diff: * The only files with ref==0 that are still live are files that have been part of move compaction. Don't include moved files in obsolete_files. * Get the list of obsolete files (which exclude moved files). * No need to get the list of live files, since all files in obsolete_files need to be deleted. I'll post the benchmark results, but you can get the feel of it here: https://reviews.facebook.net/D30123 This depends on D30123. P.S. We should do full scan only in failure scenarios, not every 6 hours. I'll do this in a follow-up diff. Test Plan: One new unit test. Made sure that unit test fails if we don't have a `if (!f->moved)` safeguard in ~Version. make check Big number of compactions and flushes: ./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000 Reviewers: yhchiang, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D30249
10 years ago
// next time when we should run DeleteObsoleteFiles with full scan
uint64_t delete_obsolete_files_next_run_;
// last time stats were dumped to LOG
std::atomic<uint64_t> last_stats_dump_time_microsec_;
bool flush_on_destroy_; // Used when disableWAL is true.
static const int KEEP_LOG_FILE_NUM = 1000;
std::string db_absolute_path_;
// The options to access storage files
const EnvOptions env_options_;
#ifndef ROCKSDB_LITE
WalManager wal_manager_;
#endif // ROCKSDB_LITE
// A value of true temporarily disables scheduling of background work
bool bg_work_gate_closed_;
// Guard against multiple concurrent refitting
bool refitting_level_;
// Indicate DB was opened successfully
bool opened_successfully_;
// The list of registered event listeners.
std::list<EventListener*> listeners_;
// count how many events are currently being notified.
int notifying_events_;
// No copying allowed
DBImpl(const DBImpl&);
void operator=(const DBImpl&);
// Return the earliest snapshot where seqno is visible.
// Store the snapshot right before that, if any, in prev_snapshot
inline SequenceNumber findEarliestVisibleSnapshot(
SequenceNumber in,
std::vector<SequenceNumber>& snapshots,
SequenceNumber* prev_snapshot);
// Background threads call this function, which is just a wrapper around
// the InstallSuperVersion() function. Background threads carry
// job_context which can have new_superversion already
// allocated.
void InstallSuperVersionBackground(
ColumnFamilyData* cfd, JobContext* job_context,
const MutableCFOptions& mutable_cf_options);
Rewritten system for scheduling background work Summary: When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue. The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction. Here are the performance results: Command: ./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333 Before the patch: fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s After the patch: fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got: fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s Test Plan: make check two stress tests: Big number of compactions and flushes: ./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000 max_background_flushes=0, to verify that this case also works correctly ./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000 Reviewers: ljin, rven, yhchiang, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D30123
10 years ago
// All ColumnFamily state changes go through this function. Here we analyze
// the new state and we schedule background work if we detect that the new
// state needs flush or compaction.
// If dont_schedule_bg_work == true, then caller asks us to not schedule flush
// or compaction here, but it also promises to schedule needed background
// work. We use this to scheduling background compactions when we are in the
// write thread, which is very performance critical. Caller schedules
// background work as soon as it exits the write thread
SuperVersion* InstallSuperVersion(ColumnFamilyData* cfd, SuperVersion* new_sv,
const MutableCFOptions& mutable_cf_options,
bool dont_schedule_bg_work = false);
// Find Super version and reference it. Based on options, it might return
// the thread local cached one.
inline SuperVersion* GetAndRefSuperVersion(ColumnFamilyData* cfd);
// Un-reference the super version and return it to thread local cache if
// needed. If it is the last reference of the super version. Clean it up
// after un-referencing it.
inline void ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
SuperVersion* sv);
#ifndef ROCKSDB_LITE
using DB::GetPropertiesOfAllTables;
virtual Status GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
TablePropertiesCollection* props)
override;
#endif // ROCKSDB_LITE
// Function that Get and KeyMayExist call with no_io true or false
// Note: 'value_found' from KeyMayExist propagates here
Status GetImpl(const ReadOptions& options, ColumnFamilyHandle* column_family,
const Slice& key, std::string* value,
bool* value_found = nullptr);
bool GetIntPropertyInternal(ColumnFamilyHandle* column_family,
DBPropertyType property_type,
bool need_out_of_mutex, uint64_t* value);
};
// Sanitize db options. The caller should delete result.info_log if
// it is not equal to src.info_log.
extern Options SanitizeOptions(const std::string& db,
const InternalKeyComparator* icmp,
const Options& src);
extern DBOptions SanitizeOptions(const std::string& db, const DBOptions& src);
// Fix user-supplied options to be reasonable
template <class T, class V>
static void ClipToRange(T* ptr, V minvalue, V maxvalue) {
if (static_cast<V>(*ptr) > maxvalue) *ptr = maxvalue;
if (static_cast<V>(*ptr) < minvalue) *ptr = minvalue;
}
} // namespace rocksdb