@ -322,6 +322,7 @@ Status CompactionJob::Run() {
shared_ptr < Iterator > backup_input (
shared_ptr < Iterator > backup_input (
versions_ - > MakeInputIterator ( compact_ - > compaction ) ) ;
versions_ - > MakeInputIterator ( compact_ - > compaction ) ) ;
backup_input - > SeekToFirst ( ) ;
backup_input - > SeekToFirst ( ) ;
uint64_t total_filter_time = 0 ;
while ( backup_input - > Valid ( ) & &
while ( backup_input - > Valid ( ) & &
! shutting_down_ - > load ( std : : memory_order_acquire ) & &
! shutting_down_ - > load ( std : : memory_order_acquire ) & &
! cfd - > IsDropped ( ) ) {
! cfd - > IsDropped ( ) ) {
@ -369,7 +370,9 @@ Status CompactionJob::Run() {
// Now prefix changes, this batch is done.
// Now prefix changes, this batch is done.
// Call compaction filter on the buffered values to change the value
// Call compaction filter on the buffered values to change the value
if ( compact_ - > key_str_buf_ . size ( ) > 0 ) {
if ( compact_ - > key_str_buf_ . size ( ) > 0 ) {
CallCompactionFilterV2 ( compaction_filter_v2 ) ;
uint64_t time = 0 ;
CallCompactionFilterV2 ( compaction_filter_v2 , & time ) ;
total_filter_time + = time ;
}
}
compact_ - > cur_prefix_ = key_prefix . ToString ( ) ;
compact_ - > cur_prefix_ = key_prefix . ToString ( ) ;
}
}
@ -401,7 +404,9 @@ Status CompactionJob::Run() {
if ( ! backup_input - > Valid ( ) ) {
if ( ! backup_input - > Valid ( ) ) {
// If this is the single last value, we need to merge it.
// If this is the single last value, we need to merge it.
if ( compact_ - > key_str_buf_ . size ( ) > 0 ) {
if ( compact_ - > key_str_buf_ . size ( ) > 0 ) {
CallCompactionFilterV2 ( compaction_filter_v2 ) ;
uint64_t time = 0 ;
CallCompactionFilterV2 ( compaction_filter_v2 , & time ) ;
total_filter_time + = time ;
}
}
compact_ - > MergeKeyValueSliceBuffer ( & cfd - > internal_comparator ( ) ) ;
compact_ - > MergeKeyValueSliceBuffer ( & cfd - > internal_comparator ( ) ) ;
@ -417,11 +422,14 @@ Status CompactionJob::Run() {
// finish the last batch
// finish the last batch
if ( status . ok ( ) ) {
if ( status . ok ( ) ) {
if ( compact_ - > key_str_buf_ . size ( ) > 0 ) {
if ( compact_ - > key_str_buf_ . size ( ) > 0 ) {
CallCompactionFilterV2 ( compaction_filter_v2 ) ;
uint64_t time = 0 ;
CallCompactionFilterV2 ( compaction_filter_v2 , & time ) ;
total_filter_time + = time ;
}
}
compact_ - > MergeKeyValueSliceBuffer ( & cfd - > internal_comparator ( ) ) ;
compact_ - > MergeKeyValueSliceBuffer ( & cfd - > internal_comparator ( ) ) ;
status = ProcessKeyValueCompaction ( & imm_micros , input . get ( ) , true ) ;
status = ProcessKeyValueCompaction ( & imm_micros , input . get ( ) , true ) ;
}
}
RecordTick ( stats_ , FILTER_OPERATION_TOTAL_TIME , total_filter_time ) ;
} // checking for compaction filter v2
} // checking for compaction filter v2
if ( status . ok ( ) & &
if ( status . ok ( ) & &
@ -556,6 +564,9 @@ Status CompactionJob::ProcessKeyValueCompaction(int64_t* imm_micros,
int64_t key_drop_newer_entry = 0 ;
int64_t key_drop_newer_entry = 0 ;
int64_t key_drop_obsolete = 0 ;
int64_t key_drop_obsolete = 0 ;
int64_t loop_cnt = 0 ;
int64_t loop_cnt = 0 ;
StopWatchNano timer ( env_ , stats_ ! = nullptr ) ;
uint64_t total_filter_time = 0 ;
while ( input - > Valid ( ) & & ! shutting_down_ - > load ( std : : memory_order_acquire ) & &
while ( input - > Valid ( ) & & ! shutting_down_ - > load ( std : : memory_order_acquire ) & &
! cfd - > IsDropped ( ) & & status . ok ( ) ) {
! cfd - > IsDropped ( ) & & status . ok ( ) ) {
compact_ - > num_input_records + + ;
compact_ - > num_input_records + + ;
@ -642,9 +653,13 @@ Status CompactionJob::ProcessKeyValueCompaction(int64_t* imm_micros,
// the entry with a delete marker.
// the entry with a delete marker.
bool value_changed = false ;
bool value_changed = false ;
compaction_filter_value . clear ( ) ;
compaction_filter_value . clear ( ) ;
if ( stats_ ! = nullptr ) {
timer . Start ( ) ;
}
bool to_delete = compaction_filter - > Filter (
bool to_delete = compaction_filter - > Filter (
compact_ - > compaction - > level ( ) , ikey . user_key , value ,
compact_ - > compaction - > level ( ) , ikey . user_key , value ,
& compaction_filter_value , & value_changed ) ;
& compaction_filter_value , & value_changed ) ;
total_filter_time + = timer . ElapsedNanos ( ) ;
if ( to_delete ) {
if ( to_delete ) {
// make a copy of the original key and convert it to a delete
// make a copy of the original key and convert it to a delete
delete_key . SetInternalKey ( ExtractUserKey ( key ) , ikey . sequence ,
delete_key . SetInternalKey ( ExtractUserKey ( key ) , ikey . sequence ,
@ -712,7 +727,7 @@ Status CompactionJob::ProcessKeyValueCompaction(int64_t* imm_micros,
// optimization in BuildTable.
// optimization in BuildTable.
int steps = 0 ;
int steps = 0 ;
merge . MergeUntil ( input , prev_snapshot , bottommost_level_ ,
merge . MergeUntil ( input , prev_snapshot , bottommost_level_ ,
db_options_ . statistics . get ( ) , & steps ) ;
db_options_ . statistics . get ( ) , & steps , env_ ) ;
// Skip the Merge ops
// Skip the Merge ops
combined_idx = combined_idx - 1 + steps ;
combined_idx = combined_idx - 1 + steps ;
@ -844,6 +859,7 @@ Status CompactionJob::ProcessKeyValueCompaction(int64_t* imm_micros,
input - > Next ( ) ;
input - > Next ( ) ;
}
}
}
}
RecordTick ( stats_ , FILTER_OPERATION_TOTAL_TIME , total_filter_time ) ;
if ( key_drop_user > 0 ) {
if ( key_drop_user > 0 ) {
RecordTick ( stats_ , COMPACTION_KEY_DROP_USER , key_drop_user ) ;
RecordTick ( stats_ , COMPACTION_KEY_DROP_USER , key_drop_user ) ;
}
}
@ -859,7 +875,7 @@ Status CompactionJob::ProcessKeyValueCompaction(int64_t* imm_micros,
}
}
void CompactionJob : : CallCompactionFilterV2 (
void CompactionJob : : CallCompactionFilterV2 (
CompactionFilterV2 * compaction_filter_v2 ) {
CompactionFilterV2 * compaction_filter_v2 , uint64_t * time ) {
if ( compact_ = = nullptr | | compaction_filter_v2 = = nullptr ) {
if ( compact_ = = nullptr | | compaction_filter_v2 = = nullptr ) {
return ;
return ;
}
}
@ -889,10 +905,11 @@ void CompactionJob::CallCompactionFilterV2(
// filter.
// filter.
// If the return value of the compaction filter is true, replace
// If the return value of the compaction filter is true, replace
// the entry with a delete marker.
// the entry with a delete marker.
StopWatchNano timer ( env_ , stats_ ! = nullptr ) ;
compact_ - > to_delete_buf_ = compaction_filter_v2 - > Filter (
compact_ - > to_delete_buf_ = compaction_filter_v2 - > Filter (
compact_ - > compaction - > level ( ) , user_key_buf , existing_value_buf ,
compact_ - > compaction - > level ( ) , user_key_buf , existing_value_buf ,
& compact_ - > new_value_buf_ , & compact_ - > value_changed_buf_ ) ;
& compact_ - > new_value_buf_ , & compact_ - > value_changed_buf_ ) ;
* time = timer . ElapsedNanos ( ) ;
// new_value_buf_.size() <= to_delete__buf_.size(). "=" iff all
// new_value_buf_.size() <= to_delete__buf_.size(). "=" iff all
// kv-pairs in this compaction run needs to be deleted.
// kv-pairs in this compaction run needs to be deleted.
assert ( compact_ - > to_delete_buf_ . size ( ) = = compact_ - > key_str_buf_ . size ( ) ) ;
assert ( compact_ - > to_delete_buf_ . size ( ) = = compact_ - > key_str_buf_ . size ( ) ) ;