@ -71,7 +71,6 @@ struct CompactionJob::CompactionState {
SequenceNumber smallest_seqno , largest_seqno ;
} ;
std : : vector < Output > outputs ;
std : : list < uint64_t > allocated_file_numbers ;
// State kept for output being generated
std : : unique_ptr < WritableFile > outfile ;
@ -204,10 +203,10 @@ struct CompactionJob::CompactionState {
CompactionJob : : CompactionJob (
Compaction * compaction , const DBOptions & db_options ,
const MutableCFOptions & mutable_cf_options , const EnvOptions & env_options ,
VersionSet * versions , port : : Mutex * db_mutex ,
std : : atomic < bool > * shutting_down , LogBuffer * log_buffer ,
Directory * db_directory , Statistics * stats , SnapshotList * snapshots ,
bool is_snapshot_supported , std : : shared_ptr < Cache > table_cache ,
VersionSet * versions , std : : atomic < bool > * shutting_down ,
LogBuffer * log_buffer , Directory * db_directory , Statistics * stats ,
SnapshotList * snapshots , bool is_snapshot_supported ,
std : : shared_ptr < Cache > table_cache ,
std : : function < uint64_t ( ) > yield_callback )
: compact_ ( new CompactionState ( compaction ) ) ,
compaction_stats_ ( 1 ) ,
@ -216,7 +215,6 @@ CompactionJob::CompactionJob(
env_options_ ( env_options ) ,
env_ ( db_options . env ) ,
versions_ ( versions ) ,
db_mutex_ ( db_mutex ) ,
shutting_down_ ( shutting_down ) ,
log_buffer_ ( log_buffer ) ,
db_directory_ ( db_directory ) ,
@ -227,7 +225,6 @@ CompactionJob::CompactionJob(
yield_callback_ ( std : : move ( yield_callback ) ) { }
void CompactionJob : : Prepare ( ) {
db_mutex_ - > AssertHeld ( ) ;
compact_ - > CleanupBatchBuffer ( ) ;
compact_ - > CleanupMergedBuffer ( ) ;
@ -267,9 +264,6 @@ void CompactionJob::Prepare() {
// Is this compaction producing files at the bottommost level?
bottommost_level_ = compact_ - > compaction - > BottomMostLevel ( ) ;
// Allocate the output file numbers before we release the lock
AllocateCompactionOutputFileNumbers ( ) ;
}
Status CompactionJob : : Run ( ) {
@ -461,14 +455,14 @@ Status CompactionJob::Run() {
return status ;
}
Status CompactionJob : : Install ( Status status ) {
db_mutex_ - > AssertHeld ( ) ;
Status CompactionJob : : Install ( Status status , port : : Mutex * db_mutex ) {
db_mutex - > AssertHeld ( ) ;
ColumnFamilyData * cfd = compact_ - > compaction - > column_family_data ( ) ;
cfd - > internal_stats ( ) - > AddCompactionStats (
compact_ - > compaction - > output_level ( ) , compaction_stats_ ) ;
if ( status . ok ( ) ) {
status = InstallCompactionResults ( ) ;
status = InstallCompactionResults ( db_mutex ) ;
}
VersionStorageInfo : : LevelSummaryStorage tmp ;
const auto & stats = compaction_stats_ ;
@ -496,19 +490,6 @@ Status CompactionJob::Install(Status status) {
return status ;
}
// Allocate the file numbers for the output file. We allocate as
// many output file numbers as there are files in level+1 (at least one)
// Insert them into pending_outputs so that they do not get deleted.
void CompactionJob : : AllocateCompactionOutputFileNumbers ( ) {
db_mutex_ - > AssertHeld ( ) ;
assert ( compact_ - > builder = = nullptr ) ;
int filesNeeded = compact_ - > compaction - > num_input_files ( 1 ) ;
for ( int i = 0 ; i < std : : max ( filesNeeded , 1 ) ; i + + ) {
uint64_t file_number = versions_ - > NewFileNumber ( ) ;
compact_ - > allocated_file_numbers . push_back ( file_number ) ;
}
}
Status CompactionJob : : ProcessKeyValueCompaction ( int64_t * imm_micros ,
Iterator * input ,
bool is_compaction_v2 ) {
@ -958,8 +939,8 @@ Status CompactionJob::FinishCompactionOutputFile(Iterator* input) {
return s ;
}
Status CompactionJob : : InstallCompactionResults ( ) {
db_mutex_ - > AssertHeld ( ) ;
Status CompactionJob : : InstallCompactionResults ( port : : Mutex * db_mutex ) {
db_mutex - > AssertHeld ( ) ;
// paranoia: verify that the files that we started with
// still exist in the current version and in the same original level.
@ -995,7 +976,7 @@ Status CompactionJob::InstallCompactionResults() {
}
return versions_ - > LogAndApply (
compact_ - > compaction - > column_family_data ( ) , mutable_cf_options_ ,
compact_ - > compaction - > edit ( ) , db_mutex_ , db_directory_ ) ;
compact_ - > compaction - > edit ( ) , db_mutex , db_directory_ ) ;
}
// Given a sequence number, return the sequence number of the
@ -1036,21 +1017,8 @@ void CompactionJob::RecordCompactionIOStats() {
Status CompactionJob : : OpenCompactionOutputFile ( ) {
assert ( compact_ ! = nullptr ) ;
assert ( compact_ - > builder = = nullptr ) ;
uint64_t file_number ;
// If we have not yet exhausted the pre-allocated file numbers,
// then use the one from the front. Otherwise, we have to acquire
// the heavyweight lock and allocate a new file number.
if ( ! compact_ - > allocated_file_numbers . empty ( ) ) {
file_number = compact_ - > allocated_file_numbers . front ( ) ;
compact_ - > allocated_file_numbers . pop_front ( ) ;
} else {
db_mutex_ - > Lock ( ) ;
// TODO(icanadi) make Versions::next_file_number_ atomic and remove db_lock
// around here. Once we do that, AllocateCompactionOutputFileNumbers() will
// not be needed.
file_number = versions_ - > NewFileNumber ( ) ;
db_mutex_ - > Unlock ( ) ;
}
// no need to lock because VersionSet::next_file_number_ is atomic
uint64_t file_number = versions_ - > NewFileNumber ( ) ;
// Make the output file
std : : string fname = TableFileName ( db_options_ . db_paths , file_number ,
compact_ - > compaction - > GetOutputPathId ( ) ) ;
@ -1087,7 +1055,6 @@ Status CompactionJob::OpenCompactionOutputFile() {
}
void CompactionJob : : CleanupCompaction ( Status status ) {
db_mutex_ - > AssertHeld ( ) ;
if ( compact_ - > builder ! = nullptr ) {
// May happen if we get a shutdown call in the middle of compaction
compact_ - > builder - > Abandon ( ) ;