-
Notifications
You must be signed in to change notification settings - Fork 6.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Track the total number of compaction sorted runs #13320
base: main
Are you sure you want to change the base?
Changes from 6 commits
87399e4
2c2ba95
1e15c3d
e14d51a
010fdd4
6a71c06
00f5c9a
9fec093
f73c9eb
68ea158
03dd23a
21fa2c2
b8230f3
cae1281
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -2417,8 +2417,9 @@ class DBImpl : public DB { | |
Env::Priority thread_pri); | ||
void BackgroundCallFlush(Env::Priority thread_pri); | ||
void BackgroundCallPurge(); | ||
Status BackgroundCompaction(bool* madeProgress, JobContext* job_context, | ||
LogBuffer* log_buffer, | ||
Status BackgroundCompaction(bool* madeProgress, | ||
int& num_compaction_iterators_added, | ||
JobContext* job_context, LogBuffer* log_buffer, | ||
PrepickedCompaction* prepicked_compaction, | ||
Env::Priority thread_pri); | ||
Status BackgroundFlush(bool* madeProgress, JobContext* job_context, | ||
|
@@ -2430,6 +2431,8 @@ class DBImpl : public DB { | |
const std::vector<CompactionInputFiles>& inputs, | ||
bool* sfm_bookkeeping, LogBuffer* log_buffer); | ||
|
||
size_t GetNumberCompactionInputIterators(Compaction* c); | ||
|
||
// Request compaction tasks token from compaction thread limiter. | ||
// It always succeeds if force = true or limiter is disable. | ||
bool RequestCompactionToken(ColumnFamilyData* cfd, bool force, | ||
|
@@ -2963,6 +2966,10 @@ class DBImpl : public DB { | |
// stores the number of compactions are currently running | ||
int num_running_compactions_; | ||
|
||
// stores the number of input iterators required for currently running | ||
// compactions | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
int num_running_compaction_input_iterators_; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see we have I guess that means we do not need |
||
|
||
// number of background memtable flush jobs, submitted to the HIGH pool | ||
int bg_flush_scheduled_; | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -61,6 +61,17 @@ bool DBImpl::EnoughRoomForCompaction( | |
return enough_room; | ||
} | ||
|
||
size_t DBImpl::GetNumberCompactionInputIterators(Compaction* c) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We should check for and skip trivial moves and deletion compactions since they won't have any input iterators There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah, good catch. TIL about "deletion compactions" |
||
assert(c); | ||
if (c->start_level() == 0) { | ||
assert(0 < c->num_input_levels()); | ||
size_t num_l0_files = c->num_input_files(0); | ||
size_t num_non_l0_levels = c->num_input_levels() - 1; | ||
return num_l0_files + num_non_l0_levels; | ||
} | ||
return c->num_input_levels(); | ||
} | ||
|
||
bool DBImpl::RequestCompactionToken(ColumnFamilyData* cfd, bool force, | ||
std::unique_ptr<TaskLimiterToken>* token, | ||
LogBuffer* log_buffer) { | ||
|
@@ -3418,8 +3429,14 @@ void DBImpl::BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction, | |
assert((bg_thread_pri == Env::Priority::BOTTOM && | ||
bg_bottom_compaction_scheduled_) || | ||
(bg_thread_pri == Env::Priority::LOW && bg_compaction_scheduled_)); | ||
Status s = BackgroundCompaction(&made_progress, &job_context, &log_buffer, | ||
prepicked_compaction, bg_thread_pri); | ||
|
||
// BackgroundCompaction will update the | ||
// num_running_compaction_input_iterators_ total and later we will subtract | ||
// what was added | ||
int num_compaction_input_iterators_added = 0; | ||
Status s = BackgroundCompaction( | ||
&made_progress, num_compaction_input_iterators_added, &job_context, | ||
&log_buffer, prepicked_compaction, bg_thread_pri); | ||
TEST_SYNC_POINT("BackgroundCallCompaction:1"); | ||
if (s.IsBusy()) { | ||
bg_cv_.SignalAll(); // In case a waiter can proceed despite the error | ||
|
@@ -3484,6 +3501,11 @@ void DBImpl::BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction, | |
|
||
assert(num_running_compactions_ > 0); | ||
num_running_compactions_--; | ||
assert(num_running_compaction_input_iterators_ >= 0); | ||
assert(num_running_compaction_input_iterators_ >= | ||
num_compaction_input_iterators_added); | ||
num_running_compaction_input_iterators_ -= | ||
num_compaction_input_iterators_added; | ||
|
||
if (bg_thread_pri == Env::Priority::LOW) { | ||
bg_compaction_scheduled_--; | ||
|
@@ -3523,6 +3545,7 @@ void DBImpl::BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction, | |
} | ||
|
||
Status DBImpl::BackgroundCompaction(bool* made_progress, | ||
int& num_compaction_input_iterators_added, | ||
JobContext* job_context, | ||
LogBuffer* log_buffer, | ||
PrepickedCompaction* prepicked_compaction, | ||
|
@@ -3720,6 +3743,12 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, | |
num_files += each_level.files.size(); | ||
} | ||
RecordInHistogram(stats_, NUM_FILES_IN_SINGLE_COMPACTION, num_files); | ||
num_compaction_input_iterators_added = | ||
static_cast<int>(GetNumberCompactionInputIterators(c.get())); | ||
assert(num_compaction_input_iterators_added >= 0); | ||
assert(num_running_compaction_input_iterators_ >= 0); | ||
num_running_compaction_input_iterators_ += | ||
num_compaction_input_iterators_added; | ||
|
||
// There are three things that can change compaction score: | ||
// 1) When flush or compaction finish. This case is covered by | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
C++ style guide suggests that output parameters go at the end