@ -42,7 +42,6 @@
# include "rocksdb/cache.h"
# include "rocksdb/compaction_filter.h"
# include "rocksdb/db.h"
# include "rocksdb/column_family.h"
# include "rocksdb/env.h"
# include "rocksdb/merge_operator.h"
# include "rocksdb/statistics.h"
@ -218,6 +217,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
shutting_down_ ( nullptr ) ,
bg_cv_ ( & mutex_ ) ,
logfile_number_ ( 0 ) ,
default_cf_handle_ ( nullptr ) ,
tmp_batch_ ( ) ,
bg_compaction_scheduled_ ( 0 ) ,
bg_manual_only_ ( 0 ) ,
@ -270,14 +270,24 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
DBImpl : : ~ DBImpl ( ) {
// Wait for background work to finish
mutex_ . Lock ( ) ;
if ( flush_on_destroy_ ) {
autovector < ColumnFamilyData * > to_delete ;
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
if ( cfd - > mem ( ) - > GetFirstSequenceNumber ( ) ! = 0 ) {
cfd - > Ref ( ) ;
mutex_ . Unlock ( ) ;
FlushMemTable ( cfd , FlushOptions ( ) ) ;
mutex_ . Lock ( ) ;
if ( cfd - > Unref ( ) ) {
to_delete . push_back ( cfd ) ;
}
}
}
for ( auto cfd : to_delete ) {
delete cfd ;
}
}
mutex_ . Lock ( ) ;
shutting_down_ . Release_Store ( this ) ; // Any non-nullptr value is ok
while ( bg_compaction_scheduled_ | |
bg_flush_scheduled_ | |
@ -285,6 +295,10 @@ DBImpl::~DBImpl() {
bg_cv_ . Wait ( ) ;
}
mutex_ . Unlock ( ) ;
if ( default_cf_handle_ ! = nullptr ) {
// we need to delete handle outside of lock because it does its own locking
delete default_cf_handle_ ;
}
if ( db_lock_ ! = nullptr ) {
env_ - > UnlockFile ( db_lock_ ) ;
@ -816,7 +830,8 @@ Status DBImpl::Recover(
Status s = versions_ - > Recover ( column_families ) ;
if ( s . ok ( ) ) {
SequenceNumber max_sequence ( 0 ) ;
default_cfd_ = versions_ - > GetColumnFamilySet ( ) - > GetDefault ( ) ;
default_cf_handle_ = new ColumnFamilyHandleImpl (
versions_ - > GetColumnFamilySet ( ) - > GetDefault ( ) , this , & mutex_ ) ;
// Recover from all newer log files than the ones named in the
// descriptor (new log files may have been added by the previous
@ -891,6 +906,7 @@ Status DBImpl::RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence,
mutex_ . AssertHeld ( ) ;
std : : unordered_map < int , VersionEdit > version_edits ;
// no need to refcount because iteration is under mutex
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
VersionEdit edit ;
edit . SetColumnFamily ( cfd - > GetID ( ) ) ;
@ -934,7 +950,6 @@ Status DBImpl::RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence,
}
WriteBatchInternal : : SetContents ( & batch , record ) ;
// No need to lock ColumnFamilySet here since its under a DB mutex
status = WriteBatchInternal : : InsertInto (
& batch , column_family_memtables_ . get ( ) , log_number ) ;
@ -950,6 +965,8 @@ Status DBImpl::RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence,
}
if ( ! read_only ) {
// no need to refcount since client still doesn't have access
// to the DB and can not drop column families while we iterate
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
if ( cfd - > mem ( ) - > ApproximateMemoryUsage ( ) >
cfd - > options ( ) - > write_buffer_size ) {
@ -973,6 +990,8 @@ Status DBImpl::RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence,
}
if ( ! read_only ) {
// no need to refcount since client still doesn't have access
// to the DB and can not drop column families while we iterate
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
auto iter = version_edits . find ( cfd - > GetID ( ) ) ;
assert ( iter ! = version_edits . end ( ) ) ;
@ -1198,10 +1217,8 @@ Status DBImpl::FlushMemTableToOutputFile(ColumnFamilyData* cfd,
// This will release and re-acquire the mutex.
Status s = WriteLevel0Table ( cfd , mems , edit , & file_number ) ;
if ( s . ok ( ) & & shutting_down_ . Acquire_Load ( ) ) {
s = Status : : IOError (
" Database shutdown started during memtable compaction "
) ;
if ( s . ok ( ) & & ( shutting_down_ . Acquire_Load ( ) | | cfd - > IsDropped ( ) ) ) {
s = Status : : IOError ( " Column family closed during memtable flush " ) ;
}
// Replace immutable memtable with the generated Table
@ -1229,15 +1246,11 @@ Status DBImpl::FlushMemTableToOutputFile(ColumnFamilyData* cfd,
return s ;
}
Status DBImpl : : CompactRange ( const ColumnFamilyHandle & column_family ,
Status DBImpl : : CompactRange ( ColumnFamilyHandle * column_family ,
const Slice * begin , const Slice * end ,
bool reduce_level , int target_level ) {
mutex_ . Lock ( ) ;
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
mutex_ . Unlock ( ) ;
// this is asserting because client calling DB methods with undefined
// ColumnFamilyHandle is undefined behavior.
assert ( cfd ! = nullptr ) ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
auto cfd = cfh - > cfd ( ) ;
Status s = FlushMemTable ( cfd , FlushOptions ( ) ) ;
if ( ! s . ok ( ) ) {
@ -1367,38 +1380,25 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) {
return status ;
}
int DBImpl : : NumberLevels ( const ColumnFamilyHandle & column_family ) {
mutex_ . Lock ( ) ;
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
mutex_ . Unlock ( ) ;
assert ( cfd ! = nullptr ) ;
return cfd - > NumberLevels ( ) ;
int DBImpl : : NumberLevels ( ColumnFamilyHandle * column_family ) {
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
return cfh - > cfd ( ) - > NumberLevels ( ) ;
}
int DBImpl : : MaxMemCompactionLevel ( const ColumnFamilyHandle & column_family ) {
mutex_ . Lock ( ) ;
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
mutex_ . Unlock ( ) ;
assert ( cfd ! = nullptr ) ;
return cfd - > options ( ) - > max_mem_compaction_level ;
int DBImpl : : MaxMemCompactionLevel ( ColumnFamilyHandle * column_family ) {
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
return cfh - > cfd ( ) - > options ( ) - > max_mem_compaction_level ;
}
int DBImpl : : Level0StopWriteTrigger ( const ColumnFamilyHandle & column_family ) {
mutex_ . Lock ( ) ;
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
mutex_ . Unlock ( ) ;
assert ( cfd ! = nullptr ) ;
return cfd - > options ( ) - > level0_stop_writes_trigger ;
int DBImpl : : Level0StopWriteTrigger ( ColumnFamilyHandle * column_family ) {
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
return cfh - > cfd ( ) - > options ( ) - > level0_stop_writes_trigger ;
}
Status DBImpl : : Flush ( const FlushOptions & options ,
const ColumnFamilyHandle & column_family ) {
mutex_ . Lock ( ) ;
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
mutex_ . Unlock ( ) ;
assert ( cfd ! = nullptr ) ;
return FlushMemTable ( cfd , options ) ;
ColumnFamilyHandle * column_family ) {
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
return FlushMemTable ( cfh - > cfd ( ) , options ) ;
}
SequenceNumber DBImpl : : GetLatestSequenceNumber ( ) const {
@ -1666,11 +1666,12 @@ Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
Status DBImpl : : TEST_CompactRange ( int level ,
const Slice * begin ,
const Slice * end ) {
auto default_cfd = default_cf_handle_ - > cfd ( ) ;
int output_level =
( default_cfd_ - > options ( ) - > compaction_style = = kCompactionStyleUniversal )
( default_cfd - > options ( ) - > compaction_style = = kCompactionStyleUniversal )
? level
: level + 1 ;
return RunManualCompaction ( default_cfd_ , level , output_level , begin , end ) ;
return RunManualCompaction ( default_cfd , level , output_level , begin , end ) ;
}
Status DBImpl : : FlushMemTable ( ColumnFamilyData * cfd ,
@ -1698,11 +1699,11 @@ Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
}
Status DBImpl : : TEST_FlushMemTable ( ) {
return FlushMemTable ( default_cfd_ , FlushOptions ( ) ) ;
return FlushMemTable ( default_cf_han dle _ - > cfd ( ) , FlushOptions ( ) ) ;
}
Status DBImpl : : TEST_WaitForFlushMemTable ( ) {
return WaitForFlushMemTable ( default_cfd_ ) ;
return WaitForFlushMemTable ( default_cf_han dle _ - > cfd ( ) ) ;
}
Status DBImpl : : TEST_WaitForCompact ( ) {
@ -1728,6 +1729,7 @@ void DBImpl::MaybeScheduleFlushOrCompaction() {
// DB is being deleted; no more background compactions
} else {
bool is_flush_pending = false ;
// no need to refcount since we're under a mutex
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
if ( cfd - > imm ( ) - > IsFlushPending ( ) ) {
is_flush_pending = true ;
@ -1744,6 +1746,7 @@ void DBImpl::MaybeScheduleFlushOrCompaction() {
}
}
bool is_compaction_needed = false ;
// no need to refcount since we're under a mutex
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
if ( cfd - > current ( ) - > NeedsCompaction ( ) ) {
is_compaction_needed = true ;
@ -1774,17 +1777,38 @@ void DBImpl::BGWorkCompaction(void* db) {
Status DBImpl : : BackgroundFlush ( bool * madeProgress ,
DeletionState & deletion_state ) {
Status stat ;
mutex_ . AssertHeld ( ) ;
// call_status is failure if at least one flush was a failure. even if
// flushing one column family reports a failure, we will continue flushing
// other column families. however, call_status will be a failure in that case.
Status call_status ;
autovector < ColumnFamilyData * > to_delete ;
// refcounting in iteration
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
while ( stat . ok ( ) & & cfd - > imm ( ) - > IsFlushPending ( ) ) {
if ( cfd - > IsDropped ( ) ) {
continue ;
}
cfd - > Ref ( ) ;
Status flush_status ;
while ( flush_status . ok ( ) & & cfd - > imm ( ) - > IsFlushPending ( ) ) {
Log ( options_ . info_log ,
" BackgroundCallFlush doing FlushMemTableToOutputFile with column "
" family %u, flush slots available %d " ,
cfd - > GetID ( ) , options_ . max_background_flushes - bg_flush_scheduled_ ) ;
stat = FlushMemTableToOutputFile ( cfd , madeProgress , deletion_state ) ;
flush_status =
FlushMemTableToOutputFile ( cfd , madeProgress , deletion_state ) ;
}
if ( call_status . ok ( ) & & ! flush_status . ok ( ) ) {
call_status = flush_status ;
}
if ( cfd - > Unref ( ) ) {
to_delete . push_back ( cfd ) ;
}
}
for ( auto cfd : to_delete ) {
delete cfd ;
}
return stat ;
return call_ status ;
}
void DBImpl : : BackgroundCallFlush ( ) {
@ -1835,7 +1859,7 @@ void DBImpl::TEST_PurgeObsoleteteWAL() {
uint64_t DBImpl : : TEST_GetLevel0TotalSize ( ) {
MutexLock l ( & mutex_ ) ;
return default_cfd_ - > current ( ) - > NumLevelBytes ( 0 ) ;
return default_cf_han dle _ - > cfd ( ) - > current ( ) - > NumLevelBytes ( 0 ) ;
}
void DBImpl : : BackgroundCallCompaction ( ) {
@ -1921,8 +1945,9 @@ Status DBImpl::BackgroundCompaction(bool* madeProgress,
? " (end) "
: manual_end - > DebugString ( ) . c_str ( ) ) ) ;
} else {
// no need to refcount in iteration since it's always under a mutex
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
if ( ! cfd - > options ( ) - > disable_auto_compactions ) {
if ( ! cfd - > options ( ) - > disable_auto_compactions & & ! cfd - > IsDropped ( ) ) {
c . reset ( cfd - > PickCompaction ( ) ) ;
if ( c ! = nullptr ) {
// update statistics
@ -2302,7 +2327,8 @@ Status DBImpl::DoCompactionWork(CompactionState* compact,
compaction_filter = compaction_filter_from_factory . get ( ) ;
}
for ( ; input - > Valid ( ) & & ! shutting_down_ . Acquire_Load ( ) ; ) {
while ( input - > Valid ( ) & & ! shutting_down_ . Acquire_Load ( ) & &
! cfd - > IsDropped ( ) ) {
Slice key = input - > key ( ) ;
Slice value = input - > value ( ) ;
@ -2546,8 +2572,8 @@ Status DBImpl::DoCompactionWork(CompactionState* compact,
}
}
if ( status . ok ( ) & & shutting_down_ . Acquire_Load ( ) ) {
status = Status : : IOError ( " Database shutdown started during compaction" ) ;
if ( status . ok ( ) & & ( shutting_down_ . Acquire_Load ( ) | | cfd - > IsDropped ( ) ) ) {
status = Status : : IOError ( " Column family closing started during compaction" ) ;
}
if ( status . ok ( ) & & compact - > builder ! = nullptr ) {
status = FinishCompactionOutputFile ( compact , input . get ( ) ) ;
@ -2638,8 +2664,7 @@ struct IterState {
static void CleanupIteratorState ( void * arg1 , void * arg2 ) {
IterState * state = reinterpret_cast < IterState * > ( arg1 ) ;
DBImpl : : DeletionState deletion_state ( state - > db - > GetOptions ( ) .
max_write_buffer_number ) ;
DBImpl : : DeletionState deletion_state ;
bool need_cleanup = state - > super_version - > Unref ( ) ;
if ( need_cleanup ) {
@ -2677,11 +2702,17 @@ Iterator* DBImpl::NewInternalIterator(const ReadOptions& options,
return internal_iter ;
}
ColumnFamilyHandle * DBImpl : : DefaultColumnFamily ( ) const {
return default_cf_handle_ ;
}
Iterator * DBImpl : : TEST_NewInternalIterator ( ) {
mutex_ . Lock ( ) ;
SuperVersion * super_version = default_cfd_ - > GetSuperVersion ( ) - > Ref ( ) ;
SuperVersion * super_version =
default_cf_handle_ - > cfd ( ) - > GetSuperVersion ( ) - > Ref ( ) ;
mutex_ . Unlock ( ) ;
return NewInternalIterator ( ReadOptions ( ) , default_cfd_ , super_version ) ;
return NewInternalIterator ( ReadOptions ( ) , default_cf_handle_ - > cfd ( ) ,
super_version ) ;
}
std : : pair < Iterator * , Iterator * > DBImpl : : GetTailingIteratorPair (
@ -2725,11 +2756,11 @@ std::pair<Iterator*, Iterator*> DBImpl::GetTailingIteratorPair(
int64_t DBImpl : : TEST_MaxNextLevelOverlappingBytes ( ) {
MutexLock l ( & mutex_ ) ;
return default_cfd_ - > current ( ) - > MaxNextLevelOverlappingBytes ( ) ;
return default_cf_han dle _ - > cfd ( ) - > current ( ) - > MaxNextLevelOverlappingBytes ( ) ;
}
Status DBImpl : : Get ( const ReadOptions & options ,
const ColumnFamilyHandle & column_family , const Slice & key ,
ColumnFamilyHandle * column_family , const Slice & key ,
std : : string * value ) {
return GetImpl ( options , column_family , key , value ) ;
}
@ -2762,18 +2793,16 @@ void DBImpl::InstallSuperVersion(ColumnFamilyData* cfd,
}
Status DBImpl : : GetImpl ( const ReadOptions & options ,
const ColumnFamilyHandle & column_family ,
const Slice & key , std : : string * value ,
bool * value_found ) {
ColumnFamilyHandle * column_family , const Slice & key ,
std : : string * value , bool * value_found ) {
StopWatch sw ( env_ , options_ . statistics . get ( ) , DB_GET , false ) ;
StopWatchNano snapshot_timer ( env_ , false ) ;
StartPerfTimer ( & snapshot_timer ) ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
auto cfd = cfh - > cfd ( ) ;
mutex_ . Lock ( ) ;
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
// this is asserting because client calling DB methods with undefined
// ColumnFamilyHandle is undefined behavior.
assert ( cfd ! = nullptr ) ;
SuperVersion * get_version = cfd - > GetSuperVersion ( ) - > Ref ( ) ;
mutex_ . Unlock ( ) ;
@ -2851,7 +2880,7 @@ Status DBImpl::GetImpl(const ReadOptions& options,
std : : vector < Status > DBImpl : : MultiGet (
const ReadOptions & options ,
const std : : vector < ColumnFamilyHandle > & column_family ,
const std : : vector < ColumnFamilyHandle * > & column_family ,
const std : : vector < Slice > & keys , std : : vector < std : : string > * values ) {
StopWatch sw ( env_ , options_ . statistics . get ( ) , DB_MULTIGET , false ) ;
@ -2869,8 +2898,12 @@ std::vector<Status> DBImpl::MultiGet(
std : : unordered_map < uint32_t , MultiGetColumnFamilyData * > multiget_cf_data ;
// fill up and allocate outside of mutex
for ( auto cf : column_family ) {
if ( multiget_cf_data . find ( cf . id ) = = multiget_cf_data . end ( ) ) {
multiget_cf_data . insert ( { cf . id , new MultiGetColumnFamilyData ( ) } ) ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( cf ) ;
auto cfd = cfh - > cfd ( ) ;
if ( multiget_cf_data . find ( cfd - > GetID ( ) ) = = multiget_cf_data . end ( ) ) {
auto mgcfd = new MultiGetColumnFamilyData ( ) ;
mgcfd - > cfd = cfd ;
multiget_cf_data . insert ( { cfd - > GetID ( ) , mgcfd } ) ;
}
}
@ -2881,10 +2914,8 @@ std::vector<Status> DBImpl::MultiGet(
snapshot = versions_ - > LastSequence ( ) ;
}
for ( auto mgd_iter : multiget_cf_data ) {
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( mgd_iter . first ) ;
assert ( cfd ! = nullptr ) ;
mgd_iter . second - > cfd = cfd ;
mgd_iter . second - > super_version = cfd - > GetSuperVersion ( ) - > Ref ( ) ;
mgd_iter . second - > super_version =
mgd_iter . second - > cfd - > GetSuperVersion ( ) - > Ref ( ) ;
}
mutex_ . Unlock ( ) ;
@ -2910,7 +2941,8 @@ std::vector<Status> DBImpl::MultiGet(
std : : string * value = & ( * values ) [ i ] ;
LookupKey lkey ( keys [ i ] , snapshot ) ;
auto mgd_iter = multiget_cf_data . find ( column_family [ i ] . id ) ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family [ i ] ) ;
auto mgd_iter = multiget_cf_data . find ( cfh - > cfd ( ) - > GetID ( ) ) ;
assert ( mgd_iter ! = multiget_cf_data . end ( ) ) ;
auto mgd = mgd_iter - > second ;
auto super_version = mgd - > super_version ;
@ -2974,64 +3006,61 @@ std::vector<Status> DBImpl::MultiGet(
Status DBImpl : : CreateColumnFamily ( const ColumnFamilyOptions & options ,
const std : : string & column_family_name ,
ColumnFamilyHandle * handle ) {
// whenever we are writing to column family set, we have to lock
versions_ - > GetColumnFamilySet ( ) - > Lock ( ) ;
ColumnFamilyHandle * * handle ) {
mutex_ . Lock ( ) ;
if ( versions_ - > GetColumnFamilySet ( ) - > Exists ( column_family_name ) ) {
return Status : : InvalidArgument ( " Column family already exists " ) ;
}
VersionEdit edit ;
edit . AddColumnFamily ( column_family_name ) ;
handle - > id = versions_ - > GetColumnFamilySet ( ) - > GetNextColumnFamilyID ( ) ;
edit . SetColumnFamily ( handle - > id ) ;
uint32_t new_ id = versions_ - > GetColumnFamilySet ( ) - > GetNextColumnFamilyID ( ) ;
edit . SetColumnFamily ( new_ id) ;
mutex_ . Lock ( ) ;
Status s = versions_ - > LogAndApply ( default_cfd_ , & edit , & mutex_ ) ;
Status s = versions_ - > LogAndApply ( default_cf_handle_ - > cfd ( ) , & edit , & mutex_ ) ;
if ( s . ok ( ) ) {
// add to internal data structures
versions_ - > CreateColumnFamily ( options , & edit ) ;
auto cfd = versions_ - > CreateColumnFamily ( options , & edit ) ;
* handle = new ColumnFamilyHandleImpl ( cfd , this , & mutex_ ) ;
}
mutex_ . Unlock ( ) ;
versions_ - > GetColumnFamilySet ( ) - > Unlock ( ) ;
Log ( options_ . info_log , " Created column family \" %s \" " ,
column_family_name . c_str ( ) ) ;
return s ;
}
Status DBImpl : : DropColumnFamily ( const ColumnFamilyHandle & column_family ) {
if ( column_family . id = = 0 ) {
Status DBImpl : : DropColumnFamily ( ColumnFamilyHandle * column_family ) {
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
auto cfd = cfh - > cfd ( ) ;
if ( cfd - > GetID ( ) = = 0 ) {
return Status : : InvalidArgument ( " Can't drop default column family " ) ;
}
// whenever we are writing to column family set, we have to lock
versions_ - > GetColumnFamilySet ( ) - > Lock ( ) ;
if ( ! versions_ - > GetColumnFamilySet ( ) - > Exists ( column_family . id ) ) {
return Status : : NotFound ( " Column family not found " ) ;
}
Log ( options_ . info_log , " Dropping column family with id %u \n " , cfd - > GetID ( ) ) ;
VersionEdit edit ;
edit . DropColumnFamily ( ) ;
edit . SetColumnFamily ( column_family . id ) ;
mutex_ . Lock ( ) ;
Status s = versions_ - > LogAndApply ( default_cfd_ , & edit , & mutex_ ) ;
edit . SetColumnFamily ( cfd - > GetID ( ) ) ;
MutexLock l ( & mutex_ ) ;
if ( cfd - > IsDropped ( ) ) {
return Status : : InvalidArgument ( " Column family already dropped! \n " ) ;
}
Status s = versions_ - > LogAndApply ( cfd , & edit , & mutex_ ) ;
if ( s . ok ( ) ) {
// remove from internal data structures
versions_ - > DropColumnFamily ( & edit ) ;
cfd - > SetDropped ( ) ;
// DB is holding one reference to each column family when it's alive,
// need to drop it now
if ( cfd - > Unref ( ) ) {
delete cfd ;
}
}
versions_ - > GetColumnFamilySet ( ) - > Unlock ( ) ;
DeletionState deletion_state ;
FindObsoleteFiles ( deletion_state , false , true ) ;
mutex_ . Unlock ( ) ;
PurgeObsoleteFiles ( deletion_state ) ;
Log ( options_ . info_log , " Dropped column family with id %u \n " ,
column_family . id ) ;
return s ;
}
bool DBImpl : : KeyMayExist ( const ReadOptions & options ,
const ColumnFamilyHandle & column_family ,
const Slice & key , std : : string * value ,
bool * value_found ) {
ColumnFamilyHandle * column_family , const Slice & key ,
std : : string * value , bool * value_found ) {
if ( value_found ! = nullptr ) {
// falsify later if key-may-exist but can't fetch value
* value_found = true ;
@ -3047,12 +3076,12 @@ bool DBImpl::KeyMayExist(const ReadOptions& options,
}
Iterator * DBImpl : : NewIterator ( const ReadOptions & options ,
const ColumnFamilyHandle & column_family ) {
ColumnFamilyHandle * column_family ) {
SequenceNumber latest_snapshot = 0 ;
SuperVersion * super_version = nullptr ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
auto cfd = cfh - > cfd ( ) ;
mutex_ . Lock ( ) ;
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
assert ( cfd ! = nullptr ) ;
if ( ! options . tailing ) {
super_version = cfd - > GetSuperVersion ( ) - > Ref ( ) ;
latest_snapshot = versions_ - > LastSequence ( ) ;
@ -3083,7 +3112,7 @@ Iterator* DBImpl::NewIterator(const ReadOptions& options,
Status DBImpl : : NewIterators (
const ReadOptions & options ,
const std : : vector < ColumnFamilyHandle > & column_family ,
const std : : vector < ColumnFamilyHandle * > & column_family ,
std : : vector < Iterator * > * iterators ) {
// TODO
return Status : : NotSupported ( " Not yet! " ) ;
@ -3100,20 +3129,15 @@ void DBImpl::ReleaseSnapshot(const Snapshot* s) {
}
// Convenience methods
Status DBImpl : : Put ( const WriteOptions & o ,
const ColumnFamilyHandle & column_family , const Slice & key ,
const Slice & val ) {
Status DBImpl : : Put ( const WriteOptions & o , ColumnFamilyHandle * column_family ,
const Slice & key , const Slice & val ) {
return DB : : Put ( o , column_family , key , val ) ;
}
Status DBImpl : : Merge ( const WriteOptions & o ,
const ColumnFamilyHandle & column_family , const Slice & key ,
const Slice & val ) {
mutex_ . Lock ( ) ;
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
mutex_ . Unlock ( ) ;
assert ( cfd ! = nullptr ) ;
if ( ! cfd - > options ( ) - > merge_operator ) {
Status DBImpl : : Merge ( const WriteOptions & o , ColumnFamilyHandle * column_family ,
const Slice & key , const Slice & val ) {
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
if ( ! cfh - > cfd ( ) - > options ( ) - > merge_operator ) {
return Status : : NotSupported ( " Provide a merge_operator when opening DB " ) ;
} else {
return DB : : Merge ( o , column_family , key , val ) ;
@ -3121,8 +3145,7 @@ Status DBImpl::Merge(const WriteOptions& o,
}
Status DBImpl : : Delete ( const WriteOptions & options ,
const ColumnFamilyHandle & column_family ,
const Slice & key ) {
ColumnFamilyHandle * column_family , const Slice & key ) {
return DB : : Delete ( options , column_family , key ) ;
}
@ -3155,13 +3178,22 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
}
Status status ;
autovector < ColumnFamilyData * > to_delete ;
// refcounting cfd in iteration
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
cfd - > Ref ( ) ;
// May temporarily unlock and wait.
status = MakeRoomForWrite ( cfd , my_batch = = nullptr ) ;
if ( cfd - > Unref ( ) ) {
to_delete . push_back ( cfd ) ;
}
if ( ! status . ok ( ) ) {
break ;
}
}
for ( auto cfd : to_delete ) {
delete cfd ;
}
uint64_t last_sequence = versions_ - > LastSequence ( ) ;
Writer * last_writer = & w ;
if ( status . ok ( ) & & my_batch ! = nullptr ) { // nullptr batch is for compactions
@ -3221,13 +3253,10 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
if ( status . ok ( ) ) {
StopWatchNano write_memtable_timer ( env_ , false ) ;
// reading the column family set outside of DB mutex -- should lock
versions_ - > GetColumnFamilySet ( ) - > Lock ( ) ;
StartPerfTimer ( & write_memtable_timer ) ;
status = WriteBatchInternal : : InsertInto (
updates , column_family_memtables_ . get ( ) , 0 , this , false ) ;
BumpPerfTime ( & perf_context . write_memtable_time , & write_memtable_timer ) ;
versions_ - > GetColumnFamilySet ( ) - > Unlock ( ) ;
if ( ! status . ok ( ) ) {
// Panic for in-memory corruptions
@ -3536,29 +3565,28 @@ Env* DBImpl::GetEnv() const {
return env_ ;
}
const Options & DBImpl : : GetOptions ( const ColumnFamilyHandle & column_family )
const {
return * default_cfd_ - > full_options ( ) ;
const Options & DBImpl : : GetOptions ( ColumnFamilyHandle * column_family ) const {
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
return * cfh - > cfd ( ) - > full_options ( ) ;
}
bool DBImpl : : GetProperty ( const ColumnFamilyHandle & column_family ,
bool DBImpl : : GetProperty ( ColumnFamilyHandle * column_family ,
const Slice & property , std : : string * value ) {
value - > clear ( ) ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
auto cfd = cfh - > cfd ( ) ;
MutexLock l ( & mutex_ ) ;
auto cfd = versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
assert ( cfd ! = nullptr ) ;
return cfd - > internal_stats ( ) - > GetProperty ( property , value , cfd ) ;
}
void DBImpl : : GetApproximateSizes ( const ColumnFamilyHandle & column_family ,
void DBImpl : : GetApproximateSizes ( ColumnFamilyHandle * column_family ,
const Range * range , int n , uint64_t * sizes ) {
// TODO(opt): better implementation
Version * v ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
auto cfd = cfh - > cfd ( ) ;
{
MutexLock l ( & mutex_ ) ;
auto cfd =
versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( column_family . id ) ;
assert ( cfd ! = nullptr ) ;
v = cfd - > current ( ) ;
v - > Ref ( ) ;
}
@ -3654,9 +3682,9 @@ Status DBImpl::DeleteFile(std::string name) {
return status ;
}
void DBImpl : : GetLiveFilesMetaData ( std : : vector < LiveFileMetaData > * metadata ) {
void DBImpl : : GetLiveFilesMetaData ( std : : vector < LiveFileMetaData > * metadata ) {
MutexLock l ( & mutex_ ) ;
return versions_ - > GetLiveFilesMetaData ( metadata ) ;
versions_ - > GetLiveFilesMetaData ( metadata ) ;
}
Status DBImpl : : GetDbIdentity ( std : : string & identity ) {
@ -3688,38 +3716,40 @@ Status DBImpl::GetDbIdentity(std::string& identity) {
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
Status DB : : Put ( const WriteOptions & opt , const ColumnFamilyHandle & column_family ,
Status DB : : Put ( const WriteOptions & opt , ColumnFamilyHandle * column_family ,
const Slice & key , const Slice & value ) {
// Pre-allocate size of write batch conservatively.
// 8 bytes are taken by header, 4 bytes for count, 1 byte for type,
// and we allocate 11 extra bytes for key length, as well as value length.
WriteBatch batch ( key . size ( ) + value . size ( ) + 24 ) ;
batch . Put ( column_family . id , key , value ) ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
batch . Put ( cfh - > cfd ( ) - > GetID ( ) , key , value ) ;
return Write ( opt , & batch ) ;
}
Status DB : : Delete ( const WriteOptions & opt ,
const ColumnFamilyHandle & column_family , const Slice & key ) {
Status DB : : Delete ( const WriteOptions & opt , ColumnFamilyHandle * column_family ,
const Slice & key ) {
WriteBatch batch ;
batch . Delete ( column_family . id , key ) ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
batch . Delete ( cfh - > cfd ( ) - > GetID ( ) , key ) ;
return Write ( opt , & batch ) ;
}
Status DB : : Merge ( const WriteOptions & opt ,
const ColumnFamilyHandle & column_family , const Slice & key ,
const Slice & value ) {
Status DB : : Merge ( const WriteOptions & opt , ColumnFamilyHandle * column_family ,
const Slice & key , const Slice & value ) {
WriteBatch batch ;
batch . Merge ( column_family . id , key , value ) ;
auto cfh = reinterpret_cast < ColumnFamilyHandleImpl * > ( column_family ) ;
batch . Merge ( cfh - > cfd ( ) - > GetID ( ) , key , value ) ;
return Write ( opt , & batch ) ;
}
// Default implementation -- returns not supported status
Status DB : : CreateColumnFamily ( const ColumnFamilyOptions & options ,
const std : : string & column_family_name ,
ColumnFamilyHandle * handle ) {
ColumnFamilyHandle * * handle ) {
return Status : : NotSupported ( " " ) ;
}
Status DB : : DropColumnFamily ( const ColumnFamilyHandle & column_family ) {
Status DB : : DropColumnFamily ( ColumnFamilyHandle * column_family ) {
return Status : : NotSupported ( " " ) ;
}
@ -3731,14 +3761,22 @@ Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
std : : vector < ColumnFamilyDescriptor > column_families ;
column_families . push_back (
ColumnFamilyDescriptor ( default_column_family_name , cf_options ) ) ;
std : : vector < ColumnFamilyHandle > handles ;
return DB : : Open ( db_options , dbname , column_families , & handles , dbptr ) ;
std : : vector < ColumnFamilyHandle * > handles ;
Status s = DB : : Open ( db_options , dbname , column_families , & handles , dbptr ) ;
if ( s . ok ( ) ) {
assert ( handles . size ( ) = = 1 ) ;
// i can delete the handle since DBImpl is always holding a reference to
// default column family
delete handles [ 0 ] ;
}
return s ;
}
Status DB : : Open ( const DBOptions & db_options , const std : : string & dbname ,
const std : : vector < ColumnFamilyDescriptor > & column_families ,
std : : vector < ColumnFamilyHandle > * handles , DB * * dbptr ) {
std : : vector < ColumnFamilyHandle * > * handles , DB * * dbptr ) {
* dbptr = nullptr ;
handles - > clear ( ) ;
EnvOptions soptions ;
size_t max_write_buffer_size = 0 ;
@ -3784,20 +3822,22 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname,
// that we used by calling impl->versions_->NewFileNumber()
// The used log number are already written to manifest in RecoverLogFile()
// method
s = impl - > versions_ - > LogAndApply ( impl - > default_cfd_ , & edit , & impl - > mutex_ ,
s = impl - > versions_ - > LogAndApply ( impl - > default_cf_handle_ - > cfd ( ) , & edit ,
& impl - > mutex_ ,
impl - > db_directory_ . get ( ) ) ;
}
if ( s . ok ( ) ) {
// set column family handles
handles - > clear ( ) ;
for ( auto cf : column_families ) {
if ( ! impl - > versions_ - > GetColumnFamilySet ( ) - > Exists ( cf . name ) ) {
s = Status : : InvalidArgument ( " Column family not found: " , cf . name ) ;
handles - > clear ( ) ;
break ;
}
uint32_t id = impl - > versions_ - > GetColumnFamilySet ( ) - > GetID ( cf . name ) ;
handles - > push_back ( ColumnFamilyHandle ( id ) ) ;
auto cfd = impl - > versions_ - > GetColumnFamilySet ( ) - > GetColumnFamily ( id ) ;
assert ( cfd ! = nullptr ) ;
handles - > push_back (
new ColumnFamilyHandleImpl ( cfd , impl , & impl - > mutex_ ) ) ;
}
}
if ( s . ok ( ) ) {
@ -3836,6 +3876,9 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname,
if ( s . ok ( ) ) {
* dbptr = impl ;
} else {
for ( auto h : * handles ) {
delete h ;
}
handles - > clear ( ) ;
delete impl ;
}