Skip to content

Commit 3f125d1

Browse files
committed
kvserver: throttle AddSSTable requests with IngestAsWrites
`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`.
1 parent ae17f3d commit 3f125d1

3 files changed

Lines changed: 29 additions & 11 deletions

File tree

pkg/kv/kvserver/batcheval/eval_context.go

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -32,9 +32,10 @@ import (
3232

3333
// Limiters is the collection of per-store limits used during cmd evaluation.
3434
type Limiters struct {
35-
BulkIOWriteRate *rate.Limiter
36-
ConcurrentExportRequests limit.ConcurrentRequestLimiter
37-
ConcurrentAddSSTableRequests limit.ConcurrentRequestLimiter
35+
BulkIOWriteRate *rate.Limiter
36+
ConcurrentExportRequests limit.ConcurrentRequestLimiter
37+
ConcurrentAddSSTableRequests limit.ConcurrentRequestLimiter
38+
ConcurrentAddSSTableAsWritesRequests limit.ConcurrentRequestLimiter
3839
// concurrentRangefeedIters is a semaphore used to limit the number of
3940
// rangefeeds in the "catch-up" state across the store. The "catch-up" state
4041
// is a temporary state at the beginning of a rangefeed which is expensive

pkg/kv/kvserver/store.go

Lines changed: 22 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -128,11 +128,23 @@ var bulkIOWriteLimit = settings.RegisterByteSizeSetting(
128128
// addSSTableRequestLimit limits concurrent AddSSTable requests.
129129
var addSSTableRequestLimit = settings.RegisterIntSetting(
130130
"kv.bulk_io_write.concurrent_addsstable_requests",
131-
"number of AddSSTable requests a store will handle concurrently before queuing",
131+
"number of concurrent AddSSTable requests per store before queueing",
132132
1,
133133
settings.PositiveInt,
134134
)
135135

136+
// addSSTableAsWritesRequestLimit limits concurrent AddSSTable requests with
137+
// IngestAsWrites set. These are smaller (kv.bulk_io_write.small_write_size),
138+
// and will end up in the Pebble memtable (default 64 MB) before flushing to
139+
// disk, so we can allow a greater amount of concurrency than regular AddSSTable
140+
// requests. Applied independently of concurrent_addsstable_requests.
141+
var addSSTableAsWritesRequestLimit = settings.RegisterIntSetting(
142+
"kv.bulk_io_write.concurrent_addsstable_as_writes_requests",
143+
"number of concurrent AddSSTable requests ingested as writes per store before queueing",
144+
10,
145+
settings.PositiveInt,
146+
)
147+
136148
// concurrentRangefeedItersLimit limits concurrent rangefeed catchup iterators.
137149
var concurrentRangefeedItersLimit = settings.RegisterIntSetting(
138150
"kv.rangefeed.concurrent_catchup_iterators",
@@ -1222,7 +1234,15 @@ func NewStore(
12221234
"addSSTableRequestLimiter", int(addSSTableRequestLimit.Get(&cfg.Settings.SV)),
12231235
)
12241236
addSSTableRequestLimit.SetOnChange(&cfg.Settings.SV, func(ctx context.Context) {
1225-
s.limiters.ConcurrentAddSSTableRequests.SetLimit(int(addSSTableRequestLimit.Get(&cfg.Settings.SV)))
1237+
s.limiters.ConcurrentAddSSTableRequests.SetLimit(
1238+
int(addSSTableRequestLimit.Get(&cfg.Settings.SV)))
1239+
})
1240+
s.limiters.ConcurrentAddSSTableAsWritesRequests = limit.MakeConcurrentRequestLimiter(
1241+
"addSSTableAsWritesRequestLimiter", int(addSSTableAsWritesRequestLimit.Get(&cfg.Settings.SV)),
1242+
)
1243+
addSSTableAsWritesRequestLimit.SetOnChange(&cfg.Settings.SV, func(ctx context.Context) {
1244+
s.limiters.ConcurrentAddSSTableAsWritesRequests.SetLimit(
1245+
int(addSSTableAsWritesRequestLimit.Get(&cfg.Settings.SV)))
12261246
})
12271247
s.limiters.ConcurrentRangefeedIters = limit.MakeConcurrentRequestLimiter(
12281248
"rangefeedIterLimiter", int(concurrentRangefeedItersLimit.Get(&cfg.Settings.SV)),

pkg/kv/kvserver/store_send.go

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -276,15 +276,12 @@ func (s *Store) maybeThrottleBatch(
276276

277277
switch t := ba.Requests[0].GetInner().(type) {
278278
case *roachpb.AddSSTableRequest:
279-
// Limit the number of concurrent AddSSTable requests, since they're
280-
// expensive and block all other writes to the same span. However, don't
281-
// limit AddSSTable requests that are going to ingest as a WriteBatch.
279+
limiter := s.limiters.ConcurrentAddSSTableRequests
282280
if t.IngestAsWrites {
283-
return nil, nil
281+
limiter = s.limiters.ConcurrentAddSSTableAsWritesRequests
284282
}
285-
286283
before := timeutil.Now()
287-
res, err := s.limiters.ConcurrentAddSSTableRequests.Begin(ctx)
284+
res, err := limiter.Begin(ctx)
288285
if err != nil {
289286
return nil, err
290287
}

0 commit comments

Comments
 (0)