-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
kvserver: throttle AddSSTable
requests with IngestAsWrites
#73904
kvserver: throttle AddSSTable
requests with IngestAsWrites
#73904
Conversation
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.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt, @erikgrinaker, and @nvanbenschoten)
pkg/kv/kvserver/store_send.go, line 290 at r1 (raw file):
beforeEngineDelay := timeutil.Now() s.engine.PreIngestDelay(ctx)
isn't the other behavior change (not explicitly mentioned in the PR description) that IngestAsWrites
are now also subject to PreIngestDelay
?
Is there a justification for the delay to be the same as a larger sstable?
This looks like a good idea to me.
My understanding is that the investigation of tpce index builds that are currently able to get to high read amp suggests that is happening due to these as-writes batches currently skipping the PreIngestBackpressure, and then overwhelming the LSM. Subjecting them to the same back pressure seems like a good idea, right? like, if the store is healthy, there's no back pressure anyway, and they go right though. If the store is not healthy, then backpressuring "ssts", even if they'll be switched to memtable writes, seems like a good idea? |
That was my thinking as well. This may actually be the more important part of this change. |
237fa44
to
3b20ab2
Compare
`Store.Send()` limits the number of concurrent `AddSSTable` requests and delays them depending on LSM health via `Engine.PreIngestDelay`, to prevent overwhelming Pebble. However, requests with `IngestAsWrites` were not throttled, which has been seen to cause significant read amplification. This patch subjects `IngestAsWrites` requests to `Engine.PreIngestDelay` as well, and adds a separate limit for `IngestAsWrites` requests controlled via the cluster setting `kv.bulk_io_write.concurrent_addsstable_as_writes_requests` (default 10). Since these requests are generally small, and will end up in the Pebble memtable before being flushed to disk, we can tolerate a larger limit for these requests than regular `AddSSTable` requests (1). Release note (performance improvement): Bulk ingestion of small write batches (e.g. index backfill into a large number of ranges) is now throttled, to avoid buildup of read amplification and associated performance degradation. Concurrency is controlled by the new cluster setting `kv.bulk_io_write.concurrent_addsstable_as_writes_requests`.
3b20ab2
to
3f125d1
Compare
AddSSTable
requests with IngestAsWrites
AddSSTable
requests with IngestAsWrites
You're right, that wasn't communicated properly here, and as others point out is likely to be the more significant improvement.
This limits throughput to about 4 MB/s, concurrently with 16 MB/s of SST ingestion. Since the write batches likely have more overhead, and may be executed concurrently with SSTs, I think it makes sense to use the same delay for them. |
Huh, just curious, how do we arrive at those mb/s numbers? |
16 MB/s for SST ingestion is from Of course, I see now that the /s bit doesn't make sense (getting late here). But that's the concurrent volume we're ingesting, for lack of a better term. |
I've run some quick tests with this patch, and the results are encouraging. First off, I'll note that it's still possible to get read amp blowup. This can trivially be done by setting However, using more reasonable values such as Finally, I tried This was not a comprehensive or systematic test, but I think sufficient to demonstrate that this throttling can prevent read amp blowup without significantly affecting throughput with default settings. @nvanbenschoten Did you find the same in your trials? As for |
Nice to see these results! In addition to doing something like this, that is backportable, we should create an issue to do both (a) make this more robust, (b) remove all these hard to fiddle with cluster settings. It is much easier for an operator to have a mental model for a setting like admission.l0_sub_level_count_overload_threshold, and much harder to reason about kv.bulk_io_write.concurrent_addsstable* and rocksdb.ingest_backpressure.max_delay. |
Yeah, I agree -- wrote up #73979. |
My last few trials were inconclusive. The intention was to show that with this change, we throttle sufficiently to not only stave off read amplification growth, but also enough to recover from it. The hope was that this would provide additional evidence of the stabilizing effect of this change. So my trials were running with this PR + a cluster setting to allow me to dynamically opt-in to any throttling on |
This is very surprising to me, as I have been able to consistently produce read amp growth up to 1000 -- even on the same cluster you have been testing with. Reach out to me on Slack if you'd like to give this another shot together.
I saw this in my tests, where I let read amp build up by setting large values for Do you feel like we need additional testing or tuning for this PR? |
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.
Do you feel like we need additional testing or tuning for this PR?
I'm satisfied with what I've seen on Slack and in the test clusters.
Reviewed 2 of 3 files at r1, 1 of 1 files at r2, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker)
TFTRs! bors r=dt,nvanbenschoten |
Build succeeded: |
Encountered an error creating backports. Some common things that can go wrong:
You might need to create your backport manually using the backport tool. error creating merge commit from 3f125d1 to blathers/backport-release-21.1-73904: POST https://api.github.com/repos/cockroachdb/cockroach/merges: 409 Merge conflict [] you may need to manually resolve merge conflicts with the backport tool. Backport to branch 21.1.x failed. See errors above. 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is otan. |
Fixes cockroachdb#102683. Part of cockroachdb#104154. These were added way back in cockroachdb#36403 and cockroachdb#73904, pre-dating much of IO admission control for leaseholder writes. With cockroachdb#95563, we now have IO admission control for follower writes. Put together, have ample LSM read-amp protection through AC alone. These concurrency limiters are now redundant and oblivious to more sophisticated AC measures. We recently removed the below-raft equivalents of these limiters (cockroachdb#98762), and like mentioned there, these limiters can exacerbate memory pressure. Separately, we're looking to work on speedier restores, and these limiters are starting to get in the way. While here, we also disable the pre-ingest delay mechanism in pebble, which too pre-dates AC, introduced way back in cockroachdb#34258 for RocksDB and in \cockroachdb#41839 for Pebble. IO AC is able to limit the number of L0 files, and this pre-ingest delay with its maximum per-request delay time of 5s can be less than effective. It's worth noting that the L0 file count threshold at which this pre-ingest delay mechanism kicked in was 20, while AC aims for 1000[^1]. This commit doesn't go as far as removing these limiters outright, merely disabling them. This is just out of an overabundance of caution. We can probably remove them once kvflowcontrol.enabled has had >1 release worth of baking time. Until then, it's nice to know we have these old safety hatches. We have ample time in the release to assess fallout from this commit, and also use this increased AddSST concurrency to stress the kvflowcontrol machinery. [^1]: The 1000 file limit exists to bound how long it takes to clear L0 completely. Envelope math cribbed from elsewhere: With 2MiB files, 1000 files is ~2GB, which at 40MB/s of compaction throughput (with a compaction slot consistently dedicated to L0) takes < 60s to clear the backlog. So the 'recovery' time is modest in that operators should not need to take manual action Release note: None
Fixes cockroachdb#102683. Part of cockroachdb#104154. These were added way back in cockroachdb#36403 and cockroachdb#73904, pre-dating much of IO admission control for leaseholder writes. With cockroachdb#95563, we now have IO admission control for follower writes. Put together, have ample LSM read-amp protection through AC alone. These concurrency limiters are now redundant and oblivious to more sophisticated AC measures. We recently removed the below-raft equivalents of these limiters (cockroachdb#98762), and like mentioned there, these limiters can exacerbate memory pressure. Separately, we're looking to work on speedier restores, and these limiters are starting to get in the way. While here, we also disable the pre-ingest delay mechanism in pebble, which too pre-dates AC, introduced way back in cockroachdb#34258 for RocksDB and in \cockroachdb#41839 for Pebble. IO AC is able to limit the number of L0 files, and this pre-ingest delay with its maximum per-request delay time of 5s can be less than effective. It's worth noting that the L0 file count threshold at which this pre-ingest delay mechanism kicked in was 20, while AC aims for 1000[^1]. This commit doesn't go as far as removing these limiters outright, merely disabling them. This is just out of an overabundance of caution. We can probably remove them once kvflowcontrol.enabled has had >1 release worth of baking time. Until then, it's nice to know we have these old safety hatches. We have ample time in the release to assess fallout from this commit, and also use this increased AddSST concurrency to stress the kvflowcontrol machinery. [^1]: The 1000 file limit exists to bound how long it takes to clear L0 completely. Envelope math cribbed from elsewhere: With 2MiB files, 1000 files is ~2GB, which at 40MB/s of compaction throughput (with a compaction slot consistently dedicated to L0) takes < 60s to clear the backlog. So the 'recovery' time is modest in that operators should not need to take manual action Release note: None
104861: kvserver: disable pre-AC above-raft AddSST throttling r=irfansharif a=irfansharif Fixes #102683. Part of #104154. These were added way back in #36403 and #73904, pre-dating much of IO admission control for leaseholder writes. With #95563, we now have IO admission control for follower writes. Put together, have ample LSM read-amp protection through AC alone. These concurrency limiters are now redundant and oblivious to more sophisticated AC measures. We recently removed the below-raft equivalents of these limiters (#98762), and like mentioned there, these limiters can exacerbate memory pressure. Separately, we're looking to work on speedier restores, and these limiters are starting to get in the way. While here, we also disable the pre-ingest delay mechanism in pebble, which too pre-dates AC, introduced way back in #34258 for RocksDB and in \#41839 for Pebble. IO AC is able to limit the number of L0 files, and this pre-ingest delay with its maximum per-request delay time of 5s can be less than effective. It's worth noting that the L0 file count threshold at which this pre-ingest delay mechanism kicked in was 20, while AC aims for 1000[^1]. This commit doesn't go as far as removing these limiters outright, merely disabling them. This is just out of an overabundance of caution. We can probably remove them once kvflowcontrol.enabled has had >1 release worth of baking time. Until then, it's nice to know we have these old safety hatches. We have ample time in the release to assess fallout from this commit, and also use this increased AddSST concurrency to stress the kvflowcontrol machinery. [^1]: The 1000 file limit exists to bound how long it takes to clear L0 completely. Envelope math cribbed from elsewhere: With 2MiB files, 1000 files is ~2GB, which at 40MB/s of compaction throughput (with a compaction slot consistently dedicated to L0) takes < 60s to clear the backlog. So the 'recovery' time is modest in that operators should not need to take manual action. Release note: None Co-authored-by: irfan sharif <[email protected]>
Store.Send()
limits the number of concurrentAddSSTable
requestsand delays them depending on LSM health via
Engine.PreIngestDelay
, toprevent overwhelming Pebble. However, requests with
IngestAsWrites
were not throttled, which has been seen to cause significant read
amplification.
This patch subjects
IngestAsWrites
requests toEngine.PreIngestDelay
as well, and adds a separate limit for
IngestAsWrites
requestscontrolled via the cluster setting
kv.bulk_io_write.concurrent_addsstable_as_writes_requests
(default10). Since these requests are generally small, and will end up in the
Pebble memtable before being flushed to disk, we can tolerate a larger
limit for these requests than regular
AddSSTable
requests (1).Release note (performance improvement): Bulk ingestion of small write
batches (e.g. index backfill into a large number of ranges) is now
throttled, to avoid buildup of read amplification and associated
performance degradation. Concurrency is controlled by the new cluster
setting
kv.bulk_io_write.concurrent_addsstable_as_writes_requests
.