Skip to content

Commit

Permalink
kvserver: disable pre-AC above-raft AddSST throttling
Browse files Browse the repository at this point in the history
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
  • Loading branch information
irfansharif committed Jun 14, 2023
1 parent 13addf5 commit 1352d02
Show file tree
Hide file tree
Showing 3 changed files with 31 additions and 8 deletions.
14 changes: 7 additions & 7 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,16 +158,16 @@ 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.
var addSSTableRequestLimit = settings.RegisterIntSetting(
settings.SystemOnly,
"kv.bulk_io_write.concurrent_addsstable_requests",
"number of concurrent AddSSTable requests per store before queueing",
1,
settings.PositiveInt,
"number of concurrent AddSSTable requests per store before queueing (0 = disable)",
0,
settings.NonNegativeInt,
)

// addSSTableAsWritesRequestLimit limits concurrent AddSSTable requests with
Expand All @@ -178,9 +178,9 @@ var addSSTableRequestLimit = settings.RegisterIntSetting(
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,
settings.PositiveInt,
"number of concurrent AddSSTable requests ingested as writes per store before queueing (0 = disable)",
0,
settings.NonNegativeInt,
)

// concurrentRangefeedItersLimit limits concurrent rangefeed catchup iterators.
Expand Down
10 changes: 10 additions & 0 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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)

Expand Down
15 changes: 14 additions & 1 deletion pkg/util/limit/limiter.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
type ConcurrentRequestLimiter struct {
spanName string
sem *quotapool.IntPool
disabled bool
}

// Reservation is an allocation from a limiter which should be released once the
Expand All @@ -48,6 +49,9 @@ func (l *ConcurrentRequestLimiter) Begin(ctx context.Context) (Reservation, erro
if err := ctx.Err(); err != nil {
return nil, errors.Wrap(err, "limiter begin")
}
if l.disabled {
return noopReservation{}, nil // nothing to do
}

res, err := l.sem.TryAcquire(ctx, 1)
if errors.Is(err, quotapool.ErrNotEnoughQuota) {
Expand All @@ -60,7 +64,16 @@ func (l *ConcurrentRequestLimiter) Begin(ctx context.Context) (Reservation, erro
return res, err
}

// SetLimit adjusts the size of the pool.
// SetLimit adjusts the size of the pool. If limit == 0, the concurrency limiter
// is disabled.
func (l *ConcurrentRequestLimiter) SetLimit(newLimit int) {
l.disabled = newLimit == 0
l.sem.UpdateCapacity(uint64(newLimit))
}

type noopReservation struct{}

var _ Reservation = noopReservation{}

// Release implements the Reservation interface.
func (n noopReservation) Release() {}

0 comments on commit 1352d02

Please sign in to comment.