From 5da084511bed29b35c5bd31f89d703fdfb1e108c Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 14 Jun 2023 03:22:55 -0400 Subject: [PATCH] kvserver: disable pre-AC above-raft AddSST throttling 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 --- pkg/kv/kvserver/store.go | 6 +++--- pkg/storage/engine.go | 10 ++++++++++ 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 6c25a43159e9..84d6f2a3b6e5 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -158,7 +158,7 @@ var bulkIOWriteLimit = settings.RegisterByteSizeSetting( settings.SystemOnly, "kv.bulk_io_write.max_rate", "the rate limit (bytes/sec) to use for writes to disk on behalf of bulk io ops", - 1<<40, + 1<<40, // 1 TiB ).WithPublic() // addSSTableRequestLimit limits concurrent AddSSTable requests. @@ -166,7 +166,7 @@ var addSSTableRequestLimit = settings.RegisterIntSetting( settings.SystemOnly, "kv.bulk_io_write.concurrent_addsstable_requests", "number of concurrent AddSSTable requests per store before queueing", - 1, + math.MaxInt, // unlimited settings.PositiveInt, ) @@ -179,7 +179,7 @@ var addSSTableAsWritesRequestLimit = settings.RegisterIntSetting( settings.SystemOnly, "kv.bulk_io_write.concurrent_addsstable_as_writes_requests", "number of concurrent AddSSTable requests ingested as writes per store before queueing", - 10, + math.MaxInt, // unlimited settings.PositiveInt, ) diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 0dc531e4014d..faf6d244786f 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -1554,6 +1554,13 @@ var ingestDelayTime = settings.RegisterDurationSetting( time.Second*5, ) +var preIngestDelayEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "pebble.pre_ingest_delay.enabled", + "controls whether the pre-ingest delay mechanism is active", + false, +) + // PreIngestDelay may choose to block for some duration if L0 has an excessive // number of files in it or if PendingCompactionBytesEstimate is elevated. This // it is intended to be called before ingesting a new SST, since we'd rather @@ -1567,6 +1574,9 @@ func preIngestDelay(ctx context.Context, eng Engine, settings *cluster.Settings) if settings == nil { return } + if !preIngestDelayEnabled.Get(&settings.SV) { + return + } metrics := eng.GetMetrics() targetDelay := calculatePreIngestDelay(settings, metrics.Metrics)