Skip to content
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 inside BackgroundCallCompaction #13320

Closed

Conversation

archang19
Copy link
Contributor

@archang19 archang19 commented Jan 21, 2025

Going to try and opt for #13325 instead

@archang19 archang19 changed the title Add num-running-compaction-iterators stat Add stat for total number of compaction input iterators Jan 21, 2025
@archang19 archang19 changed the title Add stat for total number of compaction input iterators Track the total number of compaction input iterators Jan 21, 2025
@archang19 archang19 force-pushed the internal-stats-total-compaction branch from 5e67c21 to 6a71c06 Compare January 21, 2025 23:59
@@ -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
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

uint64_t or size_t makes more sense to me since num_running_compaction_input_iterators_ should not be negative. However, I wanted to follow the convention here since everything else is using int

@@ -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
int num_running_compaction_input_iterators_;
Copy link
Contributor Author

@archang19 archang19 Jan 22, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see we have InstrumentedMutexLock l(&mutex_); protecting num_running_compactions_ as well as num_running_flushes_.

I guess that means we do not need std::atomic<int> for num_running_compaction_input_iterators_

@archang19 archang19 marked this pull request as ready for review January 22, 2025 02:40
@facebook-github-bot
Copy link
Contributor

@archang19 has imported this pull request. If you are a Meta employee, you can view this diff on Phabricator.

@facebook-github-bot
Copy link
Contributor

@archang19 has updated the pull request. You must reimport the pull request before landing.

@facebook-github-bot
Copy link
Contributor

@archang19 has imported this pull request. If you are a Meta employee, you can view this diff on Phabricator.

@@ -957,6 +957,13 @@ class DBImpl : public DB {
return num_running_compactions_;
}

// Returns the number of input iterators for currently running compactions.
// REQUIREMENT: mutex_ must be held when calling this function.
int num_running_compaction_input_iterators() {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I noticed num_running_compactions did not follow the example of num_running_flushes, where db->num_running_flushes() gets called inside HandleNumRunningFlushes. HandleNumRunningCompactions was just accessing num_running_compactions_ directly.

@facebook-github-bot
Copy link
Contributor

@archang19 has updated the pull request. You must reimport the pull request before landing.

@facebook-github-bot
Copy link
Contributor

@archang19 has imported this pull request. If you are a Meta employee, you can view this diff on Phabricator.

@archang19 archang19 requested review from anand1976 and cbi42 January 22, 2025 17:12
@archang19
Copy link
Contributor Author

@cbi42 I added you since this touches compaction code. Perhaps you know a better way of doing this or even another way to obtain the same information

@@ -61,6 +61,18 @@ bool DBImpl::EnoughRoomForCompaction(
return enough_room;
}

size_t DBImpl::GetNumberCompactionInputIterators(Compaction* c) {
Copy link
Contributor

Choose a reason for hiding this comment

The 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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, good catch. TIL about "deletion compactions"

@@ -1261,7 +1269,13 @@ bool InternalStats::HandleCompactionPending(uint64_t* value, DBImpl* /*db*/,

bool InternalStats::HandleNumRunningCompactions(uint64_t* value, DBImpl* db,
Version* /*version*/) {
*value = db->num_running_compactions_;
*value = db->num_running_compactions();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Doesn't num_running_compactions() assert that DB mutex is held?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, it does. I think (?) that is what we want. num_running_flushes() also checks mutex_.AssertHeld(); and is called from HandleNumRunningFlushes

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok I saw your other comment. I will update this PR to just read off the values.


bool InternalStats::HandleNumRunningCompactionInputIterators(
uint64_t* value, DBImpl* db, Version* /*version*/) {
*value = db->num_running_compaction_input_iterators();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We're not holding the mutex here. Since these are stats, it should be ok to directly read the counter even if its not 100% accurate.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should I also get rid of num_running_flushes(), which asserts the mutex is held?

Status BackgroundCompaction(bool* madeProgress, JobContext* job_context,
LogBuffer* log_buffer,
Status BackgroundCompaction(bool* madeProgress,
int& num_compaction_iterators_added,
Copy link
Contributor

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

@cbi42
Copy link
Member

cbi42 commented Jan 22, 2025

I think the name "input iterator" can be confusing, especially since this will be a public API. How about using "sorted runs" instead?

I think subcompactions can affect this since each subcompaction creates its own iterator.

@facebook-github-bot
Copy link
Contributor

@archang19 has updated the pull request. You must reimport the pull request before landing.

@archang19
Copy link
Contributor Author

archang19 commented Jan 22, 2025

I think the name "input iterator" can be confusing, especially since this will be a public API. How about using "sorted runs" instead?

I think that name works. I will update this PR

I think subcompactions can affect this since each subcompaction creates its own iterator.

@cbi42 do you mean that the actual number of iterators (sorted runs) would be more than the number of L0 files + the number of non-L0 levels when there are subcompactions?

@facebook-github-bot
Copy link
Contributor

@archang19 has updated the pull request. You must reimport the pull request before landing.

1 similar comment
@facebook-github-bot
Copy link
Contributor

@archang19 has updated the pull request. You must reimport the pull request before landing.

@archang19 archang19 changed the title Track the total number of compaction input iterators Track the total number of compaction sorted runs Jan 22, 2025
@archang19 archang19 force-pushed the internal-stats-total-compaction branch from 1231ef7 to 21fa2c2 Compare January 22, 2025 18:28
@facebook-github-bot
Copy link
Contributor

@archang19 has updated the pull request. You must reimport the pull request before landing.

@facebook-github-bot
Copy link
Contributor

@archang19 has imported this pull request. If you are a Meta employee, you can view this diff on Phabricator.

@facebook-github-bot
Copy link
Contributor

@archang19 has updated the pull request. You must reimport the pull request before landing.

@facebook-github-bot
Copy link
Contributor

@archang19 has imported this pull request. If you are a Meta employee, you can view this diff on Phabricator.

@cbi42
Copy link
Member

cbi42 commented Jan 22, 2025

@cbi42 do you mean that the actual number of iterators (sorted runs) would be more than the number of L0 files + the number of non-L0 levels when there are subcompactions?

Yes, each subcompaction creates input iterators separately.

@facebook-github-bot
Copy link
Contributor

@archang19 has updated the pull request. You must reimport the pull request before landing.

@archang19
Copy link
Contributor Author

@cbi42 do you mean that the actual number of iterators (sorted runs) would be more than the number of L0 files + the number of non-L0 levels when there are subcompactions?

Yes, each subcompaction creates input iterators separately.

@anand1976 do you think this subcompaction exception will be problematic for our purposes? I searched around and found a NUM_SUBCOMPACTIONS_SCHEDULED histogram that we could use.

@facebook-github-bot
Copy link
Contributor

@archang19 has imported this pull request. If you are a Meta employee, you can view this diff on Phabricator.

@@ -3300,11 +3300,14 @@ TEST_P(DBCompactionTestWithParam, CompressLevelCompaction) {
GenerateNewFile(&rnd, &key_idx);
ASSERT_EQ("1,4,8", FilesPerLevel(0));

ASSERT_EQ(matches, 12);
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cbi42 the check for IsTrivialMove inside my GetNumberCompactionSortedRuns method messes up the count for InputCompressionMatchesOutput. I did not dig into why the original test values were the way they are, but I do not think my PR would break this functionality. For those who do understand, I updated the expected value to be the original expected value plus what I determined was added by GetNumberCompactionSortedRuns.

Not sure if you had to work with this test case before. Let me know if this is problematic or the comments can be reworded

@cbi42
Copy link
Member

cbi42 commented Jan 22, 2025

Would it be easier to maintain this count inside each compaction job / subcompaction? Similar to how DBImpl::bg_compaction_scheduled_ is passed to CompactionJob and updated by it.

@anand1976
Copy link
Contributor

@anand1976 do you think this subcompaction exception will be problematic for our purposes? I searched around and found a NUM_SUBCOMPACTIONS_SCHEDULED histogram that we could use.

My guess is it would be problematic. To be on the safe side, like @cbi42 suggested we could maintain this count inside each compaction job/subcompaction. Or updating it in MakeInputIterator might be even easier.

@archang19 archang19 changed the title Track the total number of compaction sorted runs Track the total number of compaction sorted runs inside Jan 22, 2025
@archang19 archang19 changed the title Track the total number of compaction sorted runs inside Track the total number of compaction sorted runs inside BackgroundCallCompaction Jan 22, 2025
@archang19
Copy link
Contributor Author

thanks for the comments @cbi42 @anand1976. I am following your suggestion and am working on another PR #13325 that updates the stats inside CompactionMergingIterator. I hope to get it out for review soon

@archang19
Copy link
Contributor Author

closing in favor of #13325

@archang19 archang19 closed this Feb 6, 2025
facebook-github-bot pushed a commit that referenced this pull request Feb 6, 2025
…onMergingIterator (#13325)

Summary:
**This PR adds a new statistic to track the total number of sorted runs for running compactions.**

Context: I am currently working on a separate project, where I am trying to tune the read request sizes made by `FilePrefetchBuffer` to the storage backend. In this particular case, `FilePrefetchBuffer` will issue larger reads and have to buffer larger read responses. This means we expect to see higher memory utilization. At least for the initial rollout, we only want to enable this optimization for compaction reads.

**I want some way to get a sense of what the memory usage _impact_ will be if the prefetch read request size is increased from (for instance) 8MB to 64MB.**

**If I know the number of files that compactions are actively reading from (i.e. the number of sorted runs / "input iterators"), I can determine how much the memory usage will increase if I bump up the readahead size inside `FilePrefetchBuffer`.** For instance, if there are 16 sorted runs at any given point in time and I bump up the readahead size by 64MB, I can project an increase of 16 * 64 MB.

In most cases, the number of sorted runs processed per compaction is the number of L0 files plus the number of non-L0 levels. However, we need to be aware of exceptions like trivial compactions, deletion compactions, and subcompactions. This is a major reason why this PR chooses to implement the stats counting inside `CompactionMergingIterator`, since by the time we get down to that part of the stack, we know the "true" values for the number of input iterators / sorted runs.

Alternatives considered:
- #13299 gives you a histogram for the number of sorted runs ("input iterators") for a _single compaction_. While this statistic is interested and in the direction of what we want, we are going to be assessing the memory impact across _all_ compactions that are currently running. Thus, this statistic does not give us all the information we need.
- #13302 gives you the total prefetch buffer memory usage, but it doesn't tell you what happens when the readahead size is increased. Furthermore, the code change is error prone and very "invasive" -- look at how many places in the code had to be updated. This would be useful in the future for general memory accounting purposes, but it does not serve our immediate needs.
- #13320 aimed to track the same metric, but did this inside `DbImpl:: BackgroundCallCompaction`. It turns out that this does not handle the case where a compaction is divided into multiple subcompactions (in which case, there would be _more_ sorted runs being processed at the same time than you would otherwise predict.) The current PR handles subcompactions automatically, and I think it is cleaner overall.

Note: When I attempted to put this statistic as part of the `cf_stats_value_` array, even after updating the array to use `std::atomic<uint64_t>`, I still was able to get assertions to _fail_ inside the crash tests. These assertions checked that the unsigned integer would not underflow below zero during compaction. I experimented for many hours but could not figure out a solution, even though it would seem like things "should" work with `fetch_add` and `fetch_sub`. One possibility is that the values in `cf_stats_value_` are being cleared to 0, but I added a `fprintf` to that portion of the code and didn't see it getting printed out before my assertions failed. Regardless, I think that this statistic is different enough from the CF-specific and the other DB-wide stats that the best solution is to just have it defined as a separate `std::atomic<uint64_t>`. I also do not want to spend more hours trying to debug why the crash test assertions break, when the solution in the current version of the PR can get the assertions to consistently pass.

Pull Request resolved: #13325

Test Plan:
- I updated one unit test to confirm that `num_running_compaction_sorted_runs` starts and ends at 0. This checks that all the additions and subtractions cancel out. I also made sure the statistic got incremented at least once.
- When I added `fprintf` manually, I confirmed that my statistics updating code was being exercised numerous times inside `db_compaction_test`. I printed out the results before and after the increments/decrements, and the numbers looked good.
- We will monitor the generated statistics after this PR is merged.
- There are assertion checks after each increment and before each decrement. If there are bugs, the crash test will almost certainly find them, since they quickly found issues with my initial implementation for this PR which tried using the `cf_stats_value_` array (modified to use `std::atomic`).

Reviewed By: anand1976, hx235

Differential Revision: D68527895

Pulled By: archang19

fbshipit-source-id: 135cf210e0ff1550ea28ae4384d429ae620b1784
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants