-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
storage: rate-limit AddSST requests #36403
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.
Nice! This is even fewer lines of code than I thought it would be.
Reviewed 3 of 4 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt, @lucy-zhang, and @nvanbenschoten)
pkg/roachpb/batch.go, line 150 at r1 (raw file):
} // IsAddSSTRequest returns true iff the BatchRequest contains an AddSST request.
It looks like the convention would be to move this down below IsSingleRequest
, call this IsSingleAddSSTRequest
, and define it like IsSinglePushTxnRequest
is defined.
pkg/storage/store.go, line 2759 at r1 (raw file):
} // Limit the number of concurrent AddSST requests, since they're expensive
Do you think we should consoladate this kind of logic for Import
requests, Export
requests, and AddSSTable
requests into a single Store.maybeLimitRequest
method? I don't think the current behavior of throttling imports and exports during evaluation is what they want to be doing.
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 and @nvanbenschoten)
pkg/roachpb/batch.go, line 150 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
It looks like the convention would be to move this down below
IsSingleRequest
, call thisIsSingleAddSSTRequest
, and define it likeIsSinglePushTxnRequest
is defined.
Done.
pkg/storage/store.go, line 2759 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Do you think we should consoladate this kind of logic for
Import
requests,Export
requests, andAddSSTable
requests into a singleStore.maybeLimitRequest
method? I don't think the current behavior of throttling imports and exports during evaluation is what they want to be doing.
Yeah, I think it makes sense to move the rate limiting for import/export here eventually. They would still have to have separate Limiter
s, but that could be passed in as an argument to maybeLimitRequest
or something.
Relatedly, @dt originally suggested adding a flag isExpensive
for AddSST requests, for requests that should be rate-limited in Send()
. Ultimately I wasn't sure whether this really works if we have multiple types of expensive requests, each of which is rate-limited independently, so I went with an ad-hoc approach which hopefully shouldn't stop us from refactoring later. I could add a TODO to reconsider how we deal with rate limiting here (and I heard that there are bigger plans related to this for 19.2).
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.
We might decide that default is too low but i’d be happy backporting a default change later if we conclude that is the case.
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.
100% in favor of this. In fact, I could have sworn we already had it, but indeed we don't.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt, @lucy-zhang, and @nvanbenschoten)
pkg/storage/store.go, line 129 at r2 (raw file):
// addSSTRequestLimit limits concurrent AddSST requests. var addSSTRequestLimit = settings.RegisterPositiveIntSetting( "kv.bulk_io_write.concurrent_addsst_requests",
I'd prefer we skip this abbreviation. It's nice that "addsst" is the same length as "import" and "export" and so everything lines up, but the improved discoverability for someone doing a grep -i addsstable
is worth it.
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.
I also added a release note for the new cluster setting (now called kv.bulk_io_write.concurrent_addsstable_requests
).
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt and @nvanbenschoten)
pkg/storage/store.go, line 129 at r2 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
I'd prefer we skip this abbreviation. It's nice that "addsst" is the same length as "import" and "export" and so everything lines up, but the improved discoverability for someone doing a
grep -i addsstable
is worth it.
Done.
@nvanbenschoten this is ready for one last look. |
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.
I see a number of tests in store_test.go that test various throttlers. @nvanbenschoten do you have thoughts on which test can be used as an example for this change?
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt and @nvanbenschoten)
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.
I don't think there are any existing tests in store_test.go for import/export/rangefeed rate-limiting, which is what this change most closely resembles.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt and @nvanbenschoten)
the other concurrent request limiters ( |
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.
I'm not aware of any tests that would be appropriate to extend to this. We should add some, but that can go in a separate PR (perhaps when we centralize the logic across imports, exports, and addsstables).
Reviewed 1 of 2 files at r2, 3 of 3 files at r3.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @lucy-zhang)
pkg/storage/batcheval/eval_context.go, line 40 at r3 (raw file):
ConcurrentImports limit.ConcurrentRequestLimiter ConcurrentExports limit.ConcurrentRequestLimiter ConcurrentAddSSTableRequests limit.ConcurrentRequestLimiter
nit: make this consistent. Either ConcurentXYZRequests
or ConcurrentXYZs
everywhere.
We've been seeing extremely high latency for foreground traffic during bulk index backfills, because AddSST requests into non-empty ranges can be expensive, and write requests that are queued behind an AddSST request for an overlapping span can get stuck waiting for multiple seconds. This PR limits the number of concurrent AddSST requests for a single store, determined by a new cluster setting, `kv.bulk_io_write.concurrent_addsstable_requests`, to decrease the impact of index backfills on foreground writes. (It also decreases the risk of writing too many L0 files to RocksDB at once, which causes stalls.) Release note (general change): Add a new cluster setting, `kv.bulk_io_write.concurrent_addsstable_requests`, which limits the number of SSTables that can be added concurrently during bulk operations.
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 @nvanbenschoten)
pkg/storage/batcheval/eval_context.go, line 40 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: make this consistent. Either
ConcurentXYZRequests
orConcurrentXYZs
everywhere.
Renamed to ConcurrentImportRequests
and ConcurrentExportRequests
.
bors r+ |
Build failed (retrying...) |
36403: storage: rate-limit AddSST requests r=lucy-zhang a=lucy-zhang We've been seeing extremely high latency for foreground traffic during bulk index backfills, because AddSST requests into non-empty ranges can be expensive, and write requests that are queued behind an AddSST request for an overlapping span can get stuck waiting for multiple seconds. This PR limits the number of concurrent AddSST requests for a single store, determined by a new cluster setting, `kv.bulk_io_write.concurrent_addsstable_requests`, to decrease the impact of index backfills on foreground writes. (It also decreases the risk of writing too many L0 files to RocksDB at once, which causes stalls.) Fixes #36430 Release note (general change): Add a new cluster setting, `kv.bulk_io_write.concurrent_addsstable_requests`, which limits the number of SSTables that can be added concurrently during bulk operations. 36436: roachtest: handle duplicates in cdc/schemareg r=nvanbenschoten a=danhhz There are various internal races and retries in changefeeds that can produce duplicates. This test is really only to verify that the confluent schema registry works end-to-end, so do the simplest thing and sort + unique the output. Closes #36409 Release note: None Co-authored-by: Lucy Zhang <lucy-zhang@users.noreply.github.com> Co-authored-by: Daniel Harrison <daniel.harrison@gmail.com>
Build succeeded |
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 <irfanmahmoudsharif@gmail.com>
We've been seeing extremely high latency for foreground traffic during bulk
index backfills, because AddSST requests into non-empty ranges can be
expensive, and write requests that are queued behind an AddSST request for an
overlapping span can get stuck waiting for multiple seconds. This PR limits the
number of concurrent AddSST requests for a single store, determined by a new
cluster setting,
kv.bulk_io_write.concurrent_addsstable_requests
, to decreasethe impact of index backfills on foreground writes. (It also decreases the risk
of writing too many L0 files to RocksDB at once, which causes stalls.)
Fixes #36430
Release note (general change): Add a new cluster setting,
kv.bulk_io_write.concurrent_addsstable_requests
, which limits the number ofSSTables that can be added concurrently during bulk operations.