From 4e5d2412253c12731a7c131ab0b49ae24459e3ec Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 12 Aug 2022 12:12:14 -0400 Subject: [PATCH 1/9] admission: move ioLoadListener into its own file Pure code movement change. Trying to see what components in this package could be reasoned about independently to readers less familiar with this package. Some of these may benefit from being pulled into sub-packages, but for now a separate file is sufficient. Release note: None --- pkg/util/admission/BUILD.bazel | 1 + pkg/util/admission/doc.go | 2 +- pkg/util/admission/granter.go | 826 ------------------ pkg/util/admission/io_load_listener.go | 844 +++++++++++++++++++ pkg/util/admission/store_token_estimation.go | 7 + 5 files changed, 853 insertions(+), 827 deletions(-) create mode 100644 pkg/util/admission/io_load_listener.go diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index 11e528f3b374..2fdc12af5cc0 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -7,6 +7,7 @@ go_library( "disk_bandwidth.go", "doc.go", "granter.go", + "io_load_listener.go", "store_token_estimation.go", "work_queue.go", ], diff --git a/pkg/util/admission/doc.go b/pkg/util/admission/doc.go index 62cdfdd55fdf..bda99a40d83f 100644 --- a/pkg/util/admission/doc.go +++ b/pkg/util/admission/doc.go @@ -56,7 +56,7 @@ // provide a general framework, and specific implementations that are // initially quite simple in their heuristics but may become more // sophisticated over time. The concrete abstractions: -// - Tokens and slots are the two ways admission is granted (see grantKind) +// - Tokens and slots are the two ways admission is granted (see grantKind). // - Categorization of kinds of work (see WorkKind), and a priority ordering // across WorkKinds that is used to reflect their shared need for underlying // resources. diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 89c1369af0b7..83bfc7bb1d3a 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -12,7 +12,6 @@ package admission import ( "context" - "math" "time" "unsafe" @@ -21,13 +20,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" - "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" - "github.com/cockroachdb/logtags" "github.com/cockroachdb/pebble" "github.com/cockroachdb/redact" ) @@ -42,22 +39,6 @@ var KVSlotAdjusterOverloadThreshold = settings.RegisterIntSetting( "slot adjuster considers the cpu to be overloaded", 32, settings.PositiveInt) -// L0FileCountOverloadThreshold sets a file count threshold that signals an -// overloaded store. -var L0FileCountOverloadThreshold = settings.RegisterIntSetting( - settings.TenantWritable, - "admission.l0_file_count_overload_threshold", - "when the L0 file count exceeds this theshold, the store is considered overloaded", - l0FileCountOverloadThreshold, settings.PositiveInt) - -// L0SubLevelCountOverloadThreshold sets a sub-level count threshold that -// signals an overloaded store. -var L0SubLevelCountOverloadThreshold = settings.RegisterIntSetting( - settings.TenantWritable, - "admission.l0_sub_level_count_overload_threshold", - "when the L0 sub-level count exceeds this threshold, the store is considered overloaded", - l0SubLevelCountOverloadThreshold, settings.PositiveInt) - // EnabledSoftSlotGranting can be set to false to disable soft slot granting. var EnabledSoftSlotGranting = settings.RegisterBoolSetting( settings.TenantWritable, @@ -66,45 +47,6 @@ var EnabledSoftSlotGranting = settings.RegisterBoolSetting( true, ) -// MinFlushUtilizationFraction is a lower-bound on the dynamically adjusted -// flush utilization target fraction that attempts to reduce write stalls. Set -// it to a high fraction (>>1, e.g. 10), to effectively disable flush based -// tokens. -// -// The target fraction is used to multiply the (measured) peak flush rate, to -// compute the flush tokens. For example, if the dynamic target fraction (for -// which this setting provides a lower bound) is currently 0.75, then -// 0.75*peak-flush-rate will be used to set the flush tokens. The lower bound -// of 0.5 should not need to be tuned, and should not be tuned without -// consultation with a domain expert. If the storage.write-stall-nanos -// indicates significant write stalls, and the granter logs show that the -// dynamic target fraction has already reached the lower bound, one can -// consider lowering it slightly and then observe the effect. -var MinFlushUtilizationFraction = settings.RegisterFloatSetting( - settings.SystemOnly, - "admission.min_flush_util_fraction", - "when computing flush tokens, this fraction is a lower bound on the dynamically "+ - "adjusted flush utilization target fraction that attempts to reduce write stalls. Set "+ - "it to a high fraction (>>1, e.g. 10), to disable flush based tokens. The dynamic "+ - "target fraction is used to multiply the (measured) peak flush rate, to compute the flush "+ - "tokens. If the storage.write-stall-nanos indicates significant write stalls, and the granter "+ - "logs show that the dynamic target fraction has already reached the lower bound, one can "+ - "consider lowering it slightly (after consultation with domain experts)", 0.5, - settings.PositiveFloat) - -// DiskBandwidthTokensForElasticEnabled controls whether the disk bandwidth -// resource is considered as a possible bottleneck resource. When it becomes a -// bottleneck, tokens for elastic work are limited based on available disk -// bandwidth. The default is true since actually considering disk bandwidth as -// a bottleneck resource requires additional configuration (outside the -// admission package) to calculate the provisioned bandwidth. -var DiskBandwidthTokensForElasticEnabled = settings.RegisterBoolSetting( - settings.SystemOnly, - "admission.disk_bandwidth_tokens.elastic.enabled", - "when true, and provisioned bandwidth for the disk corresponding to a store is configured, "+ - "tokens for elastic work will be limited if disk bandwidth becomes a bottleneck", - true).WithPublic() - // grantChainID is the ID for a grant chain. See continueGrantChain for // details. type grantChainID uint64 @@ -1771,25 +1713,6 @@ func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider( }() } -// Experimental observations: -// - Sub-level count of ~40 caused a node heartbeat latency p90, p99 of 2.5s, -// 4s. With a setting that limits sub-level count to 10, before the system -// is considered overloaded, and adjustmentInterval = 60, we see the actual -// sub-level count ranging from 5-30, with p90, p99 node heartbeat latency -// showing a similar wide range, with 1s, 2s being the middle of the range -// respectively. -// - With tpcc, we sometimes see a sub-level count > 10 with only 100 files in -// L0. We don't want to restrict tokens in this case since the store is able -// to recover on its own. One possibility would be to require both the -// thresholds to be exceeded before we consider the store overloaded. But -// then we run the risk of having 100+ sub-levels when we hit a file count -// of 1000. Instead we use a sub-level overload threshold of 20. -// -// We've set these overload thresholds in a way that allows the system to -// absorb short durations (say a few minutes) of heavy write load. -const l0FileCountOverloadThreshold = 1000 -const l0SubLevelCountOverloadThreshold = 20 - func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoordinator { coord := &GrantCoordinator{ settings: sgc.settings, @@ -2227,755 +2150,6 @@ type storeRequester interface { setStoreRequestEstimates(estimates storeRequestEstimates) } -type ioLoadListenerState struct { - // Cumulative. - cumL0AddedBytes uint64 - // Gauge. - curL0Bytes int64 - // Cumulative. - cumWriteStallCount int64 - diskBW struct { - // Cumulative - bytesRead uint64 - bytesWritten uint64 - incomingLSMBytes uint64 - } - - // Exponentially smoothed per interval values. - - smoothedIntL0CompactedBytes int64 // bytes leaving L0 - // Smoothed history of byte tokens calculated based on compactions out of L0. - smoothedCompactionByteTokens float64 - - // Smoothed history of flush tokens calculated based on memtable flushes, - // before multiplying by target fraction. - smoothedNumFlushTokens float64 - // The target fraction to be used for the effective flush tokens. It is in - // the interval - // [MinFlushUtilizationFraction,maxFlushUtilTargetFraction]. - flushUtilTargetFraction float64 - - // totalNumByteTokens represents the tokens to give out until the next call to - // adjustTokens. They are parceled out in small intervals. byteTokensAllocated - // represents what has been given out. - totalNumByteTokens int64 - byteTokensAllocated int64 - // Used tokens can be negative if some tokens taken in one interval were - // returned in another, but that will be extremely rare. - byteTokensUsed int64 - - // elasticDiskBWTokens represents the tokens to give out until the next call - // to adjustTokens. They are parceled out in small intervals. - // elasticDiskTokensAllocated represents what has been given out. - elasticDiskBWTokens int64 - elasticDiskBWTokensAllocated int64 -} - -// ioLoadListener adjusts tokens in kvStoreTokenGranter for IO, specifically due to -// overload caused by writes. IO uses tokens and not slots since work -// completion is not an indicator that the "resource usage" has ceased -- it -// just means that the write has been applied to the WAL. Most of the work is -// in flushing to sstables and the following compactions, which happens later. -// -// -// Token units are in bytes and used to protect a number of virtual or -// physical resource bottlenecks: -// - Compactions out of L0: compactions out of L0 can fall behind and cause -// too many sub-levels or files in L0. -// - Flushes into L0: flushes of memtables to L0 can fall behind and cause -// write stalls due to too many memtables. -// - Disk bandwidth: there is typically an aggregate read+write provisioned -// bandwidth, and if it is fully utilized, IO operations can start queueing -// and encounter high latency. -// -// For simplicity, after ioLoadListener computes the tokens due to compaction -// or flush bottlenecks, it computes the minimum and passes that value to -// granterWithIOTokens.setAvailableIOTokensLocked. That is, instead of working -// with multiple token dimensions, these two token dimensions get collapsed -// into one for enforcement purposes. This also helps simplify the integration -// with WorkQueue which is dealing with a single dimension. The consumption of -// these tokens is based on how many bytes an admitted work adds to L0. -// -// The disk bandwidth constraint is used to compute a token count for elastic -// work (see disk_bandwidth.go for the reasons why this is limited to elastic -// work). Additionally, these tokens are meant be consumed for all incoming -// bytes into the LSM, and not just those written to L0 e.g. ingested bytes -// into L3 should also consume such tokens. Note that we call these disk -// bandwidth tokens, but that is a misnomer -- these are tokens for incoming -// bytes into the LSM, motivated by disk bandwidth as a bottleneck resource, -// and not consumed for every write to the disk (e.g. by compactions). Since -// these tokens are consumed by all incoming bytes into the LSM, and not just -// those into L0, it suggests explicitly modeling this as a separate -// dimension. However, since modeling as a separate dimension everywhere would -// lead to code complexity, we adopt the following compromise: -// -// - Like the other token dimensions, ioLoadListener computes a different -// elastic token count (using diskBandwidthLimiter), and a different model -// for consumption (via -// storePerWorkTokenEstimator.atDoneDiskBWTokensLinearModel). -// -// - granterWithIOTokens, implemented by kvStoreTokenGranter, which enforces -// the token count, also treats this as a separate dimension. -// -// - WorkQueue works with a single dimension, so the tokens consumed at -// admission time are based on L0-bytes estimate. However, when -// StoreWorkQueue informs kvStoreTokenGranter of work completion (by calling -// storeWriteDone), the tokens are adjusted differently for the -// flush/compaction L0 tokens and for the "disk bandwidth" tokens. -type ioLoadListener struct { - storeID int32 - settings *cluster.Settings - kvRequester storeRequester - mu struct { - // Used when changing state in kvGranter. This is a pointer since it is - // the same as GrantCoordinator.mu. - *syncutil.Mutex - kvGranter granterWithIOTokens - } - - // Stats used to compute interval stats. - statsInitialized bool - adjustTokensResult - perWorkTokenEstimator storePerWorkTokenEstimator - diskBandwidthLimiter diskBandwidthLimiter -} - -const unlimitedTokens = math.MaxInt64 - -// Token changes are made at a coarse time granularity of 15s since -// compactions can take ~10s to complete. The totalNumByteTokens to give out over -// the 15s interval are given out in a smoothed manner, at 250ms intervals. -// This has similarities with the following kinds of token buckets: -// - Zero replenishment rate and a burst value that is changed every 15s. We -// explicitly don't want a huge burst every 15s. -// - A replenishment rate equal to totalNumByteTokens/60, with a burst capped at -// totalNumByteTokens/60. The only difference with the code here is that if -// totalNumByteTokens is small, the integer rounding effects are compensated for. -// -// In an experiment with extreme overload using KV0 with block size 64KB, -// and 4096 clients, we observed the following states of L0 at 1min -// intervals (r-amp is the L0 sub-level count), in the absence of any -// admission control: -// -// __level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp› -// 0 96 158 M 2.09 315 M 0 B 0 0 B 0 305 M 178 0 B 3 1.0› -// 0 1026 1.7 G 3.15 4.7 G 0 B 0 0 B 0 4.7 G 2.8 K 0 B 24 1.0› -// 0 1865 3.0 G 2.86 9.1 G 0 B 0 0 B 0 9.1 G 5.5 K 0 B 38 1.0› -// 0 3225 4.9 G 3.46 13 G 0 B 0 0 B 0 13 G 8.3 K 0 B 59 1.0› -// 0 4720 7.0 G 3.46 17 G 0 B 0 0 B 0 17 G 11 K 0 B 85 1.0› -// 0 6120 9.0 G 4.13 21 G 0 B 0 0 B 0 21 G 14 K 0 B 109 1.0› -// -// -// Note the fast growth in sub-level count. Production issues typically have -// slower growth towards an unhealthy state (remember that similar stats in -// the logs of a regular CockroachDB node are at 10min intervals, and not at -// 1min). -// -// In the above experiment, L0 compaction durations at 200+ sub-levels were -// usually sane, with most L0 compactions < 10s, and with a bandwidth of -// ~80MB/s. There were some 1-2GB compactions that took ~20s. The -// expectation is that with the throttling done by admission control here, -// we should almost never see multi-minute compactions. Which makes it -// reasonable to simply use metrics that are updated when compactions -// complete (as opposed to also tracking progress in bytes of on-going -// compactions). -// -// The 250ms interval to hand out the computed tokens is due to the needs of -// flush tokens. For compaction tokens, a 1s interval is fine-grained enough. -// -// If flushing a memtable take 100ms, then 10 memtables can be sustainably -// flushed in 1s. If we dole out flush tokens in 1s intervals, then there are -// enough tokens to create 10 memtables at the very start of a 1s interval, -// which will cause a write stall. Intuitively, the faster it is to flush a -// memtable, the smaller the interval for doling out these tokens. We have -// observed flushes taking ~0.5s, so we picked a 250ms interval for doling out -// these tokens. We could use a value smaller than 250ms, but we've observed -// CPU utilization issues at lower intervals (see the discussion in -// runnable.go). -const adjustmentInterval = 15 -const ticksInAdjustmentInterval = 60 -const ioTokenTickDuration = 250 * time.Millisecond - -func cumLSMWriteAndIngestedBytes( - m *pebble.Metrics, -) (writeAndIngestedBytes uint64, ingestedBytes uint64) { - for i := range m.Levels { - writeAndIngestedBytes += m.Levels[i].BytesIngested + m.Levels[i].BytesFlushed - ingestedBytes += m.Levels[i].BytesIngested - } - return writeAndIngestedBytes, ingestedBytes -} - -// pebbleMetricsTicks is called every adjustmentInterval seconds, and decides -// the token allocations until the next call. -func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMetrics) { - ctx = logtags.AddTag(ctx, "s", io.storeID) - m := metrics.Metrics - if !io.statsInitialized { - io.statsInitialized = true - sas := io.kvRequester.getStoreAdmissionStats() - cumLSMIncomingBytes, cumLSMIngestedBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics) - io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIngestedBytes, sas) - io.adjustTokensResult = adjustTokensResult{ - ioLoadListenerState: ioLoadListenerState{ - cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, - curL0Bytes: m.Levels[0].Size, - cumWriteStallCount: metrics.WriteStallCount, - // No initial limit, i.e, the first interval is unlimited. - totalNumByteTokens: unlimitedTokens, - elasticDiskBWTokens: unlimitedTokens, - }, - aux: adjustTokensAuxComputations{}, - ioThreshold: &admissionpb.IOThreshold{ - L0NumSubLevels: int64(m.Levels[0].Sublevels), - L0NumSubLevelsThreshold: math.MaxInt64, - L0NumFiles: m.Levels[0].NumFiles, - L0NumFilesThreshold: math.MaxInt64, - }, - } - io.diskBW.bytesRead = metrics.DiskStats.BytesRead - io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten - io.diskBW.incomingLSMBytes = cumLSMIncomingBytes - io.copyAuxEtcFromPerWorkEstimator() - return - } - io.adjustTokens(ctx, metrics) -} - -// allocateTokensTick gives out 1/ticksInAdjustmentInterval of the -// various tokens every 250ms. -func (io *ioLoadListener) allocateTokensTick() { - allocateFunc := func(total int64, allocated int64) (toAllocate int64) { - // unlimitedTokens==MaxInt64, so avoid overflow in the rounding up - // calculation. - if total >= unlimitedTokens-(ticksInAdjustmentInterval-1) { - toAllocate = total / ticksInAdjustmentInterval - } else { - // Round up so that we don't accumulate tokens to give in a burst on the - // last tick. - toAllocate = (total + ticksInAdjustmentInterval - 1) / ticksInAdjustmentInterval - if toAllocate < 0 { - panic(errors.AssertionFailedf("toAllocate is negative %d", toAllocate)) - } - if toAllocate+allocated > total { - toAllocate = total - allocated - } - } - return toAllocate - } - // INVARIANT: toAllocate* >= 0. - toAllocateByteTokens := allocateFunc(io.totalNumByteTokens, io.byteTokensAllocated) - if toAllocateByteTokens < 0 { - panic(errors.AssertionFailedf("toAllocateByteTokens is negative %d", toAllocateByteTokens)) - } - toAllocateElasticDiskBWTokens := - allocateFunc(io.elasticDiskBWTokens, io.elasticDiskBWTokensAllocated) - if toAllocateElasticDiskBWTokens < 0 { - panic(errors.AssertionFailedf("toAllocateElasticDiskBWTokens is negative %d", - toAllocateElasticDiskBWTokens)) - } - // INVARIANT: toAllocate >= 0. - io.mu.Lock() - defer io.mu.Unlock() - io.byteTokensAllocated += toAllocateByteTokens - if io.byteTokensAllocated < 0 { - panic(errors.AssertionFailedf("tokens allocated is negative %d", io.byteTokensAllocated)) - } - io.byteTokensUsed += io.mu.kvGranter.setAvailableIOTokensLocked(toAllocateByteTokens) - io.elasticDiskBWTokensAllocated += toAllocateElasticDiskBWTokens - io.mu.kvGranter.setAvailableElasticDiskBandwidthTokensLocked(toAllocateElasticDiskBWTokens) -} - -func computeIntervalDiskLoadInfo( - prevCumBytesRead uint64, prevCumBytesWritten uint64, diskStats DiskStats, -) intervalDiskLoadInfo { - return intervalDiskLoadInfo{ - readBandwidth: int64((diskStats.BytesRead - prevCumBytesRead) / adjustmentInterval), - writeBandwidth: int64((diskStats.BytesWritten - prevCumBytesWritten) / adjustmentInterval), - provisionedBandwidth: diskStats.ProvisionedBandwidth, - } -} - -// adjustTokens computes a new value of totalNumByteTokens (and resets -// tokensAllocated). The new value, when overloaded, is based on comparing how -// many bytes are being moved out of L0 via compactions with the average -// number of bytes being added to L0 per KV work. We want the former to be -// (significantly) larger so that L0 returns to a healthy state. The byte -// token computation also takes into account the flush throughput, since an -// inability to flush fast enough can result in write stalls due to high -// memtable counts, which we want to avoid as it can cause latency hiccups of -// 100+ms for all write traffic. -func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics) { - sas := io.kvRequester.getStoreAdmissionStats() - res := io.adjustTokensInner(ctx, io.ioLoadListenerState, - metrics.Levels[0], metrics.WriteStallCount, metrics.InternalIntervalMetrics, - L0FileCountOverloadThreshold.Get(&io.settings.SV), - L0SubLevelCountOverloadThreshold.Get(&io.settings.SV), - MinFlushUtilizationFraction.Get(&io.settings.SV), - ) - io.adjustTokensResult = res - cumLSMIncomingBytes, cumLSMIngestedBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics) - { - // Disk Bandwidth tokens. - io.aux.diskBW.intervalDiskLoadInfo = computeIntervalDiskLoadInfo( - io.diskBW.bytesRead, io.diskBW.bytesWritten, metrics.DiskStats) - io.mu.Lock() - diskTokensUsed := io.mu.kvGranter.getDiskTokensUsedAndResetLocked() - io.mu.Unlock() - io.aux.diskBW.intervalLSMInfo = intervalLSMInfo{ - incomingBytes: int64(cumLSMIncomingBytes) - int64(io.diskBW.incomingLSMBytes), - regularTokensUsed: diskTokensUsed[regularWorkClass], - elasticTokensUsed: diskTokensUsed[elasticWorkClass], - } - if metrics.DiskStats.ProvisionedBandwidth > 0 { - io.elasticDiskBWTokens = io.diskBandwidthLimiter.computeElasticTokens(ctx, - io.aux.diskBW.intervalDiskLoadInfo, io.aux.diskBW.intervalLSMInfo) - io.elasticDiskBWTokensAllocated = 0 - } - if metrics.DiskStats.ProvisionedBandwidth == 0 || - !DiskBandwidthTokensForElasticEnabled.Get(&io.settings.SV) { - io.elasticDiskBWTokens = unlimitedTokens - } - io.diskBW.bytesRead = metrics.DiskStats.BytesRead - io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten - io.diskBW.incomingLSMBytes = cumLSMIncomingBytes - } - io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIngestedBytes, sas) - io.copyAuxEtcFromPerWorkEstimator() - requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission() - io.kvRequester.setStoreRequestEstimates(requestEstimates) - l0WriteLM, l0IngestLM, ingestLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone() - io.mu.Lock() - io.mu.kvGranter.setAdmittedDoneModelsLocked(l0WriteLM, l0IngestLM, ingestLM) - io.mu.Unlock() - if _, overloaded := io.ioThreshold.Score(); overloaded || io.aux.doLogFlush || - io.elasticDiskBWTokens != unlimitedTokens { - log.Infof(ctx, "IO overload: %s", io.adjustTokensResult) - } -} - -// copyAuxEtcFromPerWorkEstimator copies the auxiliary and other numerical -// state from io.perWorkTokenEstimator. This is helpful in keeping all the -// numerical state for understanding the behavior of ioLoadListener and its -// helpers in one place for simplicity of logging. -func (io *ioLoadListener) copyAuxEtcFromPerWorkEstimator() { - // Copy the aux so that the printing story is simplified. - io.adjustTokensResult.aux.perWorkTokensAux = io.perWorkTokenEstimator.aux - requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission() - io.adjustTokensResult.requestEstimates = requestEstimates - l0WriteLM, l0IngestLM, ingestLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone() - io.adjustTokensResult.l0WriteLM = l0WriteLM - io.adjustTokensResult.l0IngestLM = l0IngestLM - io.adjustTokensResult.ingestLM = ingestLM -} - -type tokenKind int8 - -const ( - compactionTokenKind tokenKind = iota - flushTokenKind -) - -// adjustTokensAuxComputations encapsulates auxiliary numerical state for -// ioLoadListener that is helpful for understanding its behavior. -type adjustTokensAuxComputations struct { - intL0AddedBytes int64 - intL0CompactedBytes int64 - - intFlushTokens float64 - intFlushUtilization float64 - intWriteStalls int64 - - prevTokensUsed int64 - tokenKind tokenKind - - perWorkTokensAux perWorkTokensAux - doLogFlush bool - - diskBW struct { - intervalDiskLoadInfo intervalDiskLoadInfo - intervalLSMInfo intervalLSMInfo - } -} - -// adjustTokensInner is used for computing tokens based on compaction and -// flush bottlenecks. -func (*ioLoadListener) adjustTokensInner( - ctx context.Context, - prev ioLoadListenerState, - l0Metrics pebble.LevelMetrics, - cumWriteStallCount int64, - im *pebble.InternalIntervalMetrics, - threshNumFiles, threshNumSublevels int64, - minFlushUtilTargetFraction float64, -) adjustTokensResult { - ioThreshold := &admissionpb.IOThreshold{ - L0NumFiles: l0Metrics.NumFiles, - L0NumFilesThreshold: threshNumFiles, - L0NumSubLevels: int64(l0Metrics.Sublevels), - L0NumSubLevelsThreshold: threshNumSublevels, - } - - curL0Bytes := l0Metrics.Size - cumL0AddedBytes := l0Metrics.BytesFlushed + l0Metrics.BytesIngested - // L0 growth over the last interval. - intL0AddedBytes := int64(cumL0AddedBytes) - int64(prev.cumL0AddedBytes) - if intL0AddedBytes < 0 { - // intL0AddedBytes is a simple delta computation over individually cumulative - // stats, so should not be negative. - log.Warningf(ctx, "intL0AddedBytes %d is negative", intL0AddedBytes) - intL0AddedBytes = 0 - } - // intL0CompactedBytes are due to finished compactions. - intL0CompactedBytes := prev.curL0Bytes + intL0AddedBytes - curL0Bytes - if intL0CompactedBytes < 0 { - // Ignore potential inconsistencies across cumulative stats and current L0 - // bytes (gauge). - intL0CompactedBytes = 0 - } - const alpha = 0.5 - // Compaction scheduling can be uneven in prioritizing L0 for compactions, - // so smooth out what is being removed by compactions. - smoothedIntL0CompactedBytes := int64(alpha*float64(intL0CompactedBytes) + (1-alpha)*float64(prev.smoothedIntL0CompactedBytes)) - - // Flush tokens: - // - // Write stalls happen when flushing of memtables is a bottleneck. - // - // Computing Flush Tokens: - // Flush can go from not being the bottleneck in one 15s interval - // (adjustmentInterval) to being the bottleneck in the next 15s interval - // (e.g. when L0 falls below the unhealthy threshold and compaction tokens - // become unlimited). So the flush token limit has to react quickly (cannot - // afford to wait for multiple 15s intervals). We've observed that if we - // normalize the flush rate based on flush loop utilization (the PeakRate - // computation below), and use that to compute flush tokens, the token - // counts are quite stable. Here are two examples, showing this steady token - // count computed using PeakRate of the flush ThroughputMetric, despite - // changes in flush loop utilization (the util number below). - // - // Example 1: Case where IO bandwidth was not a bottleneck - // flush: tokens: 2312382401, util: 0.90 - // flush: tokens: 2345477107, util: 0.31 - // flush: tokens: 2317829891, util: 0.29 - // flush: tokens: 2428387843, util: 0.17 - // - // Example 2: Case where IO bandwidth became a bottleneck (and mean fsync - // latency was fluctuating between 1ms and 4ms in the low util to high util - // cases). - // - // flush: tokens: 1406132615, util: 1.00 - // flush: tokens: 1356476227, util: 0.64 - // flush: tokens: 1374880806, util: 0.24 - // flush: tokens: 1328578534, util: 0.96 - // - // Hence, using PeakRate as a basis for computing flush tokens seems sound. - // The other important question is what fraction of PeakRate avoids write - // stalls. It is likely less than 100% since while a flush is ongoing, - // memtables can accumulate and cause a stall. For example, we have observed - // write stalls at 80% of PeakRate. The fraction depends on configuration - // parameters like MemTableStopWritesThreshold (defaults to 4 in - // CockroachDB), and environmental and workload factors like how long a - // flush takes to flush a single 64MB memtable. Instead of trying to measure - // and adjust for these, we use a simple multiplier, - // flushUtilTargetFraction. By default, flushUtilTargetFraction ranges - // between 0.5 and 1.5. The lower bound is configurable via - // admission.min_flush_util_percent and if configured above the upper bound, - // the upper bound will be ignored and the target fraction will not be - // dynamically adjusted. The dynamic adjustment logic uses an additive step - // size of flushUtilTargetFractionIncrement (0.025), with the following - // logic: - // - Reduce the fraction if there is a write-stall. The reduction may use a - // small multiple of flushUtilTargetFractionIncrement. This is so that - // this probing spends more time below the threshold where write stalls - // occur. - // - Increase fraction if no write-stall and flush tokens were almost all - // used. - // - // This probing unfortunately cannot eliminate write stalls altogether. - // Future improvements could use more history to settle on a good - // flushUtilTargetFraction for longer, or use some measure of how close we - // are to a write-stall to stop the increase. - // - // Ingestion and flush tokens: - // - // Ingested sstables do not utilize any flush capacity. Consider 2 cases: - // - sstable ingested into L0: there was either data overlap with L0, or - // file boundary overlap with L0-L6. To be conservative, lets assume there - // was data overlap, and that this data overlap extended into the memtable - // at the time of ingestion. Memtable(s) would have been force flushed to - // handle such overlap. The cost of flushing a memtable is based on how - // much of the allocated memtable capacity is used, so an early flush - // seems harmless. However, write stalls are based on allocated memtable - // capacity, so there is a potential negative interaction of these forced - // flushes since they cause additional memtable capacity allocation. - // - sstable ingested into L1-L6: there was no data overlap with L0, which - // implies that there was no reason to flush memtables. - // - // Since there is some interaction noted in bullet 1, and because it - // simplifies the admission control token behavior, we use flush tokens in - // an identical manner as compaction tokens -- to be consumed by all data - // flowing into L0. Some of this conservative choice will be compensated for - // by flushUtilTargetFraction (when the mix of ingestion and actual flushes - // are stable). Another thing to note is that compactions out of L0 are - // typically the more persistent bottleneck than flushes for the following - // reason: - // There is a dedicated flush thread. With a maximum compaction concurrency - // of C, we have up to C threads dedicated to handling the write-amp of W - // (caused by rewriting the same data). So C/(W-1) threads on average are - // reading the original data (that will be rewritten W-1 times). Since L0 - // can have multiple overlapping files, and intra-L0 compactions are usually - // avoided, we can assume (at best) that the original data (in L0) is being - // read only when compacting to levels lower than L0. That is, C/(W-1) - // threads are reading from L0 to compact to levels lower than L0. Since W - // can be 20+ and C defaults to 3 (we plan to dynamically adjust C but one - // can expect C to be <= 10), C/(W-1) < 1. So the main reason we are - // considering flush tokens is transient flush bottlenecks, and workloads - // where W is small. - - // Compute flush utilization for this interval. A very low flush utilization - // will cause flush tokens to be unlimited. - intFlushUtilization := float64(0) - if im.Flush.WriteThroughput.WorkDuration > 0 { - intFlushUtilization = float64(im.Flush.WriteThroughput.WorkDuration) / - float64(im.Flush.WriteThroughput.WorkDuration+im.Flush.WriteThroughput.IdleDuration) - } - // Compute flush tokens for this interval that would cause 100% utilization. - intFlushTokens := float64(im.Flush.WriteThroughput.PeakRate()) * adjustmentInterval - intWriteStalls := cumWriteStallCount - prev.cumWriteStallCount - - // Ensure flushUtilTargetFraction is in the configured bounds. This also - // does lazy initialization. - const maxFlushUtilTargetFraction = 1.5 - flushUtilTargetFraction := prev.flushUtilTargetFraction - if flushUtilTargetFraction == 0 { - // Initialization: use the maximum configured fraction. - flushUtilTargetFraction = minFlushUtilTargetFraction - if flushUtilTargetFraction < maxFlushUtilTargetFraction { - flushUtilTargetFraction = maxFlushUtilTargetFraction - } - } else if flushUtilTargetFraction < minFlushUtilTargetFraction { - // The min can be changed in a running system, so we bump up to conform to - // the min. - flushUtilTargetFraction = minFlushUtilTargetFraction - } - numFlushTokens := int64(unlimitedTokens) - // doLogFlush becomes true if something interesting is done here. - doLogFlush := false - smoothedNumFlushTokens := prev.smoothedNumFlushTokens - const flushUtilIgnoreThreshold = 0.05 - if intFlushUtilization > flushUtilIgnoreThreshold { - if smoothedNumFlushTokens == 0 { - // Initialization. - smoothedNumFlushTokens = intFlushTokens - } else { - smoothedNumFlushTokens = alpha*intFlushTokens + (1-alpha)*prev.smoothedNumFlushTokens - } - const flushUtilTargetFractionIncrement = 0.025 - // Have we used, over the last (15s) cycle, more than 90% of the tokens we - // would give out for the next cycle? If yes, highTokenUsage is true. - highTokenUsage := - float64(prev.byteTokensUsed) >= 0.9*smoothedNumFlushTokens*flushUtilTargetFraction - if intWriteStalls > 0 { - // Try decrease since there were write-stalls. - numDecreaseSteps := 1 - // These constants of 5, 3, 2, 2 were found to work reasonably well, - // without causing large decreases. We need better benchmarking to tune - // such constants. - if intWriteStalls >= 5 { - numDecreaseSteps = 3 - } else if intWriteStalls >= 2 { - numDecreaseSteps = 2 - } - for i := 0; i < numDecreaseSteps; i++ { - if flushUtilTargetFraction >= minFlushUtilTargetFraction+flushUtilTargetFractionIncrement { - flushUtilTargetFraction -= flushUtilTargetFractionIncrement - doLogFlush = true - } else { - break - } - } - } else if flushUtilTargetFraction < maxFlushUtilTargetFraction-flushUtilTargetFractionIncrement && - intWriteStalls == 0 && highTokenUsage { - // No write-stalls, and token usage was high, so give out more tokens. - flushUtilTargetFraction += flushUtilTargetFractionIncrement - doLogFlush = true - } - if highTokenUsage { - doLogFlush = true - } - flushTokensFloat := flushUtilTargetFraction * smoothedNumFlushTokens - if flushTokensFloat < float64(math.MaxInt64) { - numFlushTokens = int64(flushTokensFloat) - } - // Else avoid overflow by using the previously set unlimitedTokens. This - // should not really happen. - } - // Else intFlushUtilization is too low. We don't want to make token - // determination based on a very low utilization, so we hand out unlimited - // tokens. Note that flush utilization has been observed to fluctuate from - // 0.16 to 0.9 in a single interval, when compaction tokens are not limited, - // hence we have set flushUtilIgnoreThreshold to a very low value. If we've - // erred towards it being too low, we run the risk of computing incorrect - // tokens. If we've erred towards being too high, we run the risk of giving - // out unlimitedTokens and causing write stalls. - - // We constrain admission based on compactions, if the store is over the L0 - // threshold. - var totalNumByteTokens int64 - var smoothedCompactionByteTokens float64 - - _, overloaded := ioThreshold.Score() - if overloaded { - // Don't admit more byte work than we can remove via compactions. totalNumByteTokens - // tracks our goal for admission. - // Scale down since we want to get under the thresholds over time. This - // scaling could be adjusted based on how much above the threshold we are, - // but for now we just use a constant. - fTotalNumByteTokens := float64(smoothedIntL0CompactedBytes / 2.0) - // Smooth it. This may seem peculiar since we are already using - // smoothedIntL0CompactedBytes, but the else clause below uses a different - // computation so we also want the history of smoothedTotalNumByteTokens. - smoothedCompactionByteTokens = alpha*fTotalNumByteTokens + (1-alpha)*prev.smoothedCompactionByteTokens - if float64(math.MaxInt64) < smoothedCompactionByteTokens { - // Avoid overflow. This should not really happen. - totalNumByteTokens = math.MaxInt64 - } else { - totalNumByteTokens = int64(smoothedCompactionByteTokens) - } - } else { - // Under the threshold. Maintain a smoothedTotalNumByteTokens based on what was - // removed, so that when we go over the threshold we have some history. - // This is also useful when we temporarily dip below the threshold -- - // we've seen extreme situations with alternating 15s intervals of above - // and below the threshold. - numTokens := intL0CompactedBytes - smoothedCompactionByteTokens = alpha*float64(numTokens) + (1-alpha)*prev.smoothedCompactionByteTokens - totalNumByteTokens = unlimitedTokens - } - // Use the minimum of the token count calculated using compactions and - // flushes. - tokenKind := compactionTokenKind - if totalNumByteTokens > numFlushTokens { - totalNumByteTokens = numFlushTokens - tokenKind = flushTokenKind - } - // Install the latest cumulative stats. - return adjustTokensResult{ - ioLoadListenerState: ioLoadListenerState{ - cumL0AddedBytes: cumL0AddedBytes, - curL0Bytes: curL0Bytes, - cumWriteStallCount: cumWriteStallCount, - smoothedIntL0CompactedBytes: smoothedIntL0CompactedBytes, - smoothedCompactionByteTokens: smoothedCompactionByteTokens, - smoothedNumFlushTokens: smoothedNumFlushTokens, - flushUtilTargetFraction: flushUtilTargetFraction, - totalNumByteTokens: totalNumByteTokens, - byteTokensAllocated: 0, - byteTokensUsed: 0, - }, - aux: adjustTokensAuxComputations{ - intL0AddedBytes: intL0AddedBytes, - intL0CompactedBytes: intL0CompactedBytes, - intFlushTokens: intFlushTokens, - intFlushUtilization: intFlushUtilization, - intWriteStalls: intWriteStalls, - prevTokensUsed: prev.byteTokensUsed, - tokenKind: tokenKind, - doLogFlush: doLogFlush, - }, - ioThreshold: ioThreshold, - } -} - -// adjustTokensResult encapsulates all the numerical state of ioLoadListener. -type adjustTokensResult struct { - ioLoadListenerState - requestEstimates storeRequestEstimates - l0WriteLM tokensLinearModel - l0IngestLM tokensLinearModel - ingestLM tokensLinearModel - aux adjustTokensAuxComputations - ioThreshold *admissionpb.IOThreshold // never nil -} - -func max(i, j int64) int64 { - if i < j { - return j - } - return i -} - -func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { - ib := humanizeutil.IBytes - // NB: "≈" indicates smoothed quantities. - p.Printf("compaction score %v (%d ssts, %d sub-levels), ", res.ioThreshold, res.ioThreshold.L0NumFiles, res.ioThreshold.L0NumSubLevels) - p.Printf("L0 growth %s (write %s ingest %s ignored %s): ", ib(res.aux.intL0AddedBytes), - ib(res.aux.perWorkTokensAux.intL0WriteBytes), ib(res.aux.perWorkTokensAux.intL0IngestedBytes), - ib(res.aux.perWorkTokensAux.intL0IgnoredIngestedBytes)) - // Writes to L0 that we expected because requests told admission control. - // This is the "easy path", from an estimation perspective, if all regular - // writes accurately tell us what they write, and all ingests tell us what - // they ingest and all of ingests into L0. - p.Printf("requests %d (%d bypassed) with ", res.aux.perWorkTokensAux.intWorkCount, - res.aux.perWorkTokensAux.intBypassedWorkCount) - p.Printf("%s acc-write (%s bypassed) + ", - ib(res.aux.perWorkTokensAux.intL0WriteAccountedBytes), - ib(res.aux.perWorkTokensAux.intL0WriteBypassedAccountedBytes)) - // Ingestion bytes that we expected because requests told admission control. - p.Printf("%s acc-ingest (%s bypassed) + ", - ib(res.aux.perWorkTokensAux.intIngestedAccountedBytes), - ib(res.aux.perWorkTokensAux.intIngestedBypassedAccountedBytes)) - // The models we are fitting to compute tokens based on the reported size of - // the write and ingest. - p.Printf("write-model %.2fx+%s (smoothed %.2fx+%s) + ", - res.aux.perWorkTokensAux.intL0WriteLinearModel.multiplier, - ib(res.aux.perWorkTokensAux.intL0WriteLinearModel.constant), - res.l0WriteLM.multiplier, ib(res.l0WriteLM.constant)) - p.Printf("ingested-model %.2fx+%s (smoothed %.2fx+%s) + ", - res.aux.perWorkTokensAux.intL0IngestedLinearModel.multiplier, - ib(res.aux.perWorkTokensAux.intL0IngestedLinearModel.constant), - res.l0IngestLM.multiplier, ib(res.l0IngestLM.constant)) - // The tokens used per request at admission time, when no size information - // is known. - p.Printf("at-admission-tokens %s, ", ib(res.requestEstimates.writeTokens)) - // How much got compacted out of L0 recently. - p.Printf("compacted %s [≈%s], ", ib(res.aux.intL0CompactedBytes), ib(res.smoothedIntL0CompactedBytes)) - // The tokens computed for flush, based on observed flush throughput and - // utilization. - p.Printf("flushed %s [≈%s]; ", ib(int64(res.aux.intFlushTokens)), - ib(int64(res.smoothedNumFlushTokens))) - p.Printf("admitting ") - if n := res.ioLoadListenerState.totalNumByteTokens; n < unlimitedTokens { - p.Printf("%s (rate %s/s)", ib(n), ib(n/adjustmentInterval)) - switch res.aux.tokenKind { - case compactionTokenKind: - p.Printf(" due to L0 growth") - case flushTokenKind: - p.Printf(" due to memtable flush (multiplier %.3f)", res.flushUtilTargetFraction) - } - p.Printf(" (used %s)", ib(res.aux.prevTokensUsed)) - } else { - p.SafeString("all") - } - if res.elasticDiskBWTokens != unlimitedTokens { - p.Printf("; elastic tokens %s (used %s, regular used %s): "+ - "write model %.2fx+%s ingest model %.2fx+%s, ", - ib(res.elasticDiskBWTokens), ib(res.aux.diskBW.intervalLSMInfo.elasticTokensUsed), - ib(res.aux.diskBW.intervalLSMInfo.regularTokensUsed), - res.l0WriteLM.multiplier, ib(res.l0WriteLM.constant), - res.ingestLM.multiplier, ib(res.ingestLM.constant)) - p.Printf("disk bw read %s write %s provisioned %s", - ib(res.aux.diskBW.intervalDiskLoadInfo.readBandwidth), - ib(res.aux.diskBW.intervalDiskLoadInfo.writeBandwidth), - ib(res.aux.diskBW.intervalDiskLoadInfo.provisionedBandwidth)) - } -} - -func (res adjustTokensResult) String() string { - return redact.StringWithoutMarkers(res) -} - var _ cpuOverloadIndicator = &sqlNodeCPUOverloadIndicator{} var _ CPULoadListener = &sqlNodeCPUOverloadIndicator{} diff --git a/pkg/util/admission/io_load_listener.go b/pkg/util/admission/io_load_listener.go new file mode 100644 index 000000000000..8a328b5b7a1e --- /dev/null +++ b/pkg/util/admission/io_load_listener.go @@ -0,0 +1,844 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package admission + +import ( + "context" + "math" + "time" + + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/logtags" + "github.com/cockroachdb/pebble" + "github.com/cockroachdb/redact" +) + +// MinFlushUtilizationFraction is a lower-bound on the dynamically adjusted +// flush utilization target fraction that attempts to reduce write stalls. Set +// it to a high fraction (>>1, e.g. 10), to effectively disable flush based +// tokens. +// +// The target fraction is used to multiply the (measured) peak flush rate, to +// compute the flush tokens. For example, if the dynamic target fraction (for +// which this setting provides a lower bound) is currently 0.75, then +// 0.75*peak-flush-rate will be used to set the flush tokens. The lower bound +// of 0.5 should not need to be tuned, and should not be tuned without +// consultation with a domain expert. If the storage.write-stall-nanos +// indicates significant write stalls, and the granter logs show that the +// dynamic target fraction has already reached the lower bound, one can +// consider lowering it slightly and then observe the effect. +var MinFlushUtilizationFraction = settings.RegisterFloatSetting( + settings.SystemOnly, + "admission.min_flush_util_fraction", + "when computing flush tokens, this fraction is a lower bound on the dynamically "+ + "adjusted flush utilization target fraction that attempts to reduce write stalls. Set "+ + "it to a high fraction (>>1, e.g. 10), to disable flush based tokens. The dynamic "+ + "target fraction is used to multiply the (measured) peak flush rate, to compute the flush "+ + "tokens. If the storage.write-stall-nanos indicates significant write stalls, and the granter "+ + "logs show that the dynamic target fraction has already reached the lower bound, one can "+ + "consider lowering it slightly (after consultation with domain experts)", 0.5, + settings.PositiveFloat) + +// DiskBandwidthTokensForElasticEnabled controls whether the disk bandwidth +// resource is considered as a possible bottleneck resource. When it becomes a +// bottleneck, tokens for elastic work are limited based on available disk +// bandwidth. The default is true since actually considering disk bandwidth as +// a bottleneck resource requires additional configuration (outside the +// admission package) to calculate the provisioned bandwidth. +var DiskBandwidthTokensForElasticEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "admission.disk_bandwidth_tokens.elastic.enabled", + "when true, and provisioned bandwidth for the disk corresponding to a store is configured, "+ + "tokens for elastic work will be limited if disk bandwidth becomes a bottleneck", + true).WithPublic() + +// L0FileCountOverloadThreshold sets a file count threshold that signals an +// overloaded store. +var L0FileCountOverloadThreshold = settings.RegisterIntSetting( + settings.TenantWritable, + "admission.l0_file_count_overload_threshold", + "when the L0 file count exceeds this theshold, the store is considered overloaded", + l0FileCountOverloadThreshold, settings.PositiveInt) + +// L0SubLevelCountOverloadThreshold sets a sub-level count threshold that +// signals an overloaded store. +var L0SubLevelCountOverloadThreshold = settings.RegisterIntSetting( + settings.TenantWritable, + "admission.l0_sub_level_count_overload_threshold", + "when the L0 sub-level count exceeds this threshold, the store is considered overloaded", + l0SubLevelCountOverloadThreshold, settings.PositiveInt) + +// Experimental observations: +// - Sub-level count of ~40 caused a node heartbeat latency p90, p99 of 2.5s, +// 4s. With a setting that limits sub-level count to 10, before the system +// is considered overloaded, and adjustmentInterval = 60, we see the actual +// sub-level count ranging from 5-30, with p90, p99 node heartbeat latency +// showing a similar wide range, with 1s, 2s being the middle of the range +// respectively. +// - With tpcc, we sometimes see a sub-level count > 10 with only 100 files in +// L0. We don't want to restrict tokens in this case since the store is able +// to recover on its own. One possibility would be to require both the +// thresholds to be exceeded before we consider the store overloaded. But +// then we run the risk of having 100+ sub-levels when we hit a file count +// of 1000. Instead we use a sub-level overload threshold of 20. +// +// We've set these overload thresholds in a way that allows the system to +// absorb short durations (say a few minutes) of heavy write load. +const l0FileCountOverloadThreshold = 1000 +const l0SubLevelCountOverloadThreshold = 20 + +// ioLoadListener adjusts tokens in kvStoreTokenGranter for IO, specifically due to +// overload caused by writes. IO uses tokens and not slots since work +// completion is not an indicator that the "resource usage" has ceased -- it +// just means that the write has been applied to the WAL. Most of the work is +// in flushing to sstables and the following compactions, which happens later. +// +// +// Token units are in bytes and used to protect a number of virtual or +// physical resource bottlenecks: +// - Compactions out of L0: compactions out of L0 can fall behind and cause +// too many sub-levels or files in L0. +// - Flushes into L0: flushes of memtables to L0 can fall behind and cause +// write stalls due to too many memtables. +// - Disk bandwidth: there is typically an aggregate read+write provisioned +// bandwidth, and if it is fully utilized, IO operations can start queueing +// and encounter high latency. +// +// For simplicity, after ioLoadListener computes the tokens due to compaction +// or flush bottlenecks, it computes the minimum and passes that value to +// granterWithIOTokens.setAvailableIOTokensLocked. That is, instead of working +// with multiple token dimensions, these two token dimensions get collapsed +// into one for enforcement purposes. This also helps simplify the integration +// with WorkQueue which is dealing with a single dimension. The consumption of +// these tokens is based on how many bytes an admitted work adds to L0. +// +// The disk bandwidth constraint is used to compute a token count for elastic +// work (see disk_bandwidth.go for the reasons why this is limited to elastic +// work). Additionally, these tokens are meant be consumed for all incoming +// bytes into the LSM, and not just those written to L0 e.g. ingested bytes +// into L3 should also consume such tokens. Note that we call these disk +// bandwidth tokens, but that is a misnomer -- these are tokens for incoming +// bytes into the LSM, motivated by disk bandwidth as a bottleneck resource, +// and not consumed for every write to the disk (e.g. by compactions). Since +// these tokens are consumed by all incoming bytes into the LSM, and not just +// those into L0, it suggests explicitly modeling this as a separate +// dimension. However, since modeling as a separate dimension everywhere would +// lead to code complexity, we adopt the following compromise: +// +// - Like the other token dimensions, ioLoadListener computes a different +// elastic token count (using diskBandwidthLimiter), and a different model +// for consumption (via +// storePerWorkTokenEstimator.atDoneDiskBWTokensLinearModel). +// +// - granterWithIOTokens, implemented by kvStoreTokenGranter, which enforces +// the token count, also treats this as a separate dimension. +// +// - WorkQueue works with a single dimension, so the tokens consumed at +// admission time are based on L0-bytes estimate. However, when +// StoreWorkQueue informs kvStoreTokenGranter of work completion (by calling +// storeWriteDone), the tokens are adjusted differently for the +// flush/compaction L0 tokens and for the "disk bandwidth" tokens. +type ioLoadListener struct { + storeID int32 + settings *cluster.Settings + kvRequester storeRequester + mu struct { + // Used when changing state in kvGranter. This is a pointer since it is + // the same as GrantCoordinator.mu. + *syncutil.Mutex + kvGranter granterWithIOTokens + } + + // Stats used to compute interval stats. + statsInitialized bool + adjustTokensResult + perWorkTokenEstimator storePerWorkTokenEstimator + diskBandwidthLimiter diskBandwidthLimiter +} + +type ioLoadListenerState struct { + // Cumulative. + cumL0AddedBytes uint64 + // Gauge. + curL0Bytes int64 + // Cumulative. + cumWriteStallCount int64 + diskBW struct { + // Cumulative + bytesRead uint64 + bytesWritten uint64 + incomingLSMBytes uint64 + } + + // Exponentially smoothed per interval values. + + smoothedIntL0CompactedBytes int64 // bytes leaving L0 + // Smoothed history of byte tokens calculated based on compactions out of L0. + smoothedCompactionByteTokens float64 + + // Smoothed history of flush tokens calculated based on memtable flushes, + // before multiplying by target fraction. + smoothedNumFlushTokens float64 + // The target fraction to be used for the effective flush tokens. It is in + // the interval + // [MinFlushUtilizationFraction,maxFlushUtilTargetFraction]. + flushUtilTargetFraction float64 + + // totalNumByteTokens represents the tokens to give out until the next call to + // adjustTokens. They are parceled out in small intervals. byteTokensAllocated + // represents what has been given out. + totalNumByteTokens int64 + byteTokensAllocated int64 + // Used tokens can be negative if some tokens taken in one interval were + // returned in another, but that will be extremely rare. + byteTokensUsed int64 + + // elasticDiskBWTokens represents the tokens to give out until the next call + // to adjustTokens. They are parceled out in small intervals. + // elasticDiskTokensAllocated represents what has been given out. + elasticDiskBWTokens int64 + elasticDiskBWTokensAllocated int64 +} + +const unlimitedTokens = math.MaxInt64 + +// Token changes are made at a coarse time granularity of 15s since +// compactions can take ~10s to complete. The totalNumByteTokens to give out over +// the 15s interval are given out in a smoothed manner, at 250ms intervals. +// This has similarities with the following kinds of token buckets: +// - Zero replenishment rate and a burst value that is changed every 15s. We +// explicitly don't want a huge burst every 15s. +// - A replenishment rate equal to totalNumByteTokens/60, with a burst capped at +// totalNumByteTokens/60. The only difference with the code here is that if +// totalNumByteTokens is small, the integer rounding effects are compensated for. +// +// In an experiment with extreme overload using KV0 with block size 64KB, +// and 4096 clients, we observed the following states of L0 at 1min +// intervals (r-amp is the L0 sub-level count), in the absence of any +// admission control: +// +// __level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp› +// 0 96 158 M 2.09 315 M 0 B 0 0 B 0 305 M 178 0 B 3 1.0› +// 0 1026 1.7 G 3.15 4.7 G 0 B 0 0 B 0 4.7 G 2.8 K 0 B 24 1.0› +// 0 1865 3.0 G 2.86 9.1 G 0 B 0 0 B 0 9.1 G 5.5 K 0 B 38 1.0› +// 0 3225 4.9 G 3.46 13 G 0 B 0 0 B 0 13 G 8.3 K 0 B 59 1.0› +// 0 4720 7.0 G 3.46 17 G 0 B 0 0 B 0 17 G 11 K 0 B 85 1.0› +// 0 6120 9.0 G 4.13 21 G 0 B 0 0 B 0 21 G 14 K 0 B 109 1.0› +// +// +// Note the fast growth in sub-level count. Production issues typically have +// slower growth towards an unhealthy state (remember that similar stats in +// the logs of a regular CockroachDB node are at 10min intervals, and not at +// 1min). +// +// In the above experiment, L0 compaction durations at 200+ sub-levels were +// usually sane, with most L0 compactions < 10s, and with a bandwidth of +// ~80MB/s. There were some 1-2GB compactions that took ~20s. The +// expectation is that with the throttling done by admission control here, +// we should almost never see multi-minute compactions. Which makes it +// reasonable to simply use metrics that are updated when compactions +// complete (as opposed to also tracking progress in bytes of on-going +// compactions). +// +// The 250ms interval to hand out the computed tokens is due to the needs of +// flush tokens. For compaction tokens, a 1s interval is fine-grained enough. +// +// If flushing a memtable take 100ms, then 10 memtables can be sustainably +// flushed in 1s. If we dole out flush tokens in 1s intervals, then there are +// enough tokens to create 10 memtables at the very start of a 1s interval, +// which will cause a write stall. Intuitively, the faster it is to flush a +// memtable, the smaller the interval for doling out these tokens. We have +// observed flushes taking ~0.5s, so we picked a 250ms interval for doling out +// these tokens. We could use a value smaller than 250ms, but we've observed +// CPU utilization issues at lower intervals (see the discussion in +// runnable.go). +const adjustmentInterval = 15 +const ticksInAdjustmentInterval = 60 +const ioTokenTickDuration = 250 * time.Millisecond + +func cumLSMWriteAndIngestedBytes( + m *pebble.Metrics, +) (writeAndIngestedBytes uint64, ingestedBytes uint64) { + for i := range m.Levels { + writeAndIngestedBytes += m.Levels[i].BytesIngested + m.Levels[i].BytesFlushed + ingestedBytes += m.Levels[i].BytesIngested + } + return writeAndIngestedBytes, ingestedBytes +} + +// pebbleMetricsTicks is called every adjustmentInterval seconds, and decides +// the token allocations until the next call. +func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMetrics) { + ctx = logtags.AddTag(ctx, "s", io.storeID) + m := metrics.Metrics + if !io.statsInitialized { + io.statsInitialized = true + sas := io.kvRequester.getStoreAdmissionStats() + cumLSMIncomingBytes, cumLSMIngestedBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics) + io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIngestedBytes, sas) + io.adjustTokensResult = adjustTokensResult{ + ioLoadListenerState: ioLoadListenerState{ + cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, + curL0Bytes: m.Levels[0].Size, + cumWriteStallCount: metrics.WriteStallCount, + // No initial limit, i.e, the first interval is unlimited. + totalNumByteTokens: unlimitedTokens, + elasticDiskBWTokens: unlimitedTokens, + }, + aux: adjustTokensAuxComputations{}, + ioThreshold: &admissionpb.IOThreshold{ + L0NumSubLevels: int64(m.Levels[0].Sublevels), + L0NumSubLevelsThreshold: math.MaxInt64, + L0NumFiles: m.Levels[0].NumFiles, + L0NumFilesThreshold: math.MaxInt64, + }, + } + io.diskBW.bytesRead = metrics.DiskStats.BytesRead + io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten + io.diskBW.incomingLSMBytes = cumLSMIncomingBytes + io.copyAuxEtcFromPerWorkEstimator() + return + } + io.adjustTokens(ctx, metrics) +} + +// allocateTokensTick gives out 1/ticksInAdjustmentInterval of the +// various tokens every 250ms. +func (io *ioLoadListener) allocateTokensTick() { + allocateFunc := func(total int64, allocated int64) (toAllocate int64) { + // unlimitedTokens==MaxInt64, so avoid overflow in the rounding up + // calculation. + if total >= unlimitedTokens-(ticksInAdjustmentInterval-1) { + toAllocate = total / ticksInAdjustmentInterval + } else { + // Round up so that we don't accumulate tokens to give in a burst on the + // last tick. + toAllocate = (total + ticksInAdjustmentInterval - 1) / ticksInAdjustmentInterval + if toAllocate < 0 { + panic(errors.AssertionFailedf("toAllocate is negative %d", toAllocate)) + } + if toAllocate+allocated > total { + toAllocate = total - allocated + } + } + return toAllocate + } + // INVARIANT: toAllocate* >= 0. + toAllocateByteTokens := allocateFunc(io.totalNumByteTokens, io.byteTokensAllocated) + if toAllocateByteTokens < 0 { + panic(errors.AssertionFailedf("toAllocateByteTokens is negative %d", toAllocateByteTokens)) + } + toAllocateElasticDiskBWTokens := + allocateFunc(io.elasticDiskBWTokens, io.elasticDiskBWTokensAllocated) + if toAllocateElasticDiskBWTokens < 0 { + panic(errors.AssertionFailedf("toAllocateElasticDiskBWTokens is negative %d", + toAllocateElasticDiskBWTokens)) + } + // INVARIANT: toAllocate >= 0. + io.mu.Lock() + defer io.mu.Unlock() + io.byteTokensAllocated += toAllocateByteTokens + if io.byteTokensAllocated < 0 { + panic(errors.AssertionFailedf("tokens allocated is negative %d", io.byteTokensAllocated)) + } + io.byteTokensUsed += io.mu.kvGranter.setAvailableIOTokensLocked(toAllocateByteTokens) + io.elasticDiskBWTokensAllocated += toAllocateElasticDiskBWTokens + io.mu.kvGranter.setAvailableElasticDiskBandwidthTokensLocked(toAllocateElasticDiskBWTokens) +} + +func computeIntervalDiskLoadInfo( + prevCumBytesRead uint64, prevCumBytesWritten uint64, diskStats DiskStats, +) intervalDiskLoadInfo { + return intervalDiskLoadInfo{ + readBandwidth: int64((diskStats.BytesRead - prevCumBytesRead) / adjustmentInterval), + writeBandwidth: int64((diskStats.BytesWritten - prevCumBytesWritten) / adjustmentInterval), + provisionedBandwidth: diskStats.ProvisionedBandwidth, + } +} + +// adjustTokens computes a new value of totalNumByteTokens (and resets +// tokensAllocated). The new value, when overloaded, is based on comparing how +// many bytes are being moved out of L0 via compactions with the average +// number of bytes being added to L0 per KV work. We want the former to be +// (significantly) larger so that L0 returns to a healthy state. The byte +// token computation also takes into account the flush throughput, since an +// inability to flush fast enough can result in write stalls due to high +// memtable counts, which we want to avoid as it can cause latency hiccups of +// 100+ms for all write traffic. +func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics) { + sas := io.kvRequester.getStoreAdmissionStats() + res := io.adjustTokensInner(ctx, io.ioLoadListenerState, + metrics.Levels[0], metrics.WriteStallCount, metrics.InternalIntervalMetrics, + L0FileCountOverloadThreshold.Get(&io.settings.SV), + L0SubLevelCountOverloadThreshold.Get(&io.settings.SV), + MinFlushUtilizationFraction.Get(&io.settings.SV), + ) + io.adjustTokensResult = res + cumLSMIncomingBytes, cumLSMIngestedBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics) + { + // Disk Bandwidth tokens. + io.aux.diskBW.intervalDiskLoadInfo = computeIntervalDiskLoadInfo( + io.diskBW.bytesRead, io.diskBW.bytesWritten, metrics.DiskStats) + io.mu.Lock() + diskTokensUsed := io.mu.kvGranter.getDiskTokensUsedAndResetLocked() + io.mu.Unlock() + io.aux.diskBW.intervalLSMInfo = intervalLSMInfo{ + incomingBytes: int64(cumLSMIncomingBytes) - int64(io.diskBW.incomingLSMBytes), + regularTokensUsed: diskTokensUsed[regularWorkClass], + elasticTokensUsed: diskTokensUsed[elasticWorkClass], + } + if metrics.DiskStats.ProvisionedBandwidth > 0 { + io.elasticDiskBWTokens = io.diskBandwidthLimiter.computeElasticTokens(ctx, + io.aux.diskBW.intervalDiskLoadInfo, io.aux.diskBW.intervalLSMInfo) + io.elasticDiskBWTokensAllocated = 0 + } + if metrics.DiskStats.ProvisionedBandwidth == 0 || + !DiskBandwidthTokensForElasticEnabled.Get(&io.settings.SV) { + io.elasticDiskBWTokens = unlimitedTokens + } + io.diskBW.bytesRead = metrics.DiskStats.BytesRead + io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten + io.diskBW.incomingLSMBytes = cumLSMIncomingBytes + } + io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIngestedBytes, sas) + io.copyAuxEtcFromPerWorkEstimator() + requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission() + io.kvRequester.setStoreRequestEstimates(requestEstimates) + l0WriteLM, l0IngestLM, ingestLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone() + io.mu.Lock() + io.mu.kvGranter.setAdmittedDoneModelsLocked(l0WriteLM, l0IngestLM, ingestLM) + io.mu.Unlock() + if _, overloaded := io.ioThreshold.Score(); overloaded || io.aux.doLogFlush || + io.elasticDiskBWTokens != unlimitedTokens { + log.Infof(ctx, "IO overload: %s", io.adjustTokensResult) + } +} + +// copyAuxEtcFromPerWorkEstimator copies the auxiliary and other numerical +// state from io.perWorkTokenEstimator. This is helpful in keeping all the +// numerical state for understanding the behavior of ioLoadListener and its +// helpers in one place for simplicity of logging. +func (io *ioLoadListener) copyAuxEtcFromPerWorkEstimator() { + // Copy the aux so that the printing story is simplified. + io.adjustTokensResult.aux.perWorkTokensAux = io.perWorkTokenEstimator.aux + requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission() + io.adjustTokensResult.requestEstimates = requestEstimates + l0WriteLM, l0IngestLM, ingestLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone() + io.adjustTokensResult.l0WriteLM = l0WriteLM + io.adjustTokensResult.l0IngestLM = l0IngestLM + io.adjustTokensResult.ingestLM = ingestLM +} + +type tokenKind int8 + +const ( + compactionTokenKind tokenKind = iota + flushTokenKind +) + +// adjustTokensAuxComputations encapsulates auxiliary numerical state for +// ioLoadListener that is helpful for understanding its behavior. +type adjustTokensAuxComputations struct { + intL0AddedBytes int64 + intL0CompactedBytes int64 + + intFlushTokens float64 + intFlushUtilization float64 + intWriteStalls int64 + + prevTokensUsed int64 + tokenKind tokenKind + + perWorkTokensAux perWorkTokensAux + doLogFlush bool + + diskBW struct { + intervalDiskLoadInfo intervalDiskLoadInfo + intervalLSMInfo intervalLSMInfo + } +} + +// adjustTokensInner is used for computing tokens based on compaction and +// flush bottlenecks. +func (*ioLoadListener) adjustTokensInner( + ctx context.Context, + prev ioLoadListenerState, + l0Metrics pebble.LevelMetrics, + cumWriteStallCount int64, + im *pebble.InternalIntervalMetrics, + threshNumFiles, threshNumSublevels int64, + minFlushUtilTargetFraction float64, +) adjustTokensResult { + ioThreshold := &admissionpb.IOThreshold{ + L0NumFiles: l0Metrics.NumFiles, + L0NumFilesThreshold: threshNumFiles, + L0NumSubLevels: int64(l0Metrics.Sublevels), + L0NumSubLevelsThreshold: threshNumSublevels, + } + + curL0Bytes := l0Metrics.Size + cumL0AddedBytes := l0Metrics.BytesFlushed + l0Metrics.BytesIngested + // L0 growth over the last interval. + intL0AddedBytes := int64(cumL0AddedBytes) - int64(prev.cumL0AddedBytes) + if intL0AddedBytes < 0 { + // intL0AddedBytes is a simple delta computation over individually cumulative + // stats, so should not be negative. + log.Warningf(ctx, "intL0AddedBytes %d is negative", intL0AddedBytes) + intL0AddedBytes = 0 + } + // intL0CompactedBytes are due to finished compactions. + intL0CompactedBytes := prev.curL0Bytes + intL0AddedBytes - curL0Bytes + if intL0CompactedBytes < 0 { + // Ignore potential inconsistencies across cumulative stats and current L0 + // bytes (gauge). + intL0CompactedBytes = 0 + } + const alpha = 0.5 + // Compaction scheduling can be uneven in prioritizing L0 for compactions, + // so smooth out what is being removed by compactions. + smoothedIntL0CompactedBytes := int64(alpha*float64(intL0CompactedBytes) + (1-alpha)*float64(prev.smoothedIntL0CompactedBytes)) + + // Flush tokens: + // + // Write stalls happen when flushing of memtables is a bottleneck. + // + // Computing Flush Tokens: + // Flush can go from not being the bottleneck in one 15s interval + // (adjustmentInterval) to being the bottleneck in the next 15s interval + // (e.g. when L0 falls below the unhealthy threshold and compaction tokens + // become unlimited). So the flush token limit has to react quickly (cannot + // afford to wait for multiple 15s intervals). We've observed that if we + // normalize the flush rate based on flush loop utilization (the PeakRate + // computation below), and use that to compute flush tokens, the token + // counts are quite stable. Here are two examples, showing this steady token + // count computed using PeakRate of the flush ThroughputMetric, despite + // changes in flush loop utilization (the util number below). + // + // Example 1: Case where IO bandwidth was not a bottleneck + // flush: tokens: 2312382401, util: 0.90 + // flush: tokens: 2345477107, util: 0.31 + // flush: tokens: 2317829891, util: 0.29 + // flush: tokens: 2428387843, util: 0.17 + // + // Example 2: Case where IO bandwidth became a bottleneck (and mean fsync + // latency was fluctuating between 1ms and 4ms in the low util to high util + // cases). + // + // flush: tokens: 1406132615, util: 1.00 + // flush: tokens: 1356476227, util: 0.64 + // flush: tokens: 1374880806, util: 0.24 + // flush: tokens: 1328578534, util: 0.96 + // + // Hence, using PeakRate as a basis for computing flush tokens seems sound. + // The other important question is what fraction of PeakRate avoids write + // stalls. It is likely less than 100% since while a flush is ongoing, + // memtables can accumulate and cause a stall. For example, we have observed + // write stalls at 80% of PeakRate. The fraction depends on configuration + // parameters like MemTableStopWritesThreshold (defaults to 4 in + // CockroachDB), and environmental and workload factors like how long a + // flush takes to flush a single 64MB memtable. Instead of trying to measure + // and adjust for these, we use a simple multiplier, + // flushUtilTargetFraction. By default, flushUtilTargetFraction ranges + // between 0.5 and 1.5. The lower bound is configurable via + // admission.min_flush_util_percent and if configured above the upper bound, + // the upper bound will be ignored and the target fraction will not be + // dynamically adjusted. The dynamic adjustment logic uses an additive step + // size of flushUtilTargetFractionIncrement (0.025), with the following + // logic: + // - Reduce the fraction if there is a write-stall. The reduction may use a + // small multiple of flushUtilTargetFractionIncrement. This is so that + // this probing spends more time below the threshold where write stalls + // occur. + // - Increase fraction if no write-stall and flush tokens were almost all + // used. + // + // This probing unfortunately cannot eliminate write stalls altogether. + // Future improvements could use more history to settle on a good + // flushUtilTargetFraction for longer, or use some measure of how close we + // are to a write-stall to stop the increase. + // + // Ingestion and flush tokens: + // + // Ingested sstables do not utilize any flush capacity. Consider 2 cases: + // - sstable ingested into L0: there was either data overlap with L0, or + // file boundary overlap with L0-L6. To be conservative, lets assume there + // was data overlap, and that this data overlap extended into the memtable + // at the time of ingestion. Memtable(s) would have been force flushed to + // handle such overlap. The cost of flushing a memtable is based on how + // much of the allocated memtable capacity is used, so an early flush + // seems harmless. However, write stalls are based on allocated memtable + // capacity, so there is a potential negative interaction of these forced + // flushes since they cause additional memtable capacity allocation. + // - sstable ingested into L1-L6: there was no data overlap with L0, which + // implies that there was no reason to flush memtables. + // + // Since there is some interaction noted in bullet 1, and because it + // simplifies the admission control token behavior, we use flush tokens in + // an identical manner as compaction tokens -- to be consumed by all data + // flowing into L0. Some of this conservative choice will be compensated for + // by flushUtilTargetFraction (when the mix of ingestion and actual flushes + // are stable). Another thing to note is that compactions out of L0 are + // typically the more persistent bottleneck than flushes for the following + // reason: + // There is a dedicated flush thread. With a maximum compaction concurrency + // of C, we have up to C threads dedicated to handling the write-amp of W + // (caused by rewriting the same data). So C/(W-1) threads on average are + // reading the original data (that will be rewritten W-1 times). Since L0 + // can have multiple overlapping files, and intra-L0 compactions are usually + // avoided, we can assume (at best) that the original data (in L0) is being + // read only when compacting to levels lower than L0. That is, C/(W-1) + // threads are reading from L0 to compact to levels lower than L0. Since W + // can be 20+ and C defaults to 3 (we plan to dynamically adjust C but one + // can expect C to be <= 10), C/(W-1) < 1. So the main reason we are + // considering flush tokens is transient flush bottlenecks, and workloads + // where W is small. + + // Compute flush utilization for this interval. A very low flush utilization + // will cause flush tokens to be unlimited. + intFlushUtilization := float64(0) + if im.Flush.WriteThroughput.WorkDuration > 0 { + intFlushUtilization = float64(im.Flush.WriteThroughput.WorkDuration) / + float64(im.Flush.WriteThroughput.WorkDuration+im.Flush.WriteThroughput.IdleDuration) + } + // Compute flush tokens for this interval that would cause 100% utilization. + intFlushTokens := float64(im.Flush.WriteThroughput.PeakRate()) * adjustmentInterval + intWriteStalls := cumWriteStallCount - prev.cumWriteStallCount + + // Ensure flushUtilTargetFraction is in the configured bounds. This also + // does lazy initialization. + const maxFlushUtilTargetFraction = 1.5 + flushUtilTargetFraction := prev.flushUtilTargetFraction + if flushUtilTargetFraction == 0 { + // Initialization: use the maximum configured fraction. + flushUtilTargetFraction = minFlushUtilTargetFraction + if flushUtilTargetFraction < maxFlushUtilTargetFraction { + flushUtilTargetFraction = maxFlushUtilTargetFraction + } + } else if flushUtilTargetFraction < minFlushUtilTargetFraction { + // The min can be changed in a running system, so we bump up to conform to + // the min. + flushUtilTargetFraction = minFlushUtilTargetFraction + } + numFlushTokens := int64(unlimitedTokens) + // doLogFlush becomes true if something interesting is done here. + doLogFlush := false + smoothedNumFlushTokens := prev.smoothedNumFlushTokens + const flushUtilIgnoreThreshold = 0.05 + if intFlushUtilization > flushUtilIgnoreThreshold { + if smoothedNumFlushTokens == 0 { + // Initialization. + smoothedNumFlushTokens = intFlushTokens + } else { + smoothedNumFlushTokens = alpha*intFlushTokens + (1-alpha)*prev.smoothedNumFlushTokens + } + const flushUtilTargetFractionIncrement = 0.025 + // Have we used, over the last (15s) cycle, more than 90% of the tokens we + // would give out for the next cycle? If yes, highTokenUsage is true. + highTokenUsage := + float64(prev.byteTokensUsed) >= 0.9*smoothedNumFlushTokens*flushUtilTargetFraction + if intWriteStalls > 0 { + // Try decrease since there were write-stalls. + numDecreaseSteps := 1 + // These constants of 5, 3, 2, 2 were found to work reasonably well, + // without causing large decreases. We need better benchmarking to tune + // such constants. + if intWriteStalls >= 5 { + numDecreaseSteps = 3 + } else if intWriteStalls >= 2 { + numDecreaseSteps = 2 + } + for i := 0; i < numDecreaseSteps; i++ { + if flushUtilTargetFraction >= minFlushUtilTargetFraction+flushUtilTargetFractionIncrement { + flushUtilTargetFraction -= flushUtilTargetFractionIncrement + doLogFlush = true + } else { + break + } + } + } else if flushUtilTargetFraction < maxFlushUtilTargetFraction-flushUtilTargetFractionIncrement && + intWriteStalls == 0 && highTokenUsage { + // No write-stalls, and token usage was high, so give out more tokens. + flushUtilTargetFraction += flushUtilTargetFractionIncrement + doLogFlush = true + } + if highTokenUsage { + doLogFlush = true + } + flushTokensFloat := flushUtilTargetFraction * smoothedNumFlushTokens + if flushTokensFloat < float64(math.MaxInt64) { + numFlushTokens = int64(flushTokensFloat) + } + // Else avoid overflow by using the previously set unlimitedTokens. This + // should not really happen. + } + // Else intFlushUtilization is too low. We don't want to make token + // determination based on a very low utilization, so we hand out unlimited + // tokens. Note that flush utilization has been observed to fluctuate from + // 0.16 to 0.9 in a single interval, when compaction tokens are not limited, + // hence we have set flushUtilIgnoreThreshold to a very low value. If we've + // erred towards it being too low, we run the risk of computing incorrect + // tokens. If we've erred towards being too high, we run the risk of giving + // out unlimitedTokens and causing write stalls. + + // We constrain admission based on compactions, if the store is over the L0 + // threshold. + var totalNumByteTokens int64 + var smoothedCompactionByteTokens float64 + + _, overloaded := ioThreshold.Score() + if overloaded { + // Don't admit more byte work than we can remove via compactions. totalNumByteTokens + // tracks our goal for admission. + // Scale down since we want to get under the thresholds over time. This + // scaling could be adjusted based on how much above the threshold we are, + // but for now we just use a constant. + fTotalNumByteTokens := float64(smoothedIntL0CompactedBytes / 2.0) + // Smooth it. This may seem peculiar since we are already using + // smoothedIntL0CompactedBytes, but the else clause below uses a different + // computation so we also want the history of smoothedTotalNumByteTokens. + smoothedCompactionByteTokens = alpha*fTotalNumByteTokens + (1-alpha)*prev.smoothedCompactionByteTokens + if float64(math.MaxInt64) < smoothedCompactionByteTokens { + // Avoid overflow. This should not really happen. + totalNumByteTokens = math.MaxInt64 + } else { + totalNumByteTokens = int64(smoothedCompactionByteTokens) + } + } else { + // Under the threshold. Maintain a smoothedTotalNumByteTokens based on what was + // removed, so that when we go over the threshold we have some history. + // This is also useful when we temporarily dip below the threshold -- + // we've seen extreme situations with alternating 15s intervals of above + // and below the threshold. + numTokens := intL0CompactedBytes + smoothedCompactionByteTokens = alpha*float64(numTokens) + (1-alpha)*prev.smoothedCompactionByteTokens + totalNumByteTokens = unlimitedTokens + } + // Use the minimum of the token count calculated using compactions and + // flushes. + tokenKind := compactionTokenKind + if totalNumByteTokens > numFlushTokens { + totalNumByteTokens = numFlushTokens + tokenKind = flushTokenKind + } + // Install the latest cumulative stats. + return adjustTokensResult{ + ioLoadListenerState: ioLoadListenerState{ + cumL0AddedBytes: cumL0AddedBytes, + curL0Bytes: curL0Bytes, + cumWriteStallCount: cumWriteStallCount, + smoothedIntL0CompactedBytes: smoothedIntL0CompactedBytes, + smoothedCompactionByteTokens: smoothedCompactionByteTokens, + smoothedNumFlushTokens: smoothedNumFlushTokens, + flushUtilTargetFraction: flushUtilTargetFraction, + totalNumByteTokens: totalNumByteTokens, + byteTokensAllocated: 0, + byteTokensUsed: 0, + }, + aux: adjustTokensAuxComputations{ + intL0AddedBytes: intL0AddedBytes, + intL0CompactedBytes: intL0CompactedBytes, + intFlushTokens: intFlushTokens, + intFlushUtilization: intFlushUtilization, + intWriteStalls: intWriteStalls, + prevTokensUsed: prev.byteTokensUsed, + tokenKind: tokenKind, + doLogFlush: doLogFlush, + }, + ioThreshold: ioThreshold, + } +} + +// adjustTokensResult encapsulates all the numerical state of ioLoadListener. +type adjustTokensResult struct { + ioLoadListenerState + requestEstimates storeRequestEstimates + l0WriteLM tokensLinearModel + l0IngestLM tokensLinearModel + ingestLM tokensLinearModel + aux adjustTokensAuxComputations + ioThreshold *admissionpb.IOThreshold // never nil +} + +func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { + ib := humanizeutil.IBytes + // NB: "≈" indicates smoothed quantities. + p.Printf("compaction score %v (%d ssts, %d sub-levels), ", res.ioThreshold, res.ioThreshold.L0NumFiles, res.ioThreshold.L0NumSubLevels) + p.Printf("L0 growth %s (write %s ingest %s ignored %s): ", ib(res.aux.intL0AddedBytes), + ib(res.aux.perWorkTokensAux.intL0WriteBytes), ib(res.aux.perWorkTokensAux.intL0IngestedBytes), + ib(res.aux.perWorkTokensAux.intL0IgnoredIngestedBytes)) + // Writes to L0 that we expected because requests told admission control. + // This is the "easy path", from an estimation perspective, if all regular + // writes accurately tell us what they write, and all ingests tell us what + // they ingest and all of ingests into L0. + p.Printf("requests %d (%d bypassed) with ", res.aux.perWorkTokensAux.intWorkCount, + res.aux.perWorkTokensAux.intBypassedWorkCount) + p.Printf("%s acc-write (%s bypassed) + ", + ib(res.aux.perWorkTokensAux.intL0WriteAccountedBytes), + ib(res.aux.perWorkTokensAux.intL0WriteBypassedAccountedBytes)) + // Ingestion bytes that we expected because requests told admission control. + p.Printf("%s acc-ingest (%s bypassed) + ", + ib(res.aux.perWorkTokensAux.intIngestedAccountedBytes), + ib(res.aux.perWorkTokensAux.intIngestedBypassedAccountedBytes)) + // The models we are fitting to compute tokens based on the reported size of + // the write and ingest. + p.Printf("write-model %.2fx+%s (smoothed %.2fx+%s) + ", + res.aux.perWorkTokensAux.intL0WriteLinearModel.multiplier, + ib(res.aux.perWorkTokensAux.intL0WriteLinearModel.constant), + res.l0WriteLM.multiplier, ib(res.l0WriteLM.constant)) + p.Printf("ingested-model %.2fx+%s (smoothed %.2fx+%s) + ", + res.aux.perWorkTokensAux.intL0IngestedLinearModel.multiplier, + ib(res.aux.perWorkTokensAux.intL0IngestedLinearModel.constant), + res.l0IngestLM.multiplier, ib(res.l0IngestLM.constant)) + // The tokens used per request at admission time, when no size information + // is known. + p.Printf("at-admission-tokens %s, ", ib(res.requestEstimates.writeTokens)) + // How much got compacted out of L0 recently. + p.Printf("compacted %s [≈%s], ", ib(res.aux.intL0CompactedBytes), ib(res.smoothedIntL0CompactedBytes)) + // The tokens computed for flush, based on observed flush throughput and + // utilization. + p.Printf("flushed %s [≈%s]; ", ib(int64(res.aux.intFlushTokens)), + ib(int64(res.smoothedNumFlushTokens))) + p.Printf("admitting ") + if n := res.ioLoadListenerState.totalNumByteTokens; n < unlimitedTokens { + p.Printf("%s (rate %s/s)", ib(n), ib(n/adjustmentInterval)) + switch res.aux.tokenKind { + case compactionTokenKind: + p.Printf(" due to L0 growth") + case flushTokenKind: + p.Printf(" due to memtable flush (multiplier %.3f)", res.flushUtilTargetFraction) + } + p.Printf(" (used %s)", ib(res.aux.prevTokensUsed)) + } else { + p.SafeString("all") + } + if res.elasticDiskBWTokens != unlimitedTokens { + p.Printf("; elastic tokens %s (used %s, regular used %s): "+ + "write model %.2fx+%s ingest model %.2fx+%s, ", + ib(res.elasticDiskBWTokens), ib(res.aux.diskBW.intervalLSMInfo.elasticTokensUsed), + ib(res.aux.diskBW.intervalLSMInfo.regularTokensUsed), + res.l0WriteLM.multiplier, ib(res.l0WriteLM.constant), + res.ingestLM.multiplier, ib(res.ingestLM.constant)) + p.Printf("disk bw read %s write %s provisioned %s", + ib(res.aux.diskBW.intervalDiskLoadInfo.readBandwidth), + ib(res.aux.diskBW.intervalDiskLoadInfo.writeBandwidth), + ib(res.aux.diskBW.intervalDiskLoadInfo.provisionedBandwidth)) + } +} + +func (res adjustTokensResult) String() string { + return redact.StringWithoutMarkers(res) +} diff --git a/pkg/util/admission/store_token_estimation.go b/pkg/util/admission/store_token_estimation.go index caad156e7f50..3e76c5889193 100644 --- a/pkg/util/admission/store_token_estimation.go +++ b/pkg/util/admission/store_token_estimation.go @@ -399,3 +399,10 @@ func (e *storePerWorkTokenEstimator) getModelsAtAdmittedDone() ( e.atDoneL0IngestTokensLinearModel.smoothedLinearModel, e.atDoneIngestTokensLinearModel.smoothedLinearModel } + +func max(i, j int64) int64 { + if i < j { + return j + } + return i +} From 105c68cf15243d0169134207a9fe09b196d929e6 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 12 Aug 2022 17:19:01 -0400 Subject: [PATCH 2/9] admission: move GrantCoordinator into its own file Also bring along with it some closely related structs: StoreGrantCoordinators, Options, GrantCoordinatorMetrics. The latter was previously named GranterMetrics, but this feels more appropriate. Pure code movement change. Release note: None --- pkg/util/admission/BUILD.bazel | 1 + pkg/util/admission/grant_coordinator.go | 925 ++++++++++++++++++++++++ pkg/util/admission/granter.go | 903 ----------------------- pkg/util/admission/granter_test.go | 2 +- 4 files changed, 927 insertions(+), 904 deletions(-) create mode 100644 pkg/util/admission/grant_coordinator.go diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index 2fdc12af5cc0..eb16b558569c 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -6,6 +6,7 @@ go_library( srcs = [ "disk_bandwidth.go", "doc.go", + "grant_coordinator.go", "granter.go", "io_load_listener.go", "store_token_estimation.go", diff --git a/pkg/util/admission/grant_coordinator.go b/pkg/util/admission/grant_coordinator.go new file mode 100644 index 000000000000..85f716fd23ce --- /dev/null +++ b/pkg/util/admission/grant_coordinator.go @@ -0,0 +1,925 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package admission + +import ( + "context" + "time" + "unsafe" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" +) + +// GrantCoordinators holds a regular GrantCoordinator for all work, and a +// StoreGrantCoordinators that allows for per-store GrantCoordinators for +// KVWork that involves writes. +type GrantCoordinators struct { + Stores *StoreGrantCoordinators + Regular *GrantCoordinator +} + +// Close implements the stop.Closer interface. +func (gcs GrantCoordinators) Close() { + gcs.Stores.close() + gcs.Regular.Close() +} + +// StoreGrantCoordinators is a container for GrantCoordinators for each store, +// that is used for KV work admission that takes into account store health. +// Currently it is intended only for writes to stores. +type StoreGrantCoordinators struct { + ambientCtx log.AmbientContext + + settings *cluster.Settings + makeStoreRequesterFunc makeStoreRequesterFunc + kvIOTokensExhaustedDuration *metric.Counter + // These metrics are shared by WorkQueues across stores. + workQueueMetrics WorkQueueMetrics + + gcMap syncutil.IntMap // map[int64(StoreID)]*GrantCoordinator + // numStores is used to track the number of stores which have been added + // to the gcMap. This is used because the IntMap doesn't expose a size + // api. + numStores int + pebbleMetricsProvider PebbleMetricsProvider + closeCh chan struct{} + + disableTickerForTesting bool +} + +// SetPebbleMetricsProvider sets a PebbleMetricsProvider and causes the load +// on the various storage engines to be used for admission control. +func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider( + startupCtx context.Context, pmp PebbleMetricsProvider, iotc IOThresholdConsumer, +) { + if sgc.pebbleMetricsProvider != nil { + panic(errors.AssertionFailedf("SetPebbleMetricsProvider called more than once")) + } + sgc.pebbleMetricsProvider = pmp + sgc.closeCh = make(chan struct{}) + metrics := sgc.pebbleMetricsProvider.GetPebbleMetrics() + for _, m := range metrics { + gc := sgc.initGrantCoordinator(m.StoreID) + // Defensive call to LoadAndStore even though Store ought to be sufficient + // since SetPebbleMetricsProvider can only be called once. This code + // guards against duplication of stores returned by GetPebbleMetrics. + _, loaded := sgc.gcMap.LoadOrStore(int64(m.StoreID), unsafe.Pointer(gc)) + if !loaded { + sgc.numStores++ + } + gc.pebbleMetricsTick(startupCtx, m) + gc.allocateIOTokensTick() + } + if sgc.disableTickerForTesting { + return + } + // Attach tracer and log tags. + ctx := sgc.ambientCtx.AnnotateCtx(context.Background()) + + go func() { + var ticks int64 + ticker := time.NewTicker(ioTokenTickDuration) + done := false + for !done { + select { + case <-ticker.C: + ticks++ + if ticks%ticksInAdjustmentInterval == 0 { + metrics := sgc.pebbleMetricsProvider.GetPebbleMetrics() + if len(metrics) != sgc.numStores { + log.Warningf(ctx, + "expected %d store metrics and found %d metrics", sgc.numStores, len(metrics)) + } + for _, m := range metrics { + if unsafeGc, ok := sgc.gcMap.Load(int64(m.StoreID)); ok { + gc := (*GrantCoordinator)(unsafeGc) + gc.pebbleMetricsTick(ctx, m) + iotc.UpdateIOThreshold(roachpb.StoreID(m.StoreID), gc.ioLoadListener.ioThreshold) + } else { + log.Warningf(ctx, + "seeing metrics for unknown storeID %d", m.StoreID) + } + } + } + sgc.gcMap.Range(func(_ int64, unsafeGc unsafe.Pointer) bool { + gc := (*GrantCoordinator)(unsafeGc) + gc.allocateIOTokensTick() + // true indicates that iteration should continue after the + // current entry has been processed. + return true + }) + case <-sgc.closeCh: + done = true + } + } + ticker.Stop() + }() +} + +func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoordinator { + coord := &GrantCoordinator{ + settings: sgc.settings, + useGrantChains: false, + numProcs: 1, + } + + kvg := &kvStoreTokenGranter{ + coord: coord, + // Setting tokens to unlimited is defensive. We expect that + // pebbleMetricsTick and allocateIOTokensTick will get called during + // initialization, which will also set these to unlimited. + availableIOTokens: unlimitedTokens / ticksInAdjustmentInterval, + startingIOTokens: unlimitedTokens / ticksInAdjustmentInterval, + ioTokensExhaustedDurationMetric: sgc.kvIOTokensExhaustedDuration, + elasticDiskBWTokensAvailable: unlimitedTokens / ticksInAdjustmentInterval, + } + opts := makeWorkQueueOptions(KVWork) + // This is IO work, so override the usesTokens value. + opts.usesTokens = true + // Share the WorkQueue metrics across all stores. + // TODO(sumeer): add per-store WorkQueue state for debug.zip and db console. + opts.metrics = &sgc.workQueueMetrics + granters := [numWorkClasses]granterWithStoreWriteDone{ + &kvStoreTokenChildGranter{ + workClass: regularWorkClass, + parent: kvg, + }, + &kvStoreTokenChildGranter{ + workClass: elasticWorkClass, + parent: kvg, + }, + } + + storeReq := sgc.makeStoreRequesterFunc(sgc.ambientCtx, granters, sgc.settings, opts) + coord.queues[KVWork] = storeReq + requesters := storeReq.getRequesters() + kvg.regularRequester = requesters[regularWorkClass] + kvg.elasticRequester = requesters[elasticWorkClass] + coord.granters[KVWork] = kvg + coord.ioLoadListener = &ioLoadListener{ + storeID: storeID, + settings: sgc.settings, + kvRequester: storeReq, + perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), + diskBandwidthLimiter: makeDiskBandwidthLimiter(), + } + coord.ioLoadListener.mu.Mutex = &coord.mu + coord.ioLoadListener.mu.kvGranter = kvg + return coord +} + +// TryGetQueueForStore returns a WorkQueue for the given storeID, or nil if +// the storeID is not known. +func (sgc *StoreGrantCoordinators) TryGetQueueForStore(storeID int32) *StoreWorkQueue { + if unsafeGranter, ok := sgc.gcMap.Load(int64(storeID)); ok { + granter := (*GrantCoordinator)(unsafeGranter) + return granter.queues[KVWork].(*StoreWorkQueue) + } + return nil +} + +func (sgc *StoreGrantCoordinators) close() { + // closeCh can be nil in tests that never called SetPebbleMetricsProvider. + if sgc.closeCh != nil { + close(sgc.closeCh) + } + + sgc.gcMap.Range(func(_ int64, unsafeGc unsafe.Pointer) bool { + gc := (*GrantCoordinator)(unsafeGc) + gc.Close() + // true indicates that iteration should continue after the + // current entry has been processed. + return true + }) +} + +// GrantCoordinator is the top-level object that coordinates grants across +// different WorkKinds (for more context see the comment in doc.go, and the +// comment where WorkKind is declared). Typically there will one +// GrantCoordinator in a node for CPU intensive work, and for nodes that also +// have the KV layer, one GrantCoordinator per store (these are managed by +// StoreGrantCoordinators) for KVWork that uses that store. See the +// NewGrantCoordinators and NewGrantCoordinatorSQL functions. +type GrantCoordinator struct { + ambientCtx log.AmbientContext + + settings *cluster.Settings + lastCPULoadSamplePeriod time.Duration + + // mu is ordered before any mutex acquired in a requester implementation. + // TODO(sumeer): move everything covered by mu into a nested struct. + mu syncutil.Mutex + // NB: Some granters can be nil. + granters [numWorkKinds]granterWithLockedCalls + // The WorkQueues behaving as requesters in each granterWithLockedCalls. + // This is kept separately only to service GetWorkQueue calls and to call + // close(). + queues [numWorkKinds]requesterClose + // The cpu fields can be nil, and the IO field can be nil, since a + // GrantCoordinator typically handles one of these two resources. + cpuOverloadIndicator cpuOverloadIndicator + cpuLoadListener CPULoadListener + ioLoadListener *ioLoadListener + + // The latest value of GOMAXPROCS, received via CPULoad. Only initialized if + // the cpu resource is being handled by this GrantCoordinator. + numProcs int + + // See the comment at continueGrantChain that explains how a grant chain + // functions and the motivation. When !useGrantChains, grant chains are + // disabled. + useGrantChains bool + + // The admission control code needs high sampling frequency of the cpu load, + // and turns off admission control enforcement when the sampling frequency + // is too low. For testing queueing behavior, we do not want the enforcement + // to be turned off in a non-deterministic manner so add a testing flag to + // disable that feature. + testingDisableSkipEnforcement bool + + // grantChainActive indicates whether a grant chain is active. If active, + // grantChainID is the ID of that chain. If !active, grantChainID is the ID + // of the next chain that will become active. IDs are assigned by + // incrementing grantChainID. If !useGrantChains, grantChainActive is never + // true. + grantChainActive bool + grantChainID grantChainID + // Index into granters, which represents the current WorkKind at which the + // grant chain is operating. Only relevant when grantChainActive is true. + grantChainIndex WorkKind + // See the comment at delayForGrantChainTermination for motivation. + grantChainStartTime time.Time +} + +var _ CPULoadListener = &GrantCoordinator{} + +// Options for constructing GrantCoordinators. +type Options struct { + MinCPUSlots int + MaxCPUSlots int + // RunnableAlphaOverride is used to override the alpha value used to + // compute the ewma of the runnable goroutine counts. It is only used + // during testing. A 0 value indicates that there is no override. + RunnableAlphaOverride float64 + SQLKVResponseBurstTokens int64 + SQLSQLResponseBurstTokens int64 + SQLStatementLeafStartWorkSlots int + SQLStatementRootStartWorkSlots int + TestingDisableSkipEnforcement bool + Settings *cluster.Settings + // Only non-nil for tests. + makeRequesterFunc makeRequesterFunc + makeStoreRequesterFunc makeStoreRequesterFunc +} + +var _ base.ModuleTestingKnobs = &Options{} + +// ModuleTestingKnobs implements the base.ModuleTestingKnobs interface. +func (*Options) ModuleTestingKnobs() {} + +// DefaultOptions are the default settings for various admission control knobs. +var DefaultOptions = Options{ + MinCPUSlots: 1, + MaxCPUSlots: 100000, /* TODO(sumeer): add cluster setting */ + SQLKVResponseBurstTokens: 100000, /* TODO(sumeer): add cluster setting */ + SQLSQLResponseBurstTokens: 100000, /* TODO(sumeer): add cluster setting */ + SQLStatementLeafStartWorkSlots: 100, /* arbitrary, and unused */ + SQLStatementRootStartWorkSlots: 100, /* arbitrary, and unused */ +} + +// Override applies values from "override" to the receiver that differ from Go +// defaults. +func (o *Options) Override(override *Options) { + if override.MinCPUSlots != 0 { + o.MinCPUSlots = override.MinCPUSlots + } + if override.MaxCPUSlots != 0 { + o.MaxCPUSlots = override.MaxCPUSlots + } + if override.SQLKVResponseBurstTokens != 0 { + o.SQLKVResponseBurstTokens = override.SQLKVResponseBurstTokens + } + if override.SQLSQLResponseBurstTokens != 0 { + o.SQLSQLResponseBurstTokens = override.SQLSQLResponseBurstTokens + } + if override.SQLStatementLeafStartWorkSlots != 0 { + o.SQLStatementLeafStartWorkSlots = override.SQLStatementLeafStartWorkSlots + } + if override.SQLStatementRootStartWorkSlots != 0 { + o.SQLStatementRootStartWorkSlots = override.SQLStatementRootStartWorkSlots + } + if override.TestingDisableSkipEnforcement { + o.TestingDisableSkipEnforcement = true + } +} + +type makeRequesterFunc func( + _ log.AmbientContext, workKind WorkKind, granter granter, settings *cluster.Settings, + opts workQueueOptions) requester + +type makeStoreRequesterFunc func( + _ log.AmbientContext, granters [numWorkClasses]granterWithStoreWriteDone, + settings *cluster.Settings, opts workQueueOptions) storeRequester + +// NewGrantCoordinators constructs GrantCoordinators and WorkQueues for a +// regular cluster node. Caller is responsible for hooking up +// GrantCoordinators.Regular to receive calls to CPULoad, and to set a +// PebbleMetricsProvider on GrantCoordinators.Stores. Every request must pass +// through GrantCoordinators.Regular, while only subsets of requests pass +// through each store's GrantCoordinator. We arrange these such that requests +// (that need to) first pass through a store's GrantCoordinator and then +// through the regular one. This ensures that we are not using slots in the +// latter on requests that are blocked elsewhere for admission. Additionally, +// we don't want the CPU scheduler signal that is implicitly used in grant +// chains to delay admission through the per store GrantCoordinators since +// they are not trying to control CPU usage, so we turn off grant chaining in +// those coordinators. +func NewGrantCoordinators( + ambientCtx log.AmbientContext, opts Options, +) (GrantCoordinators, []metric.Struct) { + makeRequester := makeWorkQueue + if opts.makeRequesterFunc != nil { + makeRequester = opts.makeRequesterFunc + } + st := opts.Settings + + metrics := makeGrantCoordinatorMetrics() + metricStructs := append([]metric.Struct(nil), metrics) + kvSlotAdjuster := &kvSlotAdjuster{ + settings: st, + minCPUSlots: opts.MinCPUSlots, + maxCPUSlots: opts.MaxCPUSlots, + totalSlotsMetric: metrics.KVTotalSlots, + totalModerateSlotsMetric: metrics.KVTotalModerateSlots, + moderateSlotsClamp: opts.MaxCPUSlots, + runnableAlphaOverride: opts.RunnableAlphaOverride, + } + coord := &GrantCoordinator{ + ambientCtx: ambientCtx, + settings: st, + cpuOverloadIndicator: kvSlotAdjuster, + cpuLoadListener: kvSlotAdjuster, + useGrantChains: true, + testingDisableSkipEnforcement: opts.TestingDisableSkipEnforcement, + numProcs: 1, + grantChainID: 1, + } + + kvg := &slotGranter{ + coord: coord, + workKind: KVWork, + totalHighLoadSlots: opts.MinCPUSlots, + totalModerateLoadSlots: opts.MinCPUSlots, + usedSlotsMetric: metrics.KVUsedSlots, + usedSoftSlotsMetric: metrics.KVUsedSoftSlots, + } + + kvSlotAdjuster.granter = kvg + req := makeRequester(ambientCtx, KVWork, kvg, st, makeWorkQueueOptions(KVWork)) + coord.queues[KVWork] = req + kvg.requester = req + coord.granters[KVWork] = kvg + + tg := &tokenGranter{ + coord: coord, + workKind: SQLKVResponseWork, + availableBurstTokens: opts.SQLKVResponseBurstTokens, + maxBurstTokens: opts.SQLKVResponseBurstTokens, + cpuOverload: kvSlotAdjuster, + } + req = makeRequester( + ambientCtx, SQLKVResponseWork, tg, st, makeWorkQueueOptions(SQLKVResponseWork)) + coord.queues[SQLKVResponseWork] = req + tg.requester = req + coord.granters[SQLKVResponseWork] = tg + + tg = &tokenGranter{ + coord: coord, + workKind: SQLSQLResponseWork, + availableBurstTokens: opts.SQLSQLResponseBurstTokens, + maxBurstTokens: opts.SQLSQLResponseBurstTokens, + cpuOverload: kvSlotAdjuster, + } + req = makeRequester(ambientCtx, + SQLSQLResponseWork, tg, st, makeWorkQueueOptions(SQLSQLResponseWork)) + coord.queues[SQLSQLResponseWork] = req + tg.requester = req + coord.granters[SQLSQLResponseWork] = tg + + sg := &slotGranter{ + coord: coord, + workKind: SQLStatementLeafStartWork, + totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, + cpuOverload: kvSlotAdjuster, + usedSlotsMetric: metrics.SQLLeafStartUsedSlots, + } + req = makeRequester(ambientCtx, + SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork)) + coord.queues[SQLStatementLeafStartWork] = req + sg.requester = req + coord.granters[SQLStatementLeafStartWork] = sg + + sg = &slotGranter{ + coord: coord, + workKind: SQLStatementRootStartWork, + totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, + cpuOverload: kvSlotAdjuster, + usedSlotsMetric: metrics.SQLRootStartUsedSlots, + } + req = makeRequester(ambientCtx, + SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork)) + coord.queues[SQLStatementRootStartWork] = req + sg.requester = req + coord.granters[SQLStatementRootStartWork] = sg + + metricStructs = appendMetricStructsForQueues(metricStructs, coord) + + // TODO(sumeerbhola): these metrics are shared across all stores and all + // priorities across stores (even the coarser workClasses, which are a + // mapping from priority, share the same metrics). Fix this by adding + // labeled Prometheus metrics. + storeWorkQueueMetrics := makeWorkQueueMetrics(string(workKindString(KVWork)) + "-stores") + metricStructs = append(metricStructs, storeWorkQueueMetrics) + makeStoreRequester := makeStoreWorkQueue + if opts.makeStoreRequesterFunc != nil { + makeStoreRequester = opts.makeStoreRequesterFunc + } + storeCoordinators := &StoreGrantCoordinators{ + settings: st, + makeStoreRequesterFunc: makeStoreRequester, + kvIOTokensExhaustedDuration: metrics.KVIOTokensExhaustedDuration, + workQueueMetrics: storeWorkQueueMetrics, + } + + return GrantCoordinators{Stores: storeCoordinators, Regular: coord}, metricStructs +} + +// NewGrantCoordinatorSQL constructs a GrantCoordinator and WorkQueues for a +// single-tenant SQL node in a multi-tenant cluster. Caller is responsible for +// hooking this up to receive calls to CPULoad. +func NewGrantCoordinatorSQL( + ambientCtx log.AmbientContext, opts Options, +) (*GrantCoordinator, []metric.Struct) { + makeRequester := makeWorkQueue + if opts.makeRequesterFunc != nil { + makeRequester = opts.makeRequesterFunc + } + st := opts.Settings + + metrics := makeGrantCoordinatorMetrics() + metricStructs := append([]metric.Struct(nil), metrics) + sqlNodeCPU := &sqlNodeCPUOverloadIndicator{} + coord := &GrantCoordinator{ + ambientCtx: ambientCtx, + settings: st, + cpuOverloadIndicator: sqlNodeCPU, + cpuLoadListener: sqlNodeCPU, + useGrantChains: true, + numProcs: 1, + grantChainID: 1, + } + + tg := &tokenGranter{ + coord: coord, + workKind: SQLKVResponseWork, + availableBurstTokens: opts.SQLKVResponseBurstTokens, + maxBurstTokens: opts.SQLKVResponseBurstTokens, + cpuOverload: sqlNodeCPU, + } + req := makeRequester(ambientCtx, + SQLKVResponseWork, tg, st, makeWorkQueueOptions(SQLKVResponseWork)) + coord.queues[SQLKVResponseWork] = req + tg.requester = req + coord.granters[SQLKVResponseWork] = tg + + tg = &tokenGranter{ + coord: coord, + workKind: SQLSQLResponseWork, + availableBurstTokens: opts.SQLSQLResponseBurstTokens, + maxBurstTokens: opts.SQLSQLResponseBurstTokens, + cpuOverload: sqlNodeCPU, + } + req = makeRequester(ambientCtx, + SQLSQLResponseWork, tg, st, makeWorkQueueOptions(SQLSQLResponseWork)) + coord.queues[SQLSQLResponseWork] = req + tg.requester = req + coord.granters[SQLSQLResponseWork] = tg + + sg := &slotGranter{ + coord: coord, + workKind: SQLStatementLeafStartWork, + totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, + cpuOverload: sqlNodeCPU, + usedSlotsMetric: metrics.SQLLeafStartUsedSlots, + } + req = makeRequester(ambientCtx, + SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork)) + coord.queues[SQLStatementLeafStartWork] = req + sg.requester = req + coord.granters[SQLStatementLeafStartWork] = sg + + sg = &slotGranter{ + coord: coord, + workKind: SQLStatementRootStartWork, + totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, + cpuOverload: sqlNodeCPU, + usedSlotsMetric: metrics.SQLRootStartUsedSlots, + } + req = makeRequester(ambientCtx, + SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork)) + coord.queues[SQLStatementRootStartWork] = req + sg.requester = req + coord.granters[SQLStatementRootStartWork] = sg + + return coord, appendMetricStructsForQueues(metricStructs, coord) +} + +func appendMetricStructsForQueues(ms []metric.Struct, coord *GrantCoordinator) []metric.Struct { + for i := range coord.queues { + if coord.queues[i] != nil { + q, ok := coord.queues[i].(*WorkQueue) + if ok { + ms = append(ms, q.metrics) + } + } + } + return ms +} + +// pebbleMetricsTick is called every adjustmentInterval seconds and passes +// through to the ioLoadListener, so that it can adjust the plan for future IO +// token allocations. +func (coord *GrantCoordinator) pebbleMetricsTick(ctx context.Context, m StoreMetrics) { + coord.ioLoadListener.pebbleMetricsTick(ctx, m) +} + +// allocateIOTokensTick tells the ioLoadListener to allocate tokens. +func (coord *GrantCoordinator) allocateIOTokensTick() { + coord.ioLoadListener.allocateTokensTick() + coord.mu.Lock() + defer coord.mu.Unlock() + if !coord.grantChainActive { + coord.tryGrant() + } + // Else, let the grant chain finish. NB: we turn off grant chains on the + // GrantCoordinators used for IO, so the if-condition is always true. +} + +// testingTryGrant is only for unit tests, since they sometimes cut out +// support classes like the ioLoadListener. +func (coord *GrantCoordinator) testingTryGrant() { + coord.mu.Lock() + defer coord.mu.Unlock() + if !coord.grantChainActive { + coord.tryGrant() + } +} + +// GetWorkQueue returns the WorkQueue for a particular WorkKind. Can be nil if +// the NewGrantCoordinator* function does not construct a WorkQueue for that +// work. +// Implementation detail: don't use this method when the GrantCoordinator is +// created by the StoreGrantCoordinators since those have a StoreWorkQueues. +// The TryGetQueueForStore is the external facing method in that case since +// the individual GrantCoordinators are hidden. +func (coord *GrantCoordinator) GetWorkQueue(workKind WorkKind) *WorkQueue { + return coord.queues[workKind].(*WorkQueue) +} + +// CPULoad implements CPULoadListener and is called periodically (see +// CPULoadListener for details). The same frequency is used for refilling the +// burst tokens since synchronizing the two means that the refilled burst can +// take into account the latest schedulers stats (indirectly, via the +// implementation of cpuOverloadIndicator). +func (coord *GrantCoordinator) CPULoad(runnable int, procs int, samplePeriod time.Duration) { + ctx := coord.ambientCtx.AnnotateCtx(context.Background()) + + if coord.lastCPULoadSamplePeriod != 0 && coord.lastCPULoadSamplePeriod != samplePeriod && + KVAdmissionControlEnabled.Get(&coord.settings.SV) { + log.Infof(ctx, "CPULoad switching to period %s", samplePeriod.String()) + } + coord.lastCPULoadSamplePeriod = samplePeriod + + coord.mu.Lock() + defer coord.mu.Unlock() + coord.numProcs = procs + coord.cpuLoadListener.CPULoad(runnable, procs, samplePeriod) + + // Slot adjustment and token refilling requires 1ms periods to work well. If + // the CPULoad ticks are less frequent, there is no guarantee that the + // tokens or slots will be sufficient to service requests. This is + // particularly the case for slots where we dynamically adjust them, and + // high contention can suddenly result in high slot utilization even while + // cpu utilization stays low. We don't want to artificially bottleneck + // request processing when we are in this slow CPULoad ticks regime since we + // can't adjust slots or refill tokens fast enough. So we explicitly tell + // the granters to not do token or slot enforcement. + skipEnforcement := samplePeriod > time.Millisecond + coord.granters[SQLKVResponseWork].(*tokenGranter).refillBurstTokens(skipEnforcement) + coord.granters[SQLSQLResponseWork].(*tokenGranter).refillBurstTokens(skipEnforcement) + if coord.granters[KVWork] != nil { + if !coord.testingDisableSkipEnforcement { + kvg := coord.granters[KVWork].(*slotGranter) + kvg.skipSlotEnforcement = skipEnforcement + } + } + if coord.grantChainActive && !coord.tryTerminateGrantChain() { + return + } + coord.tryGrant() +} + +// tryGet is called by granter.tryGet with the WorkKind. +func (coord *GrantCoordinator) tryGet(workKind WorkKind, count int64, demuxHandle int8) bool { + coord.mu.Lock() + defer coord.mu.Unlock() + // It is possible that a grant chain is active, and has not yet made its way + // to this workKind. So it may be more reasonable to queue. But we have some + // concerns about incurring the delay of multiple goroutine context switches + // so we ignore this case. + res := coord.granters[workKind].tryGetLocked(count, demuxHandle) + switch res { + case grantSuccess: + // Grant chain may be active, but it did not get in the way of this grant, + // and the effect of this grant in terms of overload will be felt by the + // grant chain. + return true + case grantFailDueToSharedResource: + // This could be a transient overload, that may not be noticed by the + // grant chain. We don't want it to continue granting to lower priority + // WorkKinds, while a higher priority one is waiting, so we terminate it. + if coord.grantChainActive && coord.grantChainIndex >= workKind { + coord.tryTerminateGrantChain() + } + return false + case grantFailLocal: + return false + default: + panic(errors.AssertionFailedf("unknown grantResult")) + } +} + +// returnGrant is called by granter.returnGrant with the WorkKind. +func (coord *GrantCoordinator) returnGrant(workKind WorkKind, count int64, demuxHandle int8) { + coord.mu.Lock() + defer coord.mu.Unlock() + coord.granters[workKind].returnGrantLocked(count, demuxHandle) + if coord.grantChainActive { + if coord.grantChainIndex > workKind && + coord.granters[workKind].requesterHasWaitingRequests() { + // There are waiting requests that will not be served by the grant chain. + // Better to terminate it and start afresh. + if !coord.tryTerminateGrantChain() { + return + } + } else { + // Else either the grant chain will get to this workKind, or there are no waiting requests. + return + } + } + coord.tryGrant() +} + +// tookWithoutPermission is called by granter.tookWithoutPermission with the +// WorkKind. +func (coord *GrantCoordinator) tookWithoutPermission( + workKind WorkKind, count int64, demuxHandle int8, +) { + coord.mu.Lock() + defer coord.mu.Unlock() + coord.granters[workKind].tookWithoutPermissionLocked(count, demuxHandle) +} + +// continueGrantChain is called by granter.continueGrantChain with the +// WorkKind. Never called if !coord.useGrantChains. +func (coord *GrantCoordinator) continueGrantChain(workKind WorkKind, grantChainID grantChainID) { + if grantChainID == noGrantChain { + return + } + coord.mu.Lock() + defer coord.mu.Unlock() + if coord.grantChainID != grantChainID { + // Someone terminated grantChainID by incrementing coord.grantChainID. + return + } + coord.tryGrant() +} + +// delayForGrantChainTermination causes a delay in terminating a grant chain. +// Terminating a grant chain immediately typically causes a new one to start +// immediately that can burst up to its maximum initial grant burst. Which +// means frequent terminations followed by new starts impose little control +// over the rate at which tokens are granted (slots are better controlled +// since we know when the work finishes). This causes huge spikes in the +// runnable goroutine count, observed at 1ms granularity. This spike causes +// the kvSlotAdjuster to ratchet down the totalSlots for KV work all the way +// down to 1, which later causes the runnable gorouting count to crash down +// to a value close to 0, leading to under-utilization. +// +// TODO(sumeer): design admission behavior metrics that can be used to +// understand the behavior in detail and to quantify improvements when changing +// heuristics. One metric would be mean and variance of the runnable count, +// computed using the 1ms samples, and exported/logged every 60s. +var delayForGrantChainTermination = 100 * time.Millisecond + +// tryTerminateGrantChain attempts to terminate the current grant chain, and +// returns true iff it is terminated, in which case a new one can be +// immediately started. +// REQUIRES: coord.grantChainActive==true +func (coord *GrantCoordinator) tryTerminateGrantChain() bool { + now := timeutil.Now() + if delayForGrantChainTermination > 0 && + now.Sub(coord.grantChainStartTime) < delayForGrantChainTermination { + return false + } + // Incrementing the ID will cause the existing grant chain to die out when + // the grantee calls continueGrantChain. + coord.grantChainID++ + coord.grantChainActive = false + coord.grantChainStartTime = time.Time{} + return true +} + +// tryGrant tries to either continue an existing grant chain, or if no grant +// chain is active, tries to start a new grant chain when grant chaining is +// enabled, or grants as much as it can when grant chaining is disabled. +func (coord *GrantCoordinator) tryGrant() { + startingChain := false + if !coord.grantChainActive { + // NB: always set to true when !coord.useGrantChains, and we won't + // actually use this to start a grant chain (see below). + startingChain = true + coord.grantChainIndex = 0 + } + // Assume that we will not be able to start a new grant chain, or that the + // existing one will die out. The code below will set it to true if neither + // is true. + coord.grantChainActive = false + grantBurstCount := 0 + // Grant in a burst proportional to numProcs, to generate a runnable for + // each. + grantBurstLimit := coord.numProcs + // Additionally, increase the burst size proportional to a fourth of the + // overload threshold. We experimentally observed that this resulted in + // better CPU utilization. We don't use the full overload threshold since we + // don't want to over grant for non-KV work since that causes the KV slots + // to (unfairly) start decreasing, since we lose control over how many + // goroutines are runnable. + multiplier := int(KVSlotAdjusterOverloadThreshold.Get(&coord.settings.SV) / 4) + if multiplier == 0 { + multiplier = 1 + } + grantBurstLimit *= multiplier + // Only the case of a grant chain being active returns from within the + // OuterLoop. +OuterLoop: + for ; coord.grantChainIndex < numWorkKinds; coord.grantChainIndex++ { + localDone := false + granter := coord.granters[coord.grantChainIndex] + if granter == nil { + // A GrantCoordinator can be limited to certain WorkKinds, and the + // remaining will be nil. + continue + } + for granter.requesterHasWaitingRequests() && !localDone { + chainID := noGrantChain + if grantBurstCount+1 == grantBurstLimit && coord.useGrantChains { + chainID = coord.grantChainID + } + res := granter.tryGrantLocked(chainID) + switch res { + case grantSuccess: + grantBurstCount++ + if grantBurstCount == grantBurstLimit && coord.useGrantChains { + coord.grantChainActive = true + if startingChain { + coord.grantChainStartTime = timeutil.Now() + } + return + } + case grantFailDueToSharedResource: + break OuterLoop + case grantFailLocal: + localDone = true + default: + panic(errors.AssertionFailedf("unknown grantResult")) + } + } + } + // INVARIANT: !grantChainActive. The chain either did not start or the + // existing one died. If the existing one died, we increment grantChainID + // since it represents the ID to be used for the next chain. Note that + // startingChain is always true when !useGrantChains, so this if-block is + // not executed. + if !startingChain { + coord.grantChainID++ + } +} + +// Close implements the stop.Closer interface. +func (coord *GrantCoordinator) Close() { + for i := range coord.queues { + if coord.queues[i] != nil { + coord.queues[i].close() + } + } +} + +func (coord *GrantCoordinator) String() string { + return redact.StringWithoutMarkers(coord) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) { + coord.mu.Lock() + defer coord.mu.Unlock() + s.Printf("(chain: id: %d active: %t index: %d)", + coord.grantChainID, coord.grantChainActive, coord.grantChainIndex, + ) + + spaceStr := redact.RedactableString(" ") + newlineStr := redact.RedactableString("\n") + curSep := spaceStr + for i := range coord.granters { + kind := WorkKind(i) + switch kind { + case KVWork: + switch g := coord.granters[i].(type) { + case *slotGranter: + kvsa := coord.cpuLoadListener.(*kvSlotAdjuster) + s.Printf( + "%s%s: used: %d, high(moderate)-total: %d(%d) moderate-clamp: %d", curSep, workKindString(kind), + g.usedSlots, g.totalHighLoadSlots, g.totalModerateLoadSlots, kvsa.moderateSlotsClamp) + if g.usedSoftSlots > 0 { + s.Printf(" used-soft: %d", g.usedSoftSlots) + } + case *kvStoreTokenGranter: + s.Printf(" io-avail: %d, elastic-disk-bw-tokens-avail: %d", g.availableIOTokens, + g.elasticDiskBWTokensAvailable) + } + case SQLStatementLeafStartWork, SQLStatementRootStartWork: + if coord.granters[i] != nil { + g := coord.granters[i].(*slotGranter) + s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalHighLoadSlots) + } + case SQLKVResponseWork, SQLSQLResponseWork: + if coord.granters[i] != nil { + g := coord.granters[i].(*tokenGranter) + s.Printf("%s%s: avail: %d", curSep, workKindString(kind), g.availableBurstTokens) + if kind == SQLKVResponseWork { + curSep = newlineStr + } else { + curSep = spaceStr + } + } + } + } +} + +// GrantCoordinatorMetrics are metrics associated with a GrantCoordinator. +type GrantCoordinatorMetrics struct { + KVTotalSlots *metric.Gauge + KVUsedSlots *metric.Gauge + KVTotalModerateSlots *metric.Gauge + KVUsedSoftSlots *metric.Gauge + KVIOTokensExhaustedDuration *metric.Counter + SQLLeafStartUsedSlots *metric.Gauge + SQLRootStartUsedSlots *metric.Gauge +} + +// MetricStruct implements the metric.Struct interface. +func (GrantCoordinatorMetrics) MetricStruct() {} + +func makeGrantCoordinatorMetrics() GrantCoordinatorMetrics { + m := GrantCoordinatorMetrics{ + KVTotalSlots: metric.NewGauge(totalSlots), + KVUsedSlots: metric.NewGauge(addName(string(workKindString(KVWork)), usedSlots)), + KVTotalModerateSlots: metric.NewGauge(totalModerateSlots), + KVUsedSoftSlots: metric.NewGauge(usedSoftSlots), + KVIOTokensExhaustedDuration: metric.NewCounter(kvIOTokensExhaustedDuration), + SQLLeafStartUsedSlots: metric.NewGauge( + addName(string(workKindString(SQLStatementLeafStartWork)), usedSlots)), + SQLRootStartUsedSlots: metric.NewGauge( + addName(string(workKindString(SQLStatementRootStartWork)), usedSlots)), + } + return m +} + +// Prevent the linter from emitting unused warnings. +var _ = NewGrantCoordinatorSQL diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 83bfc7bb1d3a..a56cf5585c8d 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -11,18 +11,13 @@ package admission import ( - "context" "time" - "unsafe" - "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble" @@ -938,872 +933,6 @@ func (sg *kvStoreTokenGranter) storeWriteDone( return additionalL0TokensNeeded } -// GrantCoordinator is the top-level object that coordinates grants across -// different WorkKinds (for more context see the comment in doc.go, and the -// comment where WorkKind is declared). Typically there will one -// GrantCoordinator in a node for CPU intensive work, and for nodes that also -// have the KV layer, one GrantCoordinator per store (these are managed by -// StoreGrantCoordinators) for KVWork that uses that store. See the -// NewGrantCoordinators and NewGrantCoordinatorSQL functions. -type GrantCoordinator struct { - ambientCtx log.AmbientContext - - settings *cluster.Settings - lastCPULoadSamplePeriod time.Duration - - // mu is ordered before any mutex acquired in a requester implementation. - // TODO(sumeer): move everything covered by mu into a nested struct. - mu syncutil.Mutex - // NB: Some granters can be nil. - granters [numWorkKinds]granterWithLockedCalls - // The WorkQueues behaving as requesters in each granterWithLockedCalls. - // This is kept separately only to service GetWorkQueue calls and to call - // close(). - queues [numWorkKinds]requesterClose - // The cpu fields can be nil, and the IO field can be nil, since a - // GrantCoordinator typically handles one of these two resources. - cpuOverloadIndicator cpuOverloadIndicator - cpuLoadListener CPULoadListener - ioLoadListener *ioLoadListener - - // The latest value of GOMAXPROCS, received via CPULoad. Only initialized if - // the cpu resource is being handled by this GrantCoordinator. - numProcs int - - // See the comment at continueGrantChain that explains how a grant chain - // functions and the motivation. When !useGrantChains, grant chains are - // disabled. - useGrantChains bool - - // The admission control code needs high sampling frequency of the cpu load, - // and turns off admission control enforcement when the sampling frequency - // is too low. For testing queueing behavior, we do not want the enforcement - // to be turned off in a non-deterministic manner so add a testing flag to - // disable that feature. - testingDisableSkipEnforcement bool - - // grantChainActive indicates whether a grant chain is active. If active, - // grantChainID is the ID of that chain. If !active, grantChainID is the ID - // of the next chain that will become active. IDs are assigned by - // incrementing grantChainID. If !useGrantChains, grantChainActive is never - // true. - grantChainActive bool - grantChainID grantChainID - // Index into granters, which represents the current WorkKind at which the - // grant chain is operating. Only relevant when grantChainActive is true. - grantChainIndex WorkKind - // See the comment at delayForGrantChainTermination for motivation. - grantChainStartTime time.Time -} - -var _ CPULoadListener = &GrantCoordinator{} - -// Options for constructing GrantCoordinators. -type Options struct { - MinCPUSlots int - MaxCPUSlots int - // RunnableAlphaOverride is used to override the alpha value used to - // compute the ewma of the runnable goroutine counts. It is only used - // during testing. A 0 value indicates that there is no override. - RunnableAlphaOverride float64 - SQLKVResponseBurstTokens int64 - SQLSQLResponseBurstTokens int64 - SQLStatementLeafStartWorkSlots int - SQLStatementRootStartWorkSlots int - TestingDisableSkipEnforcement bool - Settings *cluster.Settings - // Only non-nil for tests. - makeRequesterFunc makeRequesterFunc - makeStoreRequesterFunc makeStoreRequesterFunc -} - -var _ base.ModuleTestingKnobs = &Options{} - -// ModuleTestingKnobs implements the base.ModuleTestingKnobs interface. -func (*Options) ModuleTestingKnobs() {} - -// DefaultOptions are the default settings for various admission control knobs. -var DefaultOptions = Options{ - MinCPUSlots: 1, - MaxCPUSlots: 100000, /* TODO(sumeer): add cluster setting */ - SQLKVResponseBurstTokens: 100000, /* TODO(sumeer): add cluster setting */ - SQLSQLResponseBurstTokens: 100000, /* TODO(sumeer): add cluster setting */ - SQLStatementLeafStartWorkSlots: 100, /* arbitrary, and unused */ - SQLStatementRootStartWorkSlots: 100, /* arbitrary, and unused */ -} - -// Override applies values from "override" to the receiver that differ from Go -// defaults. -func (o *Options) Override(override *Options) { - if override.MinCPUSlots != 0 { - o.MinCPUSlots = override.MinCPUSlots - } - if override.MaxCPUSlots != 0 { - o.MaxCPUSlots = override.MaxCPUSlots - } - if override.SQLKVResponseBurstTokens != 0 { - o.SQLKVResponseBurstTokens = override.SQLKVResponseBurstTokens - } - if override.SQLSQLResponseBurstTokens != 0 { - o.SQLSQLResponseBurstTokens = override.SQLSQLResponseBurstTokens - } - if override.SQLStatementLeafStartWorkSlots != 0 { - o.SQLStatementLeafStartWorkSlots = override.SQLStatementLeafStartWorkSlots - } - if override.SQLStatementRootStartWorkSlots != 0 { - o.SQLStatementRootStartWorkSlots = override.SQLStatementRootStartWorkSlots - } - if override.TestingDisableSkipEnforcement { - o.TestingDisableSkipEnforcement = true - } -} - -type makeRequesterFunc func( - _ log.AmbientContext, workKind WorkKind, granter granter, settings *cluster.Settings, - opts workQueueOptions) requester - -type makeStoreRequesterFunc func( - _ log.AmbientContext, granters [numWorkClasses]granterWithStoreWriteDone, - settings *cluster.Settings, opts workQueueOptions) storeRequester - -// NewGrantCoordinators constructs GrantCoordinators and WorkQueues for a -// regular cluster node. Caller is responsible for hooking up -// GrantCoordinators.Regular to receive calls to CPULoad, and to set a -// PebbleMetricsProvider on GrantCoordinators.Stores. Every request must pass -// through GrantCoordinators.Regular, while only subsets of requests pass -// through each store's GrantCoordinator. We arrange these such that requests -// (that need to) first pass through a store's GrantCoordinator and then -// through the regular one. This ensures that we are not using slots in the -// latter on requests that are blocked elsewhere for admission. Additionally, -// we don't want the CPU scheduler signal that is implicitly used in grant -// chains to delay admission through the per store GrantCoordinators since -// they are not trying to control CPU usage, so we turn off grant chaining in -// those coordinators. -func NewGrantCoordinators( - ambientCtx log.AmbientContext, opts Options, -) (GrantCoordinators, []metric.Struct) { - makeRequester := makeWorkQueue - if opts.makeRequesterFunc != nil { - makeRequester = opts.makeRequesterFunc - } - st := opts.Settings - - metrics := makeGranterMetrics() - metricStructs := append([]metric.Struct(nil), metrics) - kvSlotAdjuster := &kvSlotAdjuster{ - settings: st, - minCPUSlots: opts.MinCPUSlots, - maxCPUSlots: opts.MaxCPUSlots, - totalSlotsMetric: metrics.KVTotalSlots, - totalModerateSlotsMetric: metrics.KVTotalModerateSlots, - moderateSlotsClamp: opts.MaxCPUSlots, - runnableAlphaOverride: opts.RunnableAlphaOverride, - } - coord := &GrantCoordinator{ - ambientCtx: ambientCtx, - settings: st, - cpuOverloadIndicator: kvSlotAdjuster, - cpuLoadListener: kvSlotAdjuster, - useGrantChains: true, - testingDisableSkipEnforcement: opts.TestingDisableSkipEnforcement, - numProcs: 1, - grantChainID: 1, - } - - kvg := &slotGranter{ - coord: coord, - workKind: KVWork, - totalHighLoadSlots: opts.MinCPUSlots, - totalModerateLoadSlots: opts.MinCPUSlots, - usedSlotsMetric: metrics.KVUsedSlots, - usedSoftSlotsMetric: metrics.KVUsedSoftSlots, - } - - kvSlotAdjuster.granter = kvg - req := makeRequester(ambientCtx, KVWork, kvg, st, makeWorkQueueOptions(KVWork)) - coord.queues[KVWork] = req - kvg.requester = req - coord.granters[KVWork] = kvg - - tg := &tokenGranter{ - coord: coord, - workKind: SQLKVResponseWork, - availableBurstTokens: opts.SQLKVResponseBurstTokens, - maxBurstTokens: opts.SQLKVResponseBurstTokens, - cpuOverload: kvSlotAdjuster, - } - req = makeRequester( - ambientCtx, SQLKVResponseWork, tg, st, makeWorkQueueOptions(SQLKVResponseWork)) - coord.queues[SQLKVResponseWork] = req - tg.requester = req - coord.granters[SQLKVResponseWork] = tg - - tg = &tokenGranter{ - coord: coord, - workKind: SQLSQLResponseWork, - availableBurstTokens: opts.SQLSQLResponseBurstTokens, - maxBurstTokens: opts.SQLSQLResponseBurstTokens, - cpuOverload: kvSlotAdjuster, - } - req = makeRequester(ambientCtx, - SQLSQLResponseWork, tg, st, makeWorkQueueOptions(SQLSQLResponseWork)) - coord.queues[SQLSQLResponseWork] = req - tg.requester = req - coord.granters[SQLSQLResponseWork] = tg - - sg := &slotGranter{ - coord: coord, - workKind: SQLStatementLeafStartWork, - totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, - cpuOverload: kvSlotAdjuster, - usedSlotsMetric: metrics.SQLLeafStartUsedSlots, - } - req = makeRequester(ambientCtx, - SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork)) - coord.queues[SQLStatementLeafStartWork] = req - sg.requester = req - coord.granters[SQLStatementLeafStartWork] = sg - - sg = &slotGranter{ - coord: coord, - workKind: SQLStatementRootStartWork, - totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, - cpuOverload: kvSlotAdjuster, - usedSlotsMetric: metrics.SQLRootStartUsedSlots, - } - req = makeRequester(ambientCtx, - SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork)) - coord.queues[SQLStatementRootStartWork] = req - sg.requester = req - coord.granters[SQLStatementRootStartWork] = sg - - metricStructs = appendMetricStructsForQueues(metricStructs, coord) - - // TODO(sumeerbhola): these metrics are shared across all stores and all - // priorities across stores (even the coarser workClasses, which are a - // mapping from priority, share the same metrics). Fix this by adding - // labeled Prometheus metrics. - storeWorkQueueMetrics := makeWorkQueueMetrics(string(workKindString(KVWork)) + "-stores") - metricStructs = append(metricStructs, storeWorkQueueMetrics) - makeStoreRequester := makeStoreWorkQueue - if opts.makeStoreRequesterFunc != nil { - makeStoreRequester = opts.makeStoreRequesterFunc - } - storeCoordinators := &StoreGrantCoordinators{ - settings: st, - makeStoreRequesterFunc: makeStoreRequester, - kvIOTokensExhaustedDuration: metrics.KVIOTokensExhaustedDuration, - workQueueMetrics: storeWorkQueueMetrics, - } - - return GrantCoordinators{Stores: storeCoordinators, Regular: coord}, metricStructs -} - -// NewGrantCoordinatorSQL constructs a GrantCoordinator and WorkQueues for a -// single-tenant SQL node in a multi-tenant cluster. Caller is responsible for -// hooking this up to receive calls to CPULoad. -func NewGrantCoordinatorSQL( - ambientCtx log.AmbientContext, opts Options, -) (*GrantCoordinator, []metric.Struct) { - makeRequester := makeWorkQueue - if opts.makeRequesterFunc != nil { - makeRequester = opts.makeRequesterFunc - } - st := opts.Settings - - metrics := makeGranterMetrics() - metricStructs := append([]metric.Struct(nil), metrics) - sqlNodeCPU := &sqlNodeCPUOverloadIndicator{} - coord := &GrantCoordinator{ - ambientCtx: ambientCtx, - settings: st, - cpuOverloadIndicator: sqlNodeCPU, - cpuLoadListener: sqlNodeCPU, - useGrantChains: true, - numProcs: 1, - grantChainID: 1, - } - - tg := &tokenGranter{ - coord: coord, - workKind: SQLKVResponseWork, - availableBurstTokens: opts.SQLKVResponseBurstTokens, - maxBurstTokens: opts.SQLKVResponseBurstTokens, - cpuOverload: sqlNodeCPU, - } - req := makeRequester(ambientCtx, - SQLKVResponseWork, tg, st, makeWorkQueueOptions(SQLKVResponseWork)) - coord.queues[SQLKVResponseWork] = req - tg.requester = req - coord.granters[SQLKVResponseWork] = tg - - tg = &tokenGranter{ - coord: coord, - workKind: SQLSQLResponseWork, - availableBurstTokens: opts.SQLSQLResponseBurstTokens, - maxBurstTokens: opts.SQLSQLResponseBurstTokens, - cpuOverload: sqlNodeCPU, - } - req = makeRequester(ambientCtx, - SQLSQLResponseWork, tg, st, makeWorkQueueOptions(SQLSQLResponseWork)) - coord.queues[SQLSQLResponseWork] = req - tg.requester = req - coord.granters[SQLSQLResponseWork] = tg - - sg := &slotGranter{ - coord: coord, - workKind: SQLStatementLeafStartWork, - totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, - cpuOverload: sqlNodeCPU, - usedSlotsMetric: metrics.SQLLeafStartUsedSlots, - } - req = makeRequester(ambientCtx, - SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork)) - coord.queues[SQLStatementLeafStartWork] = req - sg.requester = req - coord.granters[SQLStatementLeafStartWork] = sg - - sg = &slotGranter{ - coord: coord, - workKind: SQLStatementRootStartWork, - totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, - cpuOverload: sqlNodeCPU, - usedSlotsMetric: metrics.SQLRootStartUsedSlots, - } - req = makeRequester(ambientCtx, - SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork)) - coord.queues[SQLStatementRootStartWork] = req - sg.requester = req - coord.granters[SQLStatementRootStartWork] = sg - - return coord, appendMetricStructsForQueues(metricStructs, coord) -} - -func appendMetricStructsForQueues(ms []metric.Struct, coord *GrantCoordinator) []metric.Struct { - for i := range coord.queues { - if coord.queues[i] != nil { - q, ok := coord.queues[i].(*WorkQueue) - if ok { - ms = append(ms, q.metrics) - } - } - } - return ms -} - -// pebbleMetricsTick is called every adjustmentInterval seconds and passes -// through to the ioLoadListener, so that it can adjust the plan for future IO -// token allocations. -func (coord *GrantCoordinator) pebbleMetricsTick(ctx context.Context, m StoreMetrics) { - coord.ioLoadListener.pebbleMetricsTick(ctx, m) -} - -// allocateIOTokensTick tells the ioLoadListener to allocate tokens. -func (coord *GrantCoordinator) allocateIOTokensTick() { - coord.ioLoadListener.allocateTokensTick() - coord.mu.Lock() - defer coord.mu.Unlock() - if !coord.grantChainActive { - coord.tryGrant() - } - // Else, let the grant chain finish. NB: we turn off grant chains on the - // GrantCoordinators used for IO, so the if-condition is always true. -} - -// testingTryGrant is only for unit tests, since they sometimes cut out -// support classes like the ioLoadListener. -func (coord *GrantCoordinator) testingTryGrant() { - coord.mu.Lock() - defer coord.mu.Unlock() - if !coord.grantChainActive { - coord.tryGrant() - } -} - -// GetWorkQueue returns the WorkQueue for a particular WorkKind. Can be nil if -// the NewGrantCoordinator* function does not construct a WorkQueue for that -// work. -// Implementation detail: don't use this method when the GrantCoordinator is -// created by the StoreGrantCoordinators since those have a StoreWorkQueues. -// The TryGetQueueForStore is the external facing method in that case since -// the individual GrantCoordinators are hidden. -func (coord *GrantCoordinator) GetWorkQueue(workKind WorkKind) *WorkQueue { - return coord.queues[workKind].(*WorkQueue) -} - -// CPULoad implements CPULoadListener and is called periodically (see -// CPULoadListener for details). The same frequency is used for refilling the -// burst tokens since synchronizing the two means that the refilled burst can -// take into account the latest schedulers stats (indirectly, via the -// implementation of cpuOverloadIndicator). -func (coord *GrantCoordinator) CPULoad(runnable int, procs int, samplePeriod time.Duration) { - ctx := coord.ambientCtx.AnnotateCtx(context.Background()) - - if coord.lastCPULoadSamplePeriod != 0 && coord.lastCPULoadSamplePeriod != samplePeriod && - KVAdmissionControlEnabled.Get(&coord.settings.SV) { - log.Infof(ctx, "CPULoad switching to period %s", samplePeriod.String()) - } - coord.lastCPULoadSamplePeriod = samplePeriod - - coord.mu.Lock() - defer coord.mu.Unlock() - coord.numProcs = procs - coord.cpuLoadListener.CPULoad(runnable, procs, samplePeriod) - - // Slot adjustment and token refilling requires 1ms periods to work well. If - // the CPULoad ticks are less frequent, there is no guarantee that the - // tokens or slots will be sufficient to service requests. This is - // particularly the case for slots where we dynamically adjust them, and - // high contention can suddenly result in high slot utilization even while - // cpu utilization stays low. We don't want to artificially bottleneck - // request processing when we are in this slow CPULoad ticks regime since we - // can't adjust slots or refill tokens fast enough. So we explicitly tell - // the granters to not do token or slot enforcement. - skipEnforcement := samplePeriod > time.Millisecond - coord.granters[SQLKVResponseWork].(*tokenGranter).refillBurstTokens(skipEnforcement) - coord.granters[SQLSQLResponseWork].(*tokenGranter).refillBurstTokens(skipEnforcement) - if coord.granters[KVWork] != nil { - if !coord.testingDisableSkipEnforcement { - kvg := coord.granters[KVWork].(*slotGranter) - kvg.skipSlotEnforcement = skipEnforcement - } - } - if coord.grantChainActive && !coord.tryTerminateGrantChain() { - return - } - coord.tryGrant() -} - -// tryGet is called by granter.tryGet with the WorkKind. -func (coord *GrantCoordinator) tryGet(workKind WorkKind, count int64, demuxHandle int8) bool { - coord.mu.Lock() - defer coord.mu.Unlock() - // It is possible that a grant chain is active, and has not yet made its way - // to this workKind. So it may be more reasonable to queue. But we have some - // concerns about incurring the delay of multiple goroutine context switches - // so we ignore this case. - res := coord.granters[workKind].tryGetLocked(count, demuxHandle) - switch res { - case grantSuccess: - // Grant chain may be active, but it did not get in the way of this grant, - // and the effect of this grant in terms of overload will be felt by the - // grant chain. - return true - case grantFailDueToSharedResource: - // This could be a transient overload, that may not be noticed by the - // grant chain. We don't want it to continue granting to lower priority - // WorkKinds, while a higher priority one is waiting, so we terminate it. - if coord.grantChainActive && coord.grantChainIndex >= workKind { - coord.tryTerminateGrantChain() - } - return false - case grantFailLocal: - return false - default: - panic(errors.AssertionFailedf("unknown grantResult")) - } -} - -// returnGrant is called by granter.returnGrant with the WorkKind. -func (coord *GrantCoordinator) returnGrant(workKind WorkKind, count int64, demuxHandle int8) { - coord.mu.Lock() - defer coord.mu.Unlock() - coord.granters[workKind].returnGrantLocked(count, demuxHandle) - if coord.grantChainActive { - if coord.grantChainIndex > workKind && - coord.granters[workKind].requesterHasWaitingRequests() { - // There are waiting requests that will not be served by the grant chain. - // Better to terminate it and start afresh. - if !coord.tryTerminateGrantChain() { - return - } - } else { - // Else either the grant chain will get to this workKind, or there are no waiting requests. - return - } - } - coord.tryGrant() -} - -// tookWithoutPermission is called by granter.tookWithoutPermission with the -// WorkKind. -func (coord *GrantCoordinator) tookWithoutPermission( - workKind WorkKind, count int64, demuxHandle int8, -) { - coord.mu.Lock() - defer coord.mu.Unlock() - coord.granters[workKind].tookWithoutPermissionLocked(count, demuxHandle) -} - -// continueGrantChain is called by granter.continueGrantChain with the -// WorkKind. Never called if !coord.useGrantChains. -func (coord *GrantCoordinator) continueGrantChain(workKind WorkKind, grantChainID grantChainID) { - if grantChainID == noGrantChain { - return - } - coord.mu.Lock() - defer coord.mu.Unlock() - if coord.grantChainID != grantChainID { - // Someone terminated grantChainID by incrementing coord.grantChainID. - return - } - coord.tryGrant() -} - -// delayForGrantChainTermination causes a delay in terminating a grant chain. -// Terminating a grant chain immediately typically causes a new one to start -// immediately that can burst up to its maximum initial grant burst. Which -// means frequent terminations followed by new starts impose little control -// over the rate at which tokens are granted (slots are better controlled -// since we know when the work finishes). This causes huge spikes in the -// runnable goroutine count, observed at 1ms granularity. This spike causes -// the kvSlotAdjuster to ratchet down the totalSlots for KV work all the way -// down to 1, which later causes the runnable gorouting count to crash down -// to a value close to 0, leading to under-utilization. -// -// TODO(sumeer): design admission behavior metrics that can be used to -// understand the behavior in detail and to quantify improvements when changing -// heuristics. One metric would be mean and variance of the runnable count, -// computed using the 1ms samples, and exported/logged every 60s. -var delayForGrantChainTermination = 100 * time.Millisecond - -// tryTerminateGrantChain attempts to terminate the current grant chain, and -// returns true iff it is terminated, in which case a new one can be -// immediately started. -// REQUIRES: coord.grantChainActive==true -func (coord *GrantCoordinator) tryTerminateGrantChain() bool { - now := timeutil.Now() - if delayForGrantChainTermination > 0 && - now.Sub(coord.grantChainStartTime) < delayForGrantChainTermination { - return false - } - // Incrementing the ID will cause the existing grant chain to die out when - // the grantee calls continueGrantChain. - coord.grantChainID++ - coord.grantChainActive = false - coord.grantChainStartTime = time.Time{} - return true -} - -// tryGrant tries to either continue an existing grant chain, or if no grant -// chain is active, tries to start a new grant chain when grant chaining is -// enabled, or grants as much as it can when grant chaining is disabled. -func (coord *GrantCoordinator) tryGrant() { - startingChain := false - if !coord.grantChainActive { - // NB: always set to true when !coord.useGrantChains, and we won't - // actually use this to start a grant chain (see below). - startingChain = true - coord.grantChainIndex = 0 - } - // Assume that we will not be able to start a new grant chain, or that the - // existing one will die out. The code below will set it to true if neither - // is true. - coord.grantChainActive = false - grantBurstCount := 0 - // Grant in a burst proportional to numProcs, to generate a runnable for - // each. - grantBurstLimit := coord.numProcs - // Additionally, increase the burst size proportional to a fourth of the - // overload threshold. We experimentally observed that this resulted in - // better CPU utilization. We don't use the full overload threshold since we - // don't want to over grant for non-KV work since that causes the KV slots - // to (unfairly) start decreasing, since we lose control over how many - // goroutines are runnable. - multiplier := int(KVSlotAdjusterOverloadThreshold.Get(&coord.settings.SV) / 4) - if multiplier == 0 { - multiplier = 1 - } - grantBurstLimit *= multiplier - // Only the case of a grant chain being active returns from within the - // OuterLoop. -OuterLoop: - for ; coord.grantChainIndex < numWorkKinds; coord.grantChainIndex++ { - localDone := false - granter := coord.granters[coord.grantChainIndex] - if granter == nil { - // A GrantCoordinator can be limited to certain WorkKinds, and the - // remaining will be nil. - continue - } - for granter.requesterHasWaitingRequests() && !localDone { - chainID := noGrantChain - if grantBurstCount+1 == grantBurstLimit && coord.useGrantChains { - chainID = coord.grantChainID - } - res := granter.tryGrantLocked(chainID) - switch res { - case grantSuccess: - grantBurstCount++ - if grantBurstCount == grantBurstLimit && coord.useGrantChains { - coord.grantChainActive = true - if startingChain { - coord.grantChainStartTime = timeutil.Now() - } - return - } - case grantFailDueToSharedResource: - break OuterLoop - case grantFailLocal: - localDone = true - default: - panic(errors.AssertionFailedf("unknown grantResult")) - } - } - } - // INVARIANT: !grantChainActive. The chain either did not start or the - // existing one died. If the existing one died, we increment grantChainID - // since it represents the ID to be used for the next chain. Note that - // startingChain is always true when !useGrantChains, so this if-block is - // not executed. - if !startingChain { - coord.grantChainID++ - } -} - -// Close implements the stop.Closer interface. -func (coord *GrantCoordinator) Close() { - for i := range coord.queues { - if coord.queues[i] != nil { - coord.queues[i].close() - } - } -} - -func (coord *GrantCoordinator) String() string { - return redact.StringWithoutMarkers(coord) -} - -// SafeFormat implements the redact.SafeFormatter interface. -func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) { - coord.mu.Lock() - defer coord.mu.Unlock() - s.Printf("(chain: id: %d active: %t index: %d)", - coord.grantChainID, coord.grantChainActive, coord.grantChainIndex, - ) - - spaceStr := redact.RedactableString(" ") - newlineStr := redact.RedactableString("\n") - curSep := spaceStr - for i := range coord.granters { - kind := WorkKind(i) - switch kind { - case KVWork: - switch g := coord.granters[i].(type) { - case *slotGranter: - kvsa := coord.cpuLoadListener.(*kvSlotAdjuster) - s.Printf( - "%s%s: used: %d, high(moderate)-total: %d(%d) moderate-clamp: %d", curSep, workKindString(kind), - g.usedSlots, g.totalHighLoadSlots, g.totalModerateLoadSlots, kvsa.moderateSlotsClamp) - if g.usedSoftSlots > 0 { - s.Printf(" used-soft: %d", g.usedSoftSlots) - } - case *kvStoreTokenGranter: - s.Printf(" io-avail: %d, elastic-disk-bw-tokens-avail: %d", g.availableIOTokens, - g.elasticDiskBWTokensAvailable) - } - case SQLStatementLeafStartWork, SQLStatementRootStartWork: - if coord.granters[i] != nil { - g := coord.granters[i].(*slotGranter) - s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalHighLoadSlots) - } - case SQLKVResponseWork, SQLSQLResponseWork: - if coord.granters[i] != nil { - g := coord.granters[i].(*tokenGranter) - s.Printf("%s%s: avail: %d", curSep, workKindString(kind), g.availableBurstTokens) - if kind == SQLKVResponseWork { - curSep = newlineStr - } else { - curSep = spaceStr - } - } - } - } -} - -// StoreGrantCoordinators is a container for GrantCoordinators for each store, -// that is used for KV work admission that takes into account store health. -// Currently it is intended only for writes to stores. -type StoreGrantCoordinators struct { - ambientCtx log.AmbientContext - - settings *cluster.Settings - makeStoreRequesterFunc makeStoreRequesterFunc - kvIOTokensExhaustedDuration *metric.Counter - // These metrics are shared by WorkQueues across stores. - workQueueMetrics WorkQueueMetrics - - gcMap syncutil.IntMap // map[int64(StoreID)]*GrantCoordinator - // numStores is used to track the number of stores which have been added - // to the gcMap. This is used because the IntMap doesn't expose a size - // api. - numStores int - pebbleMetricsProvider PebbleMetricsProvider - closeCh chan struct{} - - disableTickerForTesting bool -} - -// SetPebbleMetricsProvider sets a PebbleMetricsProvider and causes the load -// on the various storage engines to be used for admission control. -func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider( - startupCtx context.Context, pmp PebbleMetricsProvider, iotc IOThresholdConsumer, -) { - if sgc.pebbleMetricsProvider != nil { - panic(errors.AssertionFailedf("SetPebbleMetricsProvider called more than once")) - } - sgc.pebbleMetricsProvider = pmp - sgc.closeCh = make(chan struct{}) - metrics := sgc.pebbleMetricsProvider.GetPebbleMetrics() - for _, m := range metrics { - gc := sgc.initGrantCoordinator(m.StoreID) - // Defensive call to LoadAndStore even though Store ought to be sufficient - // since SetPebbleMetricsProvider can only be called once. This code - // guards against duplication of stores returned by GetPebbleMetrics. - _, loaded := sgc.gcMap.LoadOrStore(int64(m.StoreID), unsafe.Pointer(gc)) - if !loaded { - sgc.numStores++ - } - gc.pebbleMetricsTick(startupCtx, m) - gc.allocateIOTokensTick() - } - if sgc.disableTickerForTesting { - return - } - // Attach tracer and log tags. - ctx := sgc.ambientCtx.AnnotateCtx(context.Background()) - - go func() { - var ticks int64 - ticker := time.NewTicker(ioTokenTickDuration) - done := false - for !done { - select { - case <-ticker.C: - ticks++ - if ticks%ticksInAdjustmentInterval == 0 { - metrics := sgc.pebbleMetricsProvider.GetPebbleMetrics() - if len(metrics) != sgc.numStores { - log.Warningf(ctx, - "expected %d store metrics and found %d metrics", sgc.numStores, len(metrics)) - } - for _, m := range metrics { - if unsafeGc, ok := sgc.gcMap.Load(int64(m.StoreID)); ok { - gc := (*GrantCoordinator)(unsafeGc) - gc.pebbleMetricsTick(ctx, m) - iotc.UpdateIOThreshold(roachpb.StoreID(m.StoreID), gc.ioLoadListener.ioThreshold) - } else { - log.Warningf(ctx, - "seeing metrics for unknown storeID %d", m.StoreID) - } - } - } - sgc.gcMap.Range(func(_ int64, unsafeGc unsafe.Pointer) bool { - gc := (*GrantCoordinator)(unsafeGc) - gc.allocateIOTokensTick() - // true indicates that iteration should continue after the - // current entry has been processed. - return true - }) - case <-sgc.closeCh: - done = true - } - } - ticker.Stop() - }() -} - -func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoordinator { - coord := &GrantCoordinator{ - settings: sgc.settings, - useGrantChains: false, - numProcs: 1, - } - - kvg := &kvStoreTokenGranter{ - coord: coord, - // Setting tokens to unlimited is defensive. We expect that - // pebbleMetricsTick and allocateIOTokensTick will get called during - // initialization, which will also set these to unlimited. - availableIOTokens: unlimitedTokens / ticksInAdjustmentInterval, - startingIOTokens: unlimitedTokens / ticksInAdjustmentInterval, - ioTokensExhaustedDurationMetric: sgc.kvIOTokensExhaustedDuration, - elasticDiskBWTokensAvailable: unlimitedTokens / ticksInAdjustmentInterval, - } - opts := makeWorkQueueOptions(KVWork) - // This is IO work, so override the usesTokens value. - opts.usesTokens = true - // Share the WorkQueue metrics across all stores. - // TODO(sumeer): add per-store WorkQueue state for debug.zip and db console. - opts.metrics = &sgc.workQueueMetrics - granters := [numWorkClasses]granterWithStoreWriteDone{ - &kvStoreTokenChildGranter{ - workClass: regularWorkClass, - parent: kvg, - }, - &kvStoreTokenChildGranter{ - workClass: elasticWorkClass, - parent: kvg, - }, - } - - storeReq := sgc.makeStoreRequesterFunc(sgc.ambientCtx, granters, sgc.settings, opts) - coord.queues[KVWork] = storeReq - requesters := storeReq.getRequesters() - kvg.regularRequester = requesters[regularWorkClass] - kvg.elasticRequester = requesters[elasticWorkClass] - coord.granters[KVWork] = kvg - coord.ioLoadListener = &ioLoadListener{ - storeID: storeID, - settings: sgc.settings, - kvRequester: storeReq, - perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), - diskBandwidthLimiter: makeDiskBandwidthLimiter(), - } - coord.ioLoadListener.mu.Mutex = &coord.mu - coord.ioLoadListener.mu.kvGranter = kvg - return coord -} - -// TryGetQueueForStore returns a WorkQueue for the given storeID, or nil if -// the storeID is not known. -func (sgc *StoreGrantCoordinators) TryGetQueueForStore(storeID int32) *StoreWorkQueue { - if unsafeGranter, ok := sgc.gcMap.Load(int64(storeID)); ok { - granter := (*GrantCoordinator)(unsafeGranter) - return granter.queues[KVWork].(*StoreWorkQueue) - } - return nil -} - -func (sgc *StoreGrantCoordinators) close() { - // closeCh can be nil in tests that never called SetPebbleMetricsProvider. - if sgc.closeCh != nil { - close(sgc.closeCh) - } - - sgc.gcMap.Range(func(_ int64, unsafeGc unsafe.Pointer) bool { - gc := (*GrantCoordinator)(unsafeGc) - gc.Close() - // true indicates that iteration should continue after the - // current entry has been processed. - return true - }) -} - -// GrantCoordinators holds a regular GrantCoordinator for all work, and a -// StoreGrantCoordinators that allows for per-store GrantCoordinators for -// KVWork that involves writes. -type GrantCoordinators struct { - Stores *StoreGrantCoordinators - Regular *GrantCoordinator -} - -// Close implements the stop.Closer interface. -func (gcs GrantCoordinators) Close() { - gcs.Stores.close() - gcs.Regular.Close() -} - // cpuOverloadIndicator is meant to be an instantaneous indicator of cpu // availability. Since actual scheduler stats are periodic, we prefer to use // the KV slot availability, since it is instantaneous. The @@ -2205,38 +1334,6 @@ var ( // embedded in kvserver.StoreMetrics. We should also separate the metrics // related to cpu slots from the IO metrics. -// GranterMetrics are metrics associated with a GrantCoordinator. -type GranterMetrics struct { - KVTotalSlots *metric.Gauge - KVUsedSlots *metric.Gauge - KVTotalModerateSlots *metric.Gauge - KVUsedSoftSlots *metric.Gauge - KVIOTokensExhaustedDuration *metric.Counter - SQLLeafStartUsedSlots *metric.Gauge - SQLRootStartUsedSlots *metric.Gauge -} - -// MetricStruct implements the metric.Struct interface. -func (GranterMetrics) MetricStruct() {} - -func makeGranterMetrics() GranterMetrics { - m := GranterMetrics{ - KVTotalSlots: metric.NewGauge(totalSlots), - KVUsedSlots: metric.NewGauge(addName(string(workKindString(KVWork)), usedSlots)), - KVTotalModerateSlots: metric.NewGauge(totalModerateSlots), - KVUsedSoftSlots: metric.NewGauge(usedSoftSlots), - KVIOTokensExhaustedDuration: metric.NewCounter(kvIOTokensExhaustedDuration), - SQLLeafStartUsedSlots: metric.NewGauge( - addName(string(workKindString(SQLStatementLeafStartWork)), usedSlots)), - SQLRootStartUsedSlots: metric.NewGauge( - addName(string(workKindString(SQLStatementRootStartWork)), usedSlots)), - } - return m -} - -// Prevent the linter from emitting unused warnings. -var _ = NewGrantCoordinatorSQL - // TODO(sumeer): experiment with approaches to adjust slots for // SQLStatementLeafStartWork and SQLStatementRootStartWork for SQL nodes. Note // that for these WorkKinds we are currently setting very high slot counts diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index 4c5a607412ed..cd41ac5450fe 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -207,7 +207,7 @@ func TestGranterBasic(t *testing.T) { case "init-store-grant-coordinator": clearRequesterAndCoord() - metrics := makeGranterMetrics() + metrics := makeGrantCoordinatorMetrics() storeCoordinators := &StoreGrantCoordinators{ settings: settings, makeStoreRequesterFunc: func( From af9c69c17232cccfb19452956045da23ea18e5ad Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 12 Aug 2022 17:35:20 -0400 Subject: [PATCH 3/9] admission: move kvSlotAdjuster into its own file Pure code movement. We're hurting some git history tracking with these code changes, but hopefully they lead to more understandability with future work. Release note: None --- pkg/util/admission/BUILD.bazel | 1 + pkg/util/admission/granter.go | 215 ++----------------------- pkg/util/admission/kv_slot_adjuster.go | 207 ++++++++++++++++++++++++ 3 files changed, 221 insertions(+), 202 deletions(-) create mode 100644 pkg/util/admission/kv_slot_adjuster.go diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index eb16b558569c..82a6af64654c 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "grant_coordinator.go", "granter.go", "io_load_listener.go", + "kv_slot_adjuster.go", "store_token_estimation.go", "work_queue.go", ], diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index a56cf5585c8d..0a06b0b52dda 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -24,16 +23,6 @@ import ( "github.com/cockroachdb/redact" ) -// KVSlotAdjusterOverloadThreshold sets a goroutine runnable threshold at -// which the CPU will be considered overloaded, when running in a node that -// executes KV operations. -var KVSlotAdjusterOverloadThreshold = settings.RegisterIntSetting( - settings.TenantWritable, - "admission.kv_slot_adjuster.overload_threshold", - "when the number of runnable goroutines per CPU is greater than this threshold, the "+ - "slot adjuster considers the cpu to be overloaded", - 32, settings.PositiveInt) - // EnabledSoftSlotGranting can be set to false to disable soft slot granting. var EnabledSoftSlotGranting = settings.RegisterBoolSetting( settings.TenantWritable, @@ -957,197 +946,6 @@ type CPULoadListener interface { CPULoad(runnable int, procs int, samplePeriod time.Duration) } -// kvSlotAdjuster is an implementer of CPULoadListener and -// cpuOverloadIndicator. -type kvSlotAdjuster struct { - settings *cluster.Settings - // This is the slotGranter used for KVWork. In single-tenant settings, it - // is the only one we adjust using the periodic cpu overload signal. We - // don't adjust slots for SQLStatementLeafStartWork and - // SQLStatementRootStartWork using the periodic cpu overload signal since: - // - these are potentially long-lived work items and not CPU bound - // - we don't know how to coordinate adjustment of those slots and the KV - // slots. - granter *slotGranter - minCPUSlots int - maxCPUSlots int - // moderateSlotsClamp is the most recent value which may have been used to - // clamp down on slotGranter.totalModerateLoadSlots. Justification for - // clamping down on totalModerateLoadSlots is given where the moderateSlotsClamp - // value is written to. - moderateSlotsClamp int - // moderateSlotsClampOverride is used during testing to override the value of the - // moderateSlotsClamp. Its purpose is to make it easier to write tests. A default - // value of 0 implies no override. - moderateSlotsClampOverride int - // runnableEWMA is a weighted average of the most recent runnable goroutine counts. - // runnableEWMA is used to tune the slotGranter.totalModerateLoadSlots. - runnableEWMA float64 - // runnableAlphaOverride is used to override the value of runnable alpha during testing. - // A 0 value indicates that there is no override. - runnableAlphaOverride float64 - - totalSlotsMetric *metric.Gauge - totalModerateSlotsMetric *metric.Gauge -} - -var _ cpuOverloadIndicator = &kvSlotAdjuster{} -var _ CPULoadListener = &kvSlotAdjuster{} - -func (kvsa *kvSlotAdjuster) CPULoad(runnable int, procs int, samplePeriod time.Duration) { - threshold := int(KVSlotAdjusterOverloadThreshold.Get(&kvsa.settings.SV)) - - // 0.009 gives weight to at least a few hundred samples at a 1ms sampling rate. - alpha := 0.009 * float64(samplePeriod/time.Millisecond) - if alpha > 0.5 { - alpha = 0.5 - } else if alpha < 0.001 { - alpha = 0.001 - } - if kvsa.runnableAlphaOverride > 0 { - alpha = kvsa.runnableAlphaOverride - } - kvsa.runnableEWMA = kvsa.runnableEWMA*(1-alpha) + float64(runnable)*alpha - - // Simple heuristic, which worked ok in experiments. More sophisticated ones - // could be devised. - usedSlots := kvsa.granter.usedSlots + kvsa.granter.usedSoftSlots - tryDecreaseSlots := func(total int) int { - // Overload. - // If using some slots, and the used slots is less than the total slots, - // and total slots hasn't bottomed out at the min, decrease the total - // slots. If currently using more than the total slots, it suggests that - // the previous slot reduction has not taken effect yet, so we hold off on - // further decreasing. - // TODO(sumeer): despite the additive decrease and high multiplier value, - // the metric showed some drops from 40 slots to 1 slot on a kv50 overload - // workload. It was not accompanied by a drop in runnable count per proc, - // so it is suggests that the drop in slots should not be causing cpu - // under-utilization, but one cannot be sure. Experiment with a smoothed - // signal or other ways to prevent a fast drop. - if usedSlots > 0 && total > kvsa.minCPUSlots && usedSlots <= total { - total-- - } - return total - } - tryIncreaseSlots := func(total int) int { - // Underload. - // Used all its slots and can increase further, so additive increase. We - // also handle the case where the used slots are a bit less than total - // slots, since callers for soft slots don't block. - if usedSlots >= total && total < kvsa.maxCPUSlots { - // NB: If the workload is IO bound, the slot count here will keep - // incrementing until these slots are no longer the bottleneck for - // admission. So it is not unreasonable to see this slot count go into - // the 1000s. If the workload switches to being CPU bound, we can - // decrease by 1000 slots every second (because the CPULoad ticks are at - // 1ms intervals, and we do additive decrease). - total++ - } - return total - } - - if runnable >= threshold*procs { - // Very overloaded. - kvsa.granter.totalHighLoadSlots = tryDecreaseSlots(kvsa.granter.totalHighLoadSlots) - kvsa.granter.totalModerateLoadSlots = tryDecreaseSlots(kvsa.granter.totalModerateLoadSlots) - } else if float64(runnable) <= float64((threshold*procs)/4) { - // Very underloaded. - kvsa.granter.totalHighLoadSlots = tryIncreaseSlots(kvsa.granter.totalHighLoadSlots) - kvsa.granter.totalModerateLoadSlots = tryIncreaseSlots(kvsa.granter.totalModerateLoadSlots) - } else if float64(runnable) <= float64((threshold*procs)/2) { - // Moderately underloaded -- can afford to increase regular slots. - kvsa.granter.totalHighLoadSlots = tryIncreaseSlots(kvsa.granter.totalHighLoadSlots) - } else if runnable >= 3*threshold*procs/4 { - // Moderately overloaded -- should decrease moderate load slots. - // - // NB: decreasing moderate load slots may not halt the runnable growth - // since the regular traffic may be high and can use up to the high load - // slots. When usedSlots>totalModerateLoadSlots, we won't actually - // decrease totalModerateLoadSlots (see the logic in tryDecreaseSlots). - // However, that doesn't mean that totalModerateLoadSlots is accurate. - // This inaccuracy is fine since we have chosen to be in a high load - // regime, since all the work we are doing is non-optional regular work - // (not background work). - // - // Where this will help is when what is pushing us over moderate load is - // optional background work, so by decreasing totalModerateLoadSlots we will - // contain the load due to that work. - kvsa.granter.totalModerateLoadSlots = tryDecreaseSlots(kvsa.granter.totalModerateLoadSlots) - } - // Consider the following cases, when we started this method with - // totalHighLoadSlots==totalModerateLoadSlots. - // - underload such that we are able to increase totalModerateLoadSlots: in - // this case we will also be able to increase totalHighLoadSlots (since - // the used and total comparisons gating the increase in tryIncreaseSlots - // will also be true for totalHighLoadSlots). - // - overload such that we are able to decrease totalHighLoadSlots: in this - // case the logic in tryDecreaseSlots will also be able to decrease - // totalModerateLoadSlots. - // So the natural behavior of the slot adjustment itself guarantees - // totalHighLoadSlots >= totalModerateLoadSlots. But as a defensive measure - // we clamp totalModerateLoadSlots to not exceed totalHighLoadSlots. - if kvsa.granter.totalHighLoadSlots < kvsa.granter.totalModerateLoadSlots { - kvsa.granter.totalModerateLoadSlots = kvsa.granter.totalHighLoadSlots - } - - // During a kv50 workload, we noticed soft slots grants succeeding despite - // high cpu utilization, and high runnable goroutine counts. - // - // Consider the following log lines from the kv50 experiment: - // [runnable count 372 threshold*procs 256] - // [totalHighLoadSlots 254 totalModerateLoadSlots 164 usedSlots 0 usedSoftSlots 1] - // - // Note that even though the runnable count is high, of the (254, 164), - // (totalHighLoad, totalModerateLoad) slots respectively, only 1 slot is - // being used. The slot mechanism behaves in a bi-modal manner in nodes that - // do both KV and SQL processing. While there is backlogged KV work, the slot - // usage is high, and blocks all SQL work, but eventually all callers have done - // their KV processing and are queued up for SQL work. The latter causes bursts - // of grants (because it uses tokens), gated only by the grant-chain mechanism, - // during which runnable count is high but used (KV) slots are low. This is exactly - // the case where we have low slot usage, but high CPU utilization. - // - // We can afford to be more conservative in calculating totalModerateLoadSlots - // since we don't care about saturating CPU for the less important work that is - // controlled by these slots. So we could use a slow reacting and conservative - // signal to decide on the value of totalModerateLoadSlots. - // - // To account for the increased CPU utilization and runnable counts when the used - // slots are low, we clamp down on the totalModerateSlots value by keeping track - // of a historical runnable goroutine average. - kvsa.moderateSlotsClamp = int(float64(threshold*procs)/2 - kvsa.runnableEWMA) - if kvsa.moderateSlotsClampOverride != 0 { - kvsa.moderateSlotsClamp = kvsa.moderateSlotsClampOverride - } - if kvsa.granter.totalModerateLoadSlots > kvsa.moderateSlotsClamp { - kvsa.granter.totalModerateLoadSlots = kvsa.moderateSlotsClamp - } - if kvsa.granter.totalModerateLoadSlots < 0 { - kvsa.granter.totalModerateLoadSlots = 0 - } - - kvsa.totalSlotsMetric.Update(int64(kvsa.granter.totalHighLoadSlots)) - kvsa.totalModerateSlotsMetric.Update(int64(kvsa.granter.totalModerateLoadSlots)) -} - -func (kvsa *kvSlotAdjuster) isOverloaded() bool { - return kvsa.granter.usedSlots >= kvsa.granter.totalHighLoadSlots && !kvsa.granter.skipSlotEnforcement -} - -// sqlNodeCPUOverloadIndicator is the implementation of cpuOverloadIndicator -// for a single-tenant SQL node in a multi-tenant cluster. This has to rely on -// the periodic load information from the cpu scheduler and will therefore be -// tuned towards indicating overload at higher overload points (otherwise we -// could fluctuate into underloaded territory due to restricting admission, -// and not be work conserving). Such tuning towards more overload, and -// therefore more queueing inside the scheduler, is somewhat acceptable since -// a SQL node is not multi-tenant. -// -// TODO(sumeer): implement. -type sqlNodeCPUOverloadIndicator struct { -} - // PebbleMetricsProvider provides the pebble.Metrics for all stores. type PebbleMetricsProvider interface { GetPebbleMetrics() []StoreMetrics @@ -1279,6 +1077,19 @@ type storeRequester interface { setStoreRequestEstimates(estimates storeRequestEstimates) } +// sqlNodeCPUOverloadIndicator is the implementation of cpuOverloadIndicator +// for a single-tenant SQL node in a multi-tenant cluster. This has to rely on +// the periodic load information from the cpu scheduler and will therefore be +// tuned towards indicating overload at higher overload points (otherwise we +// could fluctuate into underloaded territory due to restricting admission, +// and not be work conserving). Such tuning towards more overload, and +// therefore more queueing inside the scheduler, is somewhat acceptable since +// a SQL node is not multi-tenant. +// +// TODO(sumeer): implement. +type sqlNodeCPUOverloadIndicator struct { +} + var _ cpuOverloadIndicator = &sqlNodeCPUOverloadIndicator{} var _ CPULoadListener = &sqlNodeCPUOverloadIndicator{} diff --git a/pkg/util/admission/kv_slot_adjuster.go b/pkg/util/admission/kv_slot_adjuster.go new file mode 100644 index 000000000000..166ba0e46437 --- /dev/null +++ b/pkg/util/admission/kv_slot_adjuster.go @@ -0,0 +1,207 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package admission + +import ( + "time" + + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/metric" +) + +// KVSlotAdjusterOverloadThreshold sets a goroutine runnable threshold at +// which the CPU will be considered overloaded, when running in a node that +// executes KV operations. +var KVSlotAdjusterOverloadThreshold = settings.RegisterIntSetting( + settings.TenantWritable, + "admission.kv_slot_adjuster.overload_threshold", + "when the number of runnable goroutines per CPU is greater than this threshold, the "+ + "slot adjuster considers the cpu to be overloaded", + 32, settings.PositiveInt) + +// kvSlotAdjuster is an implementer of CPULoadListener and +// cpuOverloadIndicator. +type kvSlotAdjuster struct { + settings *cluster.Settings + // This is the slotGranter used for KVWork. In single-tenant settings, it + // is the only one we adjust using the periodic cpu overload signal. We + // don't adjust slots for SQLStatementLeafStartWork and + // SQLStatementRootStartWork using the periodic cpu overload signal since: + // - these are potentially long-lived work items and not CPU bound + // - we don't know how to coordinate adjustment of those slots and the KV + // slots. + granter *slotGranter + minCPUSlots int + maxCPUSlots int + // moderateSlotsClamp is the most recent value which may have been used to + // clamp down on slotGranter.totalModerateLoadSlots. Justification for + // clamping down on totalModerateLoadSlots is given where the moderateSlotsClamp + // value is written to. + moderateSlotsClamp int + // moderateSlotsClampOverride is used during testing to override the value of the + // moderateSlotsClamp. Its purpose is to make it easier to write tests. A default + // value of 0 implies no override. + moderateSlotsClampOverride int + // runnableEWMA is a weighted average of the most recent runnable goroutine counts. + // runnableEWMA is used to tune the slotGranter.totalModerateLoadSlots. + runnableEWMA float64 + // runnableAlphaOverride is used to override the value of runnable alpha during testing. + // A 0 value indicates that there is no override. + runnableAlphaOverride float64 + + totalSlotsMetric *metric.Gauge + totalModerateSlotsMetric *metric.Gauge +} + +var _ cpuOverloadIndicator = &kvSlotAdjuster{} +var _ CPULoadListener = &kvSlotAdjuster{} + +func (kvsa *kvSlotAdjuster) CPULoad(runnable int, procs int, samplePeriod time.Duration) { + threshold := int(KVSlotAdjusterOverloadThreshold.Get(&kvsa.settings.SV)) + + // 0.009 gives weight to at least a few hundred samples at a 1ms sampling rate. + alpha := 0.009 * float64(samplePeriod/time.Millisecond) + if alpha > 0.5 { + alpha = 0.5 + } else if alpha < 0.001 { + alpha = 0.001 + } + if kvsa.runnableAlphaOverride > 0 { + alpha = kvsa.runnableAlphaOverride + } + kvsa.runnableEWMA = kvsa.runnableEWMA*(1-alpha) + float64(runnable)*alpha + + // Simple heuristic, which worked ok in experiments. More sophisticated ones + // could be devised. + usedSlots := kvsa.granter.usedSlots + kvsa.granter.usedSoftSlots + tryDecreaseSlots := func(total int) int { + // Overload. + // If using some slots, and the used slots is less than the total slots, + // and total slots hasn't bottomed out at the min, decrease the total + // slots. If currently using more than the total slots, it suggests that + // the previous slot reduction has not taken effect yet, so we hold off on + // further decreasing. + // TODO(sumeer): despite the additive decrease and high multiplier value, + // the metric showed some drops from 40 slots to 1 slot on a kv50 overload + // workload. It was not accompanied by a drop in runnable count per proc, + // so it is suggests that the drop in slots should not be causing cpu + // under-utilization, but one cannot be sure. Experiment with a smoothed + // signal or other ways to prevent a fast drop. + if usedSlots > 0 && total > kvsa.minCPUSlots && usedSlots <= total { + total-- + } + return total + } + tryIncreaseSlots := func(total int) int { + // Underload. + // Used all its slots and can increase further, so additive increase. We + // also handle the case where the used slots are a bit less than total + // slots, since callers for soft slots don't block. + if usedSlots >= total && total < kvsa.maxCPUSlots { + // NB: If the workload is IO bound, the slot count here will keep + // incrementing until these slots are no longer the bottleneck for + // admission. So it is not unreasonable to see this slot count go into + // the 1000s. If the workload switches to being CPU bound, we can + // decrease by 1000 slots every second (because the CPULoad ticks are at + // 1ms intervals, and we do additive decrease). + total++ + } + return total + } + + if runnable >= threshold*procs { + // Very overloaded. + kvsa.granter.totalHighLoadSlots = tryDecreaseSlots(kvsa.granter.totalHighLoadSlots) + kvsa.granter.totalModerateLoadSlots = tryDecreaseSlots(kvsa.granter.totalModerateLoadSlots) + } else if float64(runnable) <= float64((threshold*procs)/4) { + // Very underloaded. + kvsa.granter.totalHighLoadSlots = tryIncreaseSlots(kvsa.granter.totalHighLoadSlots) + kvsa.granter.totalModerateLoadSlots = tryIncreaseSlots(kvsa.granter.totalModerateLoadSlots) + } else if float64(runnable) <= float64((threshold*procs)/2) { + // Moderately underloaded -- can afford to increase regular slots. + kvsa.granter.totalHighLoadSlots = tryIncreaseSlots(kvsa.granter.totalHighLoadSlots) + } else if runnable >= 3*threshold*procs/4 { + // Moderately overloaded -- should decrease moderate load slots. + // + // NB: decreasing moderate load slots may not halt the runnable growth + // since the regular traffic may be high and can use up to the high load + // slots. When usedSlots>totalModerateLoadSlots, we won't actually + // decrease totalModerateLoadSlots (see the logic in tryDecreaseSlots). + // However, that doesn't mean that totalModerateLoadSlots is accurate. + // This inaccuracy is fine since we have chosen to be in a high load + // regime, since all the work we are doing is non-optional regular work + // (not background work). + // + // Where this will help is when what is pushing us over moderate load is + // optional background work, so by decreasing totalModerateLoadSlots we will + // contain the load due to that work. + kvsa.granter.totalModerateLoadSlots = tryDecreaseSlots(kvsa.granter.totalModerateLoadSlots) + } + // Consider the following cases, when we started this method with + // totalHighLoadSlots==totalModerateLoadSlots. + // - underload such that we are able to increase totalModerateLoadSlots: in + // this case we will also be able to increase totalHighLoadSlots (since + // the used and total comparisons gating the increase in tryIncreaseSlots + // will also be true for totalHighLoadSlots). + // - overload such that we are able to decrease totalHighLoadSlots: in this + // case the logic in tryDecreaseSlots will also be able to decrease + // totalModerateLoadSlots. + // So the natural behavior of the slot adjustment itself guarantees + // totalHighLoadSlots >= totalModerateLoadSlots. But as a defensive measure + // we clamp totalModerateLoadSlots to not exceed totalHighLoadSlots. + if kvsa.granter.totalHighLoadSlots < kvsa.granter.totalModerateLoadSlots { + kvsa.granter.totalModerateLoadSlots = kvsa.granter.totalHighLoadSlots + } + + // During a kv50 workload, we noticed soft slots grants succeeding despite + // high cpu utilization, and high runnable goroutine counts. + // + // Consider the following log lines from the kv50 experiment: + // [runnable count 372 threshold*procs 256] + // [totalHighLoadSlots 254 totalModerateLoadSlots 164 usedSlots 0 usedSoftSlots 1] + // + // Note that even though the runnable count is high, of the (254, 164), + // (totalHighLoad, totalModerateLoad) slots respectively, only 1 slot is + // being used. The slot mechanism behaves in a bi-modal manner in nodes that + // do both KV and SQL processing. While there is backlogged KV work, the slot + // usage is high, and blocks all SQL work, but eventually all callers have done + // their KV processing and are queued up for SQL work. The latter causes bursts + // of grants (because it uses tokens), gated only by the grant-chain mechanism, + // during which runnable count is high but used (KV) slots are low. This is exactly + // the case where we have low slot usage, but high CPU utilization. + // + // We can afford to be more conservative in calculating totalModerateLoadSlots + // since we don't care about saturating CPU for the less important work that is + // controlled by these slots. So we could use a slow reacting and conservative + // signal to decide on the value of totalModerateLoadSlots. + // + // To account for the increased CPU utilization and runnable counts when the used + // slots are low, we clamp down on the totalModerateSlots value by keeping track + // of a historical runnable goroutine average. + kvsa.moderateSlotsClamp = int(float64(threshold*procs)/2 - kvsa.runnableEWMA) + if kvsa.moderateSlotsClampOverride != 0 { + kvsa.moderateSlotsClamp = kvsa.moderateSlotsClampOverride + } + if kvsa.granter.totalModerateLoadSlots > kvsa.moderateSlotsClamp { + kvsa.granter.totalModerateLoadSlots = kvsa.moderateSlotsClamp + } + if kvsa.granter.totalModerateLoadSlots < 0 { + kvsa.granter.totalModerateLoadSlots = 0 + } + + kvsa.totalSlotsMetric.Update(int64(kvsa.granter.totalHighLoadSlots)) + kvsa.totalModerateSlotsMetric.Update(int64(kvsa.granter.totalModerateLoadSlots)) +} + +func (kvsa *kvSlotAdjuster) isOverloaded() bool { + return kvsa.granter.usedSlots >= kvsa.granter.totalHighLoadSlots && !kvsa.granter.skipSlotEnforcement +} From 51903eadde3b2b4548f1c1d7673095071f7d5886 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 12 Aug 2022 17:56:45 -0400 Subject: [PATCH 4/9] admission: move sqlNodeCPUOverloadIndicator into its own file Pure code movement. Release note: None --- pkg/util/admission/BUILD.bazel | 1 + pkg/util/admission/granter.go | 25 ------------ .../admission/sql_cpu_overload_indicator.go | 38 +++++++++++++++++++ 3 files changed, 39 insertions(+), 25 deletions(-) create mode 100644 pkg/util/admission/sql_cpu_overload_indicator.go diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index 82a6af64654c..8aa8fae68abd 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "granter.go", "io_load_listener.go", "kv_slot_adjuster.go", + "sql_cpu_overload_indicator.go", "store_token_estimation.go", "work_queue.go", ], diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 0a06b0b52dda..4db8ae3aab68 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -1077,31 +1077,6 @@ type storeRequester interface { setStoreRequestEstimates(estimates storeRequestEstimates) } -// sqlNodeCPUOverloadIndicator is the implementation of cpuOverloadIndicator -// for a single-tenant SQL node in a multi-tenant cluster. This has to rely on -// the periodic load information from the cpu scheduler and will therefore be -// tuned towards indicating overload at higher overload points (otherwise we -// could fluctuate into underloaded territory due to restricting admission, -// and not be work conserving). Such tuning towards more overload, and -// therefore more queueing inside the scheduler, is somewhat acceptable since -// a SQL node is not multi-tenant. -// -// TODO(sumeer): implement. -type sqlNodeCPUOverloadIndicator struct { -} - -var _ cpuOverloadIndicator = &sqlNodeCPUOverloadIndicator{} -var _ CPULoadListener = &sqlNodeCPUOverloadIndicator{} - -func (sn *sqlNodeCPUOverloadIndicator) CPULoad( - runnable int, procs int, samplePeriod time.Duration, -) { -} - -func (sn *sqlNodeCPUOverloadIndicator) isOverloaded() bool { - return false -} - var ( totalSlots = metric.Metadata{ Name: "admission.granter.total_slots.kv", diff --git a/pkg/util/admission/sql_cpu_overload_indicator.go b/pkg/util/admission/sql_cpu_overload_indicator.go new file mode 100644 index 000000000000..26b897bd52fa --- /dev/null +++ b/pkg/util/admission/sql_cpu_overload_indicator.go @@ -0,0 +1,38 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package admission + +import "time" + +// sqlNodeCPUOverloadIndicator is the implementation of cpuOverloadIndicator +// for a single-tenant SQL node in a multi-tenant cluster. This has to rely on +// the periodic load information from the cpu scheduler and will therefore be +// tuned towards indicating overload at higher overload points (otherwise we +// could fluctuate into underloaded territory due to restricting admission, +// and not be work conserving). Such tuning towards more overload, and +// therefore more queueing inside the scheduler, is somewhat acceptable since +// a SQL node is not multi-tenant. +// +// TODO(sumeer): implement. +type sqlNodeCPUOverloadIndicator struct { +} + +var _ cpuOverloadIndicator = &sqlNodeCPUOverloadIndicator{} +var _ CPULoadListener = &sqlNodeCPUOverloadIndicator{} + +func (sn *sqlNodeCPUOverloadIndicator) CPULoad( + runnable int, procs int, samplePeriod time.Duration, +) { +} + +func (sn *sqlNodeCPUOverloadIndicator) isOverloaded() bool { + return false +} From 35741fdca53940579e0a84ad71430445eb0b9269 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 12 Aug 2022 18:01:03 -0400 Subject: [PATCH 5/9] admission: move tokensLinearModel into its own file Pure code movement. Release note: None --- pkg/util/admission/BUILD.bazel | 1 + pkg/util/admission/store_token_estimation.go | 152 ----------------- pkg/util/admission/tokens_linear_model.go | 163 +++++++++++++++++++ 3 files changed, 164 insertions(+), 152 deletions(-) create mode 100644 pkg/util/admission/tokens_linear_model.go diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index 8aa8fae68abd..06c9c8782fb8 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "kv_slot_adjuster.go", "sql_cpu_overload_indicator.go", "store_token_estimation.go", + "tokens_linear_model.go", "work_queue.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/util/admission", diff --git a/pkg/util/admission/store_token_estimation.go b/pkg/util/admission/store_token_estimation.go index 3e76c5889193..be66b50452d1 100644 --- a/pkg/util/admission/store_token_estimation.go +++ b/pkg/util/admission/store_token_estimation.go @@ -104,158 +104,6 @@ const l0IngestMultiplierMax = 1.5 const ingestMultiplierMin = 0.5 const ingestMultiplierMax = 1.5 -// tokensLinearModel represents a model y = multiplier.x + constant. -type tokensLinearModel struct { - multiplier float64 - // constant >= 0 - constant int64 -} - -func (m tokensLinearModel) applyLinearModel(b int64) int64 { - return int64(float64(b)*m.multiplier) + m.constant -} - -// tokensLinearModelFitter fits y = multiplier.x + constant, based on the -// current interval and then exponentially smooths the multiplier and -// constant. -// -// This fitter is probably poor and could be improved by taking history into -// account in a cleverer way, such as looking at many past samples and doing -// linear regression, under the assumption that the workload is stable. -// However, the simple approach here should be an improvement on the additive -// approach we previously used. -// -// -// TODO(sumeer): improve the model based on realistic combinations of -// workloads (e.g. foreground writes + index backfills). -type tokensLinearModelFitter struct { - // [multiplierMin, multiplierMax] constrains the multiplier. - multiplierMin float64 - multiplierMax float64 - - intLinearModel tokensLinearModel - smoothedLinearModel tokensLinearModel - smoothedPerWorkAccountedBytes int64 - - // Should be set to true for the L0 ingested bytes model: if all bytes are - // ingested below L0, the actual bytes will be zero and the accounted bytes - // non-zero. We need to update the model in this case. - updateWithZeroActualNonZeroAccountedForL0IngestedModel bool -} - -func makeTokensLinearModelFitter( - multMin float64, multMax float64, updateWithZeroActualNonZeroAccountedForL0IngestedModel bool, -) tokensLinearModelFitter { - return tokensLinearModelFitter{ - multiplierMin: multMin, - multiplierMax: multMax, - smoothedLinearModel: tokensLinearModel{ - multiplier: (multMin + multMax) / 2, - constant: 1, - }, - smoothedPerWorkAccountedBytes: 1, - updateWithZeroActualNonZeroAccountedForL0IngestedModel: updateWithZeroActualNonZeroAccountedForL0IngestedModel, - } -} - -// updateModelUsingIntervalStats updates the model, based on various stats -// over the last interval: the number of work items admitted (workCount), the -// bytes claimed by these work items (accountedBytes), and the actual bytes -// observed in the LSM for that interval (actualBytes). -// - -// As mentioned earlier, the current fitting algorithm is probably poor, though an -// improvement on what we had previously. The approach taken is: -// -// - Fit the best model we can for the interval, -// multiplier*accountedBytes + workCount*constant = actualBytes, while -// minimizing the constant. We prefer the model to use the multiplier for -// most of what it needs to account for actualBytes. -// This exact model ignores inaccuracies due to integer arithmetic -- we -// don't care about rounding errors since an error of 2 bytes per request is -// inconsequential. -// -// - The multiplier has to conform to the [min,max] configured for this model, -// and constant has to conform to a value >= 1. The constant is constrained -// to be >=1 on the intuition that we want a request to consume at least 1 -// token -- it isn't clear that this intuition is meaningful in any way. -// -// - Exponentially smooth this exact model's multiplier and constant based on -// history. -func (f *tokensLinearModelFitter) updateModelUsingIntervalStats( - accountedBytes int64, actualBytes int64, workCount int64, -) { - if workCount <= 1 || (actualBytes <= 0 && - (!f.updateWithZeroActualNonZeroAccountedForL0IngestedModel || accountedBytes <= 0)) { - // Don't want to update the model if workCount is very low or actual bytes - // is zero (except for the exceptions in the if-condition above). - // - // Not updating the model at all does have the risk that a large constant - // will keep penalizing in the future. For example, if there are only - // ingests, and the regular writes model had a large constant, it will - // keep penalizing ingests. So we scale down the constant as if the new - // model had a 0 value for the constant and the exponential smoothing - // alpha was 0.5, i.e., halve the constant. - f.intLinearModel = tokensLinearModel{} - f.smoothedLinearModel.constant = max(1, f.smoothedLinearModel.constant/2) - return - } - if actualBytes < 0 { - actualBytes = 0 - } - const alpha = 0.5 - if accountedBytes <= 0 { - if actualBytes > 0 { - // Anomaly. Assume that we will see smoothedPerWorkAccountedBytes in the - // future. This prevents us from blowing up the constant in the model due - // to this anomaly. - accountedBytes = workCount * max(1, f.smoothedPerWorkAccountedBytes) - } else { - // actualBytes is also 0. - accountedBytes = 1 - } - } else { - perWorkAccountedBytes := accountedBytes / workCount - f.smoothedPerWorkAccountedBytes = int64( - alpha*float64(perWorkAccountedBytes) + (1-alpha)*float64(f.smoothedPerWorkAccountedBytes)) - } - // INVARIANT: workCount > 0, accountedBytes > 0, actualBytes >= 0. - - // Start with the lower bound of 1 on constant, since we want most of bytes - // to be fitted using the multiplier. So workCount tokens go into that. - constant := int64(1) - // Then compute the multiplier. - multiplier := float64(max(0, actualBytes-workCount*constant)) / float64(accountedBytes) - // The multiplier may be too high or too low, so make it conform to - // [min,max]. - if multiplier > f.multiplierMax { - multiplier = f.multiplierMax - } else if multiplier < f.multiplierMin { - multiplier = f.multiplierMin - } - // This is the model with the multiplier as small or large as possible, - // while minimizing constant (which is 1). - modelBytes := int64(multiplier*float64(accountedBytes)) + (constant * workCount) - // If the model is not accounting for all of actualBytes, we are forced to - // increase the constant to cover the difference. - if modelBytes < actualBytes { - constantAdjust := (actualBytes - modelBytes) / workCount - // Avoid overflow in case of bad stats. - if constantAdjust+constant > 0 { - constant += constantAdjust - } - } - // The best model we can come up for the interval. - f.intLinearModel = tokensLinearModel{ - multiplier: multiplier, - constant: constant, - } - // Smooth the multiplier and constant factors. - f.smoothedLinearModel.multiplier = alpha*multiplier + (1-alpha)*f.smoothedLinearModel.multiplier - f.smoothedLinearModel.constant = int64( - alpha*float64(constant) + (1-alpha)*float64(f.smoothedLinearModel.constant)) -} - type storePerWorkTokenEstimator struct { atAdmissionWorkTokens int64 atDoneL0WriteTokensLinearModel tokensLinearModelFitter diff --git a/pkg/util/admission/tokens_linear_model.go b/pkg/util/admission/tokens_linear_model.go new file mode 100644 index 000000000000..57b499172357 --- /dev/null +++ b/pkg/util/admission/tokens_linear_model.go @@ -0,0 +1,163 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package admission + +// tokensLinearModel represents a model y = multiplier.x + constant. +type tokensLinearModel struct { + multiplier float64 + // constant >= 0 + constant int64 +} + +func (m tokensLinearModel) applyLinearModel(b int64) int64 { + return int64(float64(b)*m.multiplier) + m.constant +} + +// tokensLinearModelFitter fits y = multiplier.x + constant, based on the +// current interval and then exponentially smooths the multiplier and +// constant. +// +// This fitter is probably poor and could be improved by taking history into +// account in a cleverer way, such as looking at many past samples and doing +// linear regression, under the assumption that the workload is stable. +// However, the simple approach here should be an improvement on the additive +// approach we previously used. +// +// +// TODO(sumeer): improve the model based on realistic combinations of +// workloads (e.g. foreground writes + index backfills). +type tokensLinearModelFitter struct { + // [multiplierMin, multiplierMax] constrains the multiplier. + multiplierMin float64 + multiplierMax float64 + + intLinearModel tokensLinearModel + smoothedLinearModel tokensLinearModel + smoothedPerWorkAccountedBytes int64 + + // Should be set to true for the L0 ingested bytes model: if all bytes are + // ingested below L0, the actual bytes will be zero and the accounted bytes + // non-zero. We need to update the model in this case. + updateWithZeroActualNonZeroAccountedForL0IngestedModel bool +} + +func makeTokensLinearModelFitter( + multMin float64, multMax float64, updateWithZeroActualNonZeroAccountedForL0IngestedModel bool, +) tokensLinearModelFitter { + return tokensLinearModelFitter{ + multiplierMin: multMin, + multiplierMax: multMax, + smoothedLinearModel: tokensLinearModel{ + multiplier: (multMin + multMax) / 2, + constant: 1, + }, + smoothedPerWorkAccountedBytes: 1, + updateWithZeroActualNonZeroAccountedForL0IngestedModel: updateWithZeroActualNonZeroAccountedForL0IngestedModel, + } +} + +// updateModelUsingIntervalStats updates the model, based on various stats +// over the last interval: the number of work items admitted (workCount), the +// bytes claimed by these work items (accountedBytes), and the actual bytes +// observed in the LSM for that interval (actualBytes). +// +// As mentioned store_token_estimation.go, the current fitting algorithm is +// probably poor, though an improvement on what we had previously. The approach +// taken is: +// +// - Fit the best model we can for the interval, +// multiplier*accountedBytes + workCount*constant = actualBytes, while +// minimizing the constant. We prefer the model to use the multiplier for +// most of what it needs to account for actualBytes. +// This exact model ignores inaccuracies due to integer arithmetic -- we +// don't care about rounding errors since an error of 2 bytes per request is +// inconsequential. +// +// - The multiplier has to conform to the [min,max] configured for this model, +// and constant has to conform to a value >= 1. The constant is constrained +// to be >=1 on the intuition that we want a request to consume at least 1 +// token -- it isn't clear that this intuition is meaningful in any way. +// +// - Exponentially smooth this exact model's multiplier and constant based on +// history. +func (f *tokensLinearModelFitter) updateModelUsingIntervalStats( + accountedBytes int64, actualBytes int64, workCount int64, +) { + if workCount <= 1 || (actualBytes <= 0 && + (!f.updateWithZeroActualNonZeroAccountedForL0IngestedModel || accountedBytes <= 0)) { + // Don't want to update the model if workCount is very low or actual bytes + // is zero (except for the exceptions in the if-condition above). + // + // Not updating the model at all does have the risk that a large constant + // will keep penalizing in the future. For example, if there are only + // ingests, and the regular writes model had a large constant, it will + // keep penalizing ingests. So we scale down the constant as if the new + // model had a 0 value for the constant and the exponential smoothing + // alpha was 0.5, i.e., halve the constant. + f.intLinearModel = tokensLinearModel{} + f.smoothedLinearModel.constant = max(1, f.smoothedLinearModel.constant/2) + return + } + if actualBytes < 0 { + actualBytes = 0 + } + const alpha = 0.5 + if accountedBytes <= 0 { + if actualBytes > 0 { + // Anomaly. Assume that we will see smoothedPerWorkAccountedBytes in the + // future. This prevents us from blowing up the constant in the model due + // to this anomaly. + accountedBytes = workCount * max(1, f.smoothedPerWorkAccountedBytes) + } else { + // actualBytes is also 0. + accountedBytes = 1 + } + } else { + perWorkAccountedBytes := accountedBytes / workCount + f.smoothedPerWorkAccountedBytes = int64( + alpha*float64(perWorkAccountedBytes) + (1-alpha)*float64(f.smoothedPerWorkAccountedBytes)) + } + // INVARIANT: workCount > 0, accountedBytes > 0, actualBytes >= 0. + + // Start with the lower bound of 1 on constant, since we want most of bytes + // to be fitted using the multiplier. So workCount tokens go into that. + constant := int64(1) + // Then compute the multiplier. + multiplier := float64(max(0, actualBytes-workCount*constant)) / float64(accountedBytes) + // The multiplier may be too high or too low, so make it conform to + // [min,max]. + if multiplier > f.multiplierMax { + multiplier = f.multiplierMax + } else if multiplier < f.multiplierMin { + multiplier = f.multiplierMin + } + // This is the model with the multiplier as small or large as possible, + // while minimizing constant (which is 1). + modelBytes := int64(multiplier*float64(accountedBytes)) + (constant * workCount) + // If the model is not accounting for all of actualBytes, we are forced to + // increase the constant to cover the difference. + if modelBytes < actualBytes { + constantAdjust := (actualBytes - modelBytes) / workCount + // Avoid overflow in case of bad stats. + if constantAdjust+constant > 0 { + constant += constantAdjust + } + } + // The best model we can come up for the interval. + f.intLinearModel = tokensLinearModel{ + multiplier: multiplier, + constant: constant, + } + // Smooth the multiplier and constant factors. + f.smoothedLinearModel.multiplier = alpha*multiplier + (1-alpha)*f.smoothedLinearModel.multiplier + f.smoothedLinearModel.constant = int64( + alpha*float64(constant) + (1-alpha)*float64(f.smoothedLinearModel.constant)) +} From d3b4b7a8b45b9bd674120864a198af838fa058e3 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Sat, 13 Aug 2022 00:32:47 -0400 Subject: [PATCH 6/9] admission: add top-level admission.go MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Subsuming doc.go and housing the central interfaces in this package. I'm not breaking things up into subpackages in this PR, but this paves the way for that. The structure I'm imagining is roughly: pkg/util/admission/ ├── admission.go ├── admissionpb/ ├── diskbandwidthlimiter/ ├── diskloadwatcher/ ├── grantcoordinator/ ├── granter/ ├── ioloadlistener/ ├── kvslotadjuster/ ├── sqlcpuoverloadindicator/ ├── storeworkqueue/ └── tokenlinearmodel/ Where the sub-packages house concrete implementations of interfaces defined in admission.go, corresponding tests, and constructors for those types. They depend on the base level pkg/util/admission package and talk to the other subpackages through interfaces. The existing interfaces are already written in a manner to make this happen. For other examples of this pattern, look at pkg/{upgrade,spanconfig}. Release note: None --- pkg/util/admission/BUILD.bazel | 2 +- pkg/util/admission/admission.go | 500 ++++++++++++++++++++++++ pkg/util/admission/doc.go | 129 ------ pkg/util/admission/grant_coordinator.go | 4 +- pkg/util/admission/granter.go | 366 ----------------- 5 files changed, 503 insertions(+), 498 deletions(-) create mode 100644 pkg/util/admission/admission.go delete mode 100644 pkg/util/admission/doc.go diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index 06c9c8782fb8..a866798b45ec 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -4,8 +4,8 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "admission", srcs = [ + "admission.go", "disk_bandwidth.go", - "doc.go", "grant_coordinator.go", "granter.go", "io_load_listener.go", diff --git a/pkg/util/admission/admission.go b/pkg/util/admission/admission.go new file mode 100644 index 000000000000..01ea2e7e4ce7 --- /dev/null +++ b/pkg/util/admission/admission.go @@ -0,0 +1,500 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// The admission package contains abstractions for admission control for +// CockroachDB nodes, both for single-tenant and multi-tenant (aka serverless) +// clusters. In the latter, both KV and SQL nodes are expected to use these +// abstractions. +// +// Admission control has the goal of +// - Limiting node overload, so that bad things don't happen due to starvation +// of resources. +// - Providing performance isolation between low and high importance +// activities, so that overload caused by the latter does not impact the +// latency of the former. Additionally, for multi-tenant KV nodes, the +// isolation should extend to inter-tenant performance isolation. +// Isolation is strictly harder than limiting node overload, and the +// abstractions here are likely to be average quality in doing so. +// +// At a high-level we are trying to shift queueing from system-provided +// resource allocation abstractions that we do not control, like the goroutine +// scheduler, to queueing in admission control, where we can reorder. This +// needs to be done while maintaining high utilization of the resource. +// +// Note that everything here operates at a single node level, and not at a +// cluster level. Cluster level admission control is insufficient for limiting +// node overload or to provide performance isolation in a distributed system +// with strong work affinity (which is true for a stateful system like +// CockroachDB, since rebalancing operates at time scales that can be higher +// than what we need). Cluster level admission control can complement node +// level admission control, in that it can prevent severe abuse, or provide +// cost controls to tenants. +// +// It is possible to also have intermediate mechanisms that gate admission of +// work on load signals of all the nodes in the raft group of the range. This +// could be especially useful for writes where non-leaseholder nodes could be +// suffering from cpu or disk IO overload. This is not considered in the +// following interfaces. +// +// TODO(sumeer): describe more of the design thinking documented in +// https://github.com/sumeerbhola/cockroach/blob/27ab4062ad1b036ab1e686a66a04723bd9f2b5a0/pkg/util/cpupool/cpu_pool.go +// either in a comment here or a separate RFC. +// + +// TODO(sumeer): update with all the recent changes. + +// Internal organization: +// +// The package is mostly structured as a set of interfaces that are meant to +// provide a general framework, and specific implementations that are +// initially quite simple in their heuristics but may become more +// sophisticated over time. The concrete abstractions: +// - Tokens and slots are the two ways admission is granted (see grantKind). +// - Categorization of kinds of work (see WorkKind), and a priority ordering +// across WorkKinds that is used to reflect their shared need for underlying +// resources. +// - The top-level GrantCoordinator which coordinates grants across these +// WorkKinds. The WorkKinds handled by an instantiation of GrantCoordinator +// will differ for single-tenant clusters, and multi-tenant clusters +// consisting of (multi-tenant) KV nodes and (single-tenant) SQL nodes. +// +// The interfaces involved: +// - requester: handles all requests for a particular WorkKind. Implemented by +// WorkQueue. The requester implementation is responsible for controlling +// the admission order within a WorkKind based on tenant fairness, +// importance of work etc. +// - granter: the counterpart to requester which grants admission tokens or +// slots. The implementations are slotGranter, tokenGranter, +// kvStoreTokenGranter. The implementation of requester interacts with the +// granter interface. +// - granterWithLockedCalls: this is an extension of granter that is used +// as part of the implementation of GrantCoordinator. This arrangement +// is partly to centralize locking in the GrantCoordinator (except for +// the lock in WorkQueue). +// - cpuOverloadIndicator: this serves as an optional additional gate on +// granting, by providing an (ideally) instantaneous signal of cpu overload. +// The kvSlotAdjuster is the concrete implementation, except for SQL +// nodes, where this will be implemented by sqlNodeCPUOverloadIndicator. +// CPULoadListener is also implemented by these structs, to listen to +// the latest CPU load information from the scheduler. +// +// Load observation and slot count or token burst adjustment: Dynamic +// adjustment is performed by kvSlotAdjuster for KVWork slots. This is because +// KVWork is expected to usually be CPU bound (due to good caching), and +// unlike SQLKVResponseWork and SQLSQLResponseWork (which are even more CPU +// bound), we have a completion indicator -- so we can expect to have a +// somewhat stable KVWork slot count even if the work sizes are extremely +// heterogeneous. +// +// There isn't token burst adjustment (except for each store -- see below), +// and the burst limits should be chosen to err on the side of fully +// saturating CPU, since we have the fallback of the cpuOverloadIndicator to +// stop granting even if tokens are available. If we figure out a way to +// dynamically tune the token burst count, or (even more ambitious) figure out +// a way to come up with a token rate, it should fit in the general framework +// that is setup here. +// + +// Partial usage example (regular cluster): +// +// var metricRegistry *metric.Registry = ... +// coord, metrics := admission.NewGrantCoordinator(admission.Options{...}) +// for i := range metrics { +// registry.AddMetricStruct(metrics[i]) +// } +// kvQueue := coord.GetWorkQueue(admission.KVWork) +// // Pass kvQueue to server.Node that implements roachpb.InternalServer. +// ... +// // Do similar things with the other WorkQueues. +// +// Usage of WorkQueue for KV: +// // Before starting some work +// if enabled, err := kvQueue.Admit(ctx, WorkInfo{TenantID: tid, ...}); err != nil { +// return err +// } +// doWork() +// if enabled { kvQueue.AdmittedWorkDone(tid) } + +// Additionally, each store has a single StoreWorkQueue and GrantCoordinator +// for writes. See kvStoreTokenGranter and how its tokens are dynamically +// adjusted based on Pebble metrics. + +package admission + +import ( + "time" + + "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" +) + +// requester is an interface implemented by an object that orders admission +// work for a particular WorkKind. See WorkQueue for the implementation of +// requester. +type requester interface { + // hasWaitingRequests returns whether there are any waiting/queued requests + // of this WorkKind. + hasWaitingRequests() bool + // granted is called by a granter to grant admission to a single queued + // request. It returns > 0 if the grant was accepted, else returns 0. A + // grant may not be accepted if the grant raced with request cancellation + // and there are now no waiting requests. The grantChainID is used when + // calling continueGrantChain -- see the comment with that method below. + // When accepted, the return value indicates the number of slots/tokens that + // were used. + // REQUIRES: count <= 1 for slots. + granted(grantChainID grantChainID) int64 + close() +} + +// granter is paired with a requester in that a requester for a particular +// WorkKind will interact with a granter. See admission.go for an overview of +// how this fits into the overall structure. +type granter interface { + grantKind() grantKind + // tryGet is used by a requester to get slots/tokens for a piece of work + // that has encountered no waiting/queued work. This is the fast path that + // avoids queueing in the requester. + // + // REQUIRES: count > 0. count == 1 for slots. + tryGet(count int64) bool + // returnGrant is called for: + // - returning slots after use. + // - returning either slots or tokens when the grant raced with the work + // being canceled, and the grantee did not end up doing any work. + // + // The last case occurs despite the return value on the requester.granted + // method -- it is possible that the work was not canceled at the time when + // requester.grant was called, and hence returned a count > 0, but later + // when the goroutine doing the work noticed that it had been granted, there + // is a possibility that that raced with cancellation. + // + // Do not use this for doing store IO-related token adjustments when work is + // done -- that should be done via granterWithStoreWriteDone.storeWriteDone. + // + // REQUIRES: count > 0. count == 1 for slots. + returnGrant(count int64) + // tookWithoutPermission informs the granter that a slot or tokens were + // taken unilaterally, without permission. This is useful: + // - Slots: this is useful since KVWork is allowed to bypass admission + // control for high priority internal activities (e.g. node liveness) and + // for KVWork that generates other KVWork (like intent resolution of + // discovered intents). Not bypassing for the latter could result in + // single node or distributed deadlock, and since such work is typically + // not a major (on average) consumer of resources, we consider bypassing + // to be acceptable. + // - Tokens: this is useful when the initial estimated tokens for a unit of + // work turned out to be an underestimate. + // + // Do not use this for doing store IO-related token adjustments when work is + // done -- that should be done via granterWithStoreWriteDone.storeWriteDone. + // + // REQUIRES: count > 0. count == 1 for slots. + tookWithoutPermission(count int64) + // continueGrantChain is called by the requester at some point after grant + // was called on the requester. The expectation is that this is called by + // the grantee after its goroutine runs and notices that it has been granted + // a slot/tokens. This provides a natural throttling that reduces grant + // bursts by taking into immediate account the capability of the goroutine + // scheduler to schedule such work. + // + // In an experiment, using such grant chains reduced burstiness of grants by + // 5x and shifted ~2s of latency (at p99) from the scheduler into admission + // control (which is desirable since the latter is where we can + // differentiate between work). + // + // TODO(sumeer): the "grant chain" concept is subtle and under-documented. + // It's easy to go through most of this package thinking it has something to + // do with dependent requests (e.g. intent resolution chains on an end txn). + // It would help for a top-level comment on grantChainID or continueGrantChain + // to spell out what grant chains are, their purpose, and how they work with + // an example. + continueGrantChain(grantChainID grantChainID) +} + +// granterWithLockedCalls is an encapsulation of typically one +// granter-requester pair, and for kvStoreTokenGranter of two +// granter-requester pairs (one for each workClass). It is used as an internal +// implementation detail of the GrantCoordinator. An implementer of +// granterWithLockedCalls responds to calls from its granter(s) by calling +// into the GrantCoordinator, which then calls the various *Locked() methods. +// The demuxHandle is meant to be opaque to the GrantCoordinator, and is used +// when this interface encapsulates multiple granter-requester pairs -- it is +// currently used only by kvStoreTokenGranter, where it is a workClass. The +// *Locked() methods are where the differences in slots and various kinds of +// tokens are handled. +type granterWithLockedCalls interface { + // tryGetLocked is the real implementation of tryGet from the granter + // interface. demuxHandle is an opaque handle that was passed into the + // GrantCoordinator. + tryGetLocked(count int64, demuxHandle int8) grantResult + // returnGrantLocked is the real implementation of returnGrant from the + // granter interface. demuxHandle is an opaque handle that was passed into + // the GrantCoordinator. + returnGrantLocked(count int64, demuxHandle int8) + // tookWithoutPermissionLocked is the real implementation of + // tookWithoutPermission from the granter interface. demuxHandle is an + // opaque handle that was passed into the GrantCoordinator. + tookWithoutPermissionLocked(count int64, demuxHandle int8) + + // The following methods are for direct use by GrantCoordinator. + + // requesterHasWaitingRequests returns whether some requester associated + // with the granter has waiting requests. + requesterHasWaitingRequests() bool + // tryGrantLocked is used to attempt to grant to waiting requests. + tryGrantLocked(grantChainID grantChainID) grantResult +} + +// granterWithIOTokens is used to abstract kvStoreTokenGranter for testing. +// The interface is used by the entity that periodically looks at load and +// computes the tokens to grant (ioLoadListener). +type granterWithIOTokens interface { + // setAvailableIOTokensLocked bounds the available tokens that can be + // granted to the value provided in the tokens parameter. This is not a + // tight bound when the callee has negative available tokens, due to the use + // of granter.tookWithoutPermission, since in that the case the callee + // increments that negative value with the value provided by tokens. This + // method needs to be called periodically. The return value is the number of + // used tokens in the interval since the prior call to this method. Note + // that tokensUsed can be negative, though that will be rare, since it is + // possible for tokens to be returned. + setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) + // setAvailableElasticDiskBandwidthTokensLocked bounds the available tokens + // that can be granted to elastic work. These tokens are based on disk + // bandwidth being a bottleneck resource. + setAvailableElasticDiskBandwidthTokensLocked(tokens int64) + // getDiskTokensUsedAndResetLocked returns the disk bandwidth tokens used + // since the last such call. + getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 + // setAdmittedDoneModelsLocked supplies the models to use when + // storeWriteDone is called, to adjust token consumption. Note that these + // models are not used for token adjustment at admission time -- that is + // handled by StoreWorkQueue and is not in scope of this granter. This + // asymmetry is due to the need to use all the functionality of WorkQueue at + // admission time. See the long explanatory comment at the beginning of + // store_token_estimation.go, regarding token estimation. + setAdmittedDoneModelsLocked(l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, + ingestLM tokensLinearModel) +} + +// granterWithStoreWriteDone is used to abstract kvStoreTokenGranter for +// testing. The interface is used by StoreWorkQueue to pass on sizing +// information provided when the work was completed. +type granterWithStoreWriteDone interface { + granter + storeWriteDone(originalTokens int64, doneInfo StoreWorkDoneInfo) (additionalTokens int64) +} + +// cpuOverloadIndicator is meant to be an instantaneous indicator of cpu +// availability. Since actual scheduler stats are periodic, we prefer to use +// the KV slot availability, since it is instantaneous. The +// cpuOverloadIndicator is used to gate admission of work other than KVWork +// (KVWork only looks at slot availability). An instantaneous indicator limits +// over-admission and queueing in the scheduler, and thereby provider better +// isolation, especially in multi-tenant environments where tenants not +// responsible for a load spike expect to suffer no increase in latency. +// +// In multi-tenant settings, for single-tenant SQL nodes, which do not do KV +// work, we do not have an instantaneous indicator and instead use +// sqlNodeCPUOverloadIndicator. +type cpuOverloadIndicator interface { + isOverloaded() bool +} + +// CPULoadListener listens to the latest CPU load information. Currently we +// expect this to be called every 1ms, unless the cpu is extremely +// underloaded. If the samplePeriod is > 1ms, admission control enforcement +// for CPU is disabled. +type CPULoadListener interface { + CPULoad(runnable int, procs int, samplePeriod time.Duration) +} + +// storeRequester is used to abstract *StoreWorkQueue for testing. +type storeRequester interface { + requesterClose + getRequesters() [numWorkClasses]requester + getStoreAdmissionStats() storeAdmissionStats + setStoreRequestEstimates(estimates storeRequestEstimates) +} + +// grantKind represents the two kind of ways we grant admission: using a slot +// or a token. The slot terminology is akin to a scheduler, where a scheduling +// slot must be free for a thread to run. But unlike a scheduler, we don't +// have visibility into the fact that work execution may be blocked on IO. So +// a slot can also be viewed as a limit on concurrency of ongoing work. The +// token terminology is inspired by token buckets. In this case the token is +// handed out for admission but it is not returned (unlike a slot). Unlike a +// token bucket, which shapes the rate, the current implementation (see +// tokenGranter) limits burstiness and does not do rate shaping -- this is +// because it is hard to predict what rate is appropriate given the difference +// in sizes of the work. This lack of rate shaping may change in the future +// and is not a limitation of the interfaces. Similarly, there is no rate +// shaping applied when granting slots and that may also change in the future. +// The main difference between a slot and a token is that a slot is used when +// we can know when the work is complete. Having this extra completion +// information can be advantageous in admission control decisions, so +// WorkKinds where this information is easily available use slots. +// +// StoreGrantCoordinators and its corresponding StoreWorkQueues are a hybrid +// -- they use tokens (as explained later). However, there is useful +// completion information such as how many tokens were actually used, which +// can differ from the up front information, and is utilized to adjust the +// available tokens. +type grantKind int8 + +const ( + slot grantKind = iota + token +) + +type grantResult int8 + +const ( + grantSuccess grantResult = iota + // grantFailDueToSharedResource is returned when the granter is unable to + // grant because a shared resource (CPU or memory) is overloaded. For grant + // chains, this is a signal to terminate. + grantFailDueToSharedResource + // grantFailLocal is returned when the granter is unable to grant due to (a) + // a local constraint -- insufficient tokens or slots, or (b) no work is + // waiting. + grantFailLocal +) + +// grantChainID is the ID for a grant chain. See continueGrantChain for +// details. +type grantChainID uint64 + +// WorkKind represents various types of work that are subject to admission +// control. +type WorkKind int8 + +// The list of WorkKinds are ordered from lower level to higher level, and +// also serves as a hard-wired ordering from most important to least important +// (for details on how this ordering is enacted, see the GrantCoordinator +// code). +// +// KVWork, SQLKVResponseWork, SQLSQLResponseWork are the lower-level work +// units that are expected to be primarily CPU bound (with disk IO for KVWork, +// but cache hit rates are typically high), and expected to be where most of +// the CPU consumption happens. These are prioritized in the order +// KVWork > SQLKVResponseWork > SQLSQLResponseWork +// +// The high prioritization of KVWork reduces the likelihood that non-SQL KV +// work will be starved. SQLKVResponseWork is prioritized over +// SQLSQLResponseWork since the former includes leaf DistSQL processing and we +// would like to release memory used up in RPC responses at lower layers of +// RPC tree. We expect that if SQLSQLResponseWork is delayed, it will +// eventually reduce new work being issued, which is a desirable form of +// natural backpressure. +// +// Furthermore, SQLStatementLeafStartWork and SQLStatementRootStartWork are +// prioritized lowest with +// SQLStatementLeafStartWork > SQLStatementRootStartWork +// This follows the same idea of prioritizing lower layers above higher layers +// since it releases memory caught up in lower layers, and exerts natural +// backpressure on the higher layer. +// +// Consider the example of a less important long-running single statement OLAP +// query competing with more important small OLTP queries in a single node +// setting. Say the OLAP query starts first and uses up all the KVWork slots, +// and the OLTP queries queue up for the KVWork slots. As the OLAP query +// KVWork completes, it will queue up for SQLKVResponseWork, which will not +// start because the OLTP queries are using up all available KVWork slots. As +// this OLTP KVWork completes, their SQLKVResponseWork will queue up. The +// WorkQueue for SQLKVResponseWork, when granting tokens, will first admit +// those for the more important OLTP queries. This will prevent or slow down +// admission of further work by the OLAP query. +// +// In an ideal world with the only shared resource (across WorkKinds) being +// CPU, and control over the CPU scheduler, we could pool all work, regardless +// of WorkKind into a single queue, and would not need to rely on this +// indirect backpressure and hard-wired ordering. However, we do not have +// control over the CPU scheduler, so we cannot preempt work with widely +// different cpu consumption. Additionally, (non-preemptible) memory is also a +// shared resource, and we wouldn't want to have partially done KVWork not +// finish, due to preemption in the CPU scheduler, since it can be holding +// significant amounts of memory (e.g. in scans). +// +// The aforementioned prioritization also enables us to get instantaneous +// feedback on CPU resource overload. This instantaneous feedback for a grant +// chain (mentioned earlier) happens in two ways: +// - the chain requires the grantee's goroutine to run. +// - the cpuOverloadIndicator (see later), specifically the implementation +// provided by kvSlotAdjuster, provides instantaneous feedback (which is +// viable only because KVWork is the highest priority). +// +// Weaknesses of this strict prioritization across WorkKinds: +// - Priority inversion: Lower importance KVWork, not derived from SQL, like +// GC of MVCC versions, will happen before user-facing SQLKVResponseWork. +// This is because the backpressure, described in the example above, does +// not apply to work generated from within the KV layer. +// TODO(sumeer): introduce a KVLowPriWork and put it last in this ordering, +// to get over this limitation. +// - Insufficient competition leading to poor isolation: Putting +// SQLStatementLeafStartWork, SQLStatementRootStartWork in this list, within +// the same GrantCoordinator, does provide node overload protection, but not +// necessarily performance isolation when we have WorkKinds of different +// importance. Consider the same OLAP example above: if the KVWork slots +// being full due to the OLAP query prevents SQLStatementRootStartWork for +// the OLTP queries, the competition is starved out before it has an +// opportunity to submit any KVWork. Given that control over admitting +// SQLStatement{Leaf,Root}StartWork is not primarily about CPU control (the +// lower-level work items are where cpu is consumed), we could decouple +// these two into a separate GrantCoordinator and only gate them with (high) +// fixed slot counts that allow for enough competition, plus a memory +// overload indicator. +// TODO(sumeer): experiment with this approach. +// - Continuing the previous bullet, low priority long-lived +// {SQLStatementLeafStartWork, SQLStatementRootStartWork} could use up all +// the slots, if there was no high priority work for some period of time, +// and therefore starve admission of the high priority work when it does +// appear. The typical solution to this is to put a max on the number of +// slots low priority can use. This would be viable if we did not allow +// arbitrary int8 values to be set for Priority. +const ( + // KVWork represents requests submitted to the KV layer, from the same node + // or a different node. They may originate from the SQL layer or the KV + // layer. + KVWork WorkKind = iota + // SQLKVResponseWork is response processing in SQL for a KV response from a + // local or remote node. This can be either leaf or root DistSQL work, i.e., + // this is inter-layer and not necessarily inter-node. + SQLKVResponseWork + // SQLSQLResponseWork is response processing in SQL, for DistSQL RPC + // responses. This is root work happening in response to leaf SQL work, + // i.e., it is inter-node. + SQLSQLResponseWork + // SQLStatementLeafStartWork represents the start of leaf-level processing + // for a SQL statement. + SQLStatementLeafStartWork + // SQLStatementRootStartWork represents the start of root-level processing + // for a SQL statement. + SQLStatementRootStartWork + numWorkKinds +) + +func workKindString(workKind WorkKind) redact.RedactableString { + switch workKind { + case KVWork: + return "kv" + case SQLKVResponseWork: + return "sql-kv-response" + case SQLSQLResponseWork: + return "sql-sql-response" + case SQLStatementLeafStartWork: + return "sql-leaf-start" + case SQLStatementRootStartWork: + return "sql-root-start" + default: + panic(errors.AssertionFailedf("unknown WorkKind")) + } +} diff --git a/pkg/util/admission/doc.go b/pkg/util/admission/doc.go deleted file mode 100644 index bda99a40d83f..000000000000 --- a/pkg/util/admission/doc.go +++ /dev/null @@ -1,129 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -// The admission package contains abstractions for admission control for -// CockroachDB nodes, both for single-tenant and multi-tenant (aka serverless) -// clusters. In the latter, both KV and SQL nodes are expected to use these -// abstractions. -// -// Admission control has the goal of -// - Limiting node overload, so that bad things don't happen due to starvation -// of resources. -// - Providing performance isolation between low and high importance -// activities, so that overload caused by the latter does not impact the -// latency of the former. Additionally, for multi-tenant KV nodes, the -// isolation should extend to inter-tenant performance isolation. -// Isolation is strictly harder than limiting node overload, and the -// abstractions here are likely to be average quality in doing so. -// -// At a high-level we are trying to shift queueing from system-provided -// resource allocation abstractions that we do not control, like the goroutine -// scheduler, to queueing in admission control, where we can reorder. This -// needs to be done while maintaining high utilization of the resource. -// -// Note that everything here operates at a single node level, and not at a -// cluster level. Cluster level admission control is insufficient for limiting -// node overload or to provide performance isolation in a distributed system -// with strong work affinity (which is true for a stateful system like -// CockroachDB, since rebalancing operates at time scales that can be higher -// than what we need). Cluster level admission control can complement node -// level admission control, in that it can prevent severe abuse, or provide -// cost controls to tenants. -// -// It is possible to also have intermediate mechanisms that gate admission of -// work on load signals of all the nodes in the raft group of the range. This -// could be especially useful for writes where non-leaseholder nodes could be -// suffering from cpu or disk IO overload. This is not considered in the -// following interfaces. -// -// TODO(sumeer): describe more of the design thinking documented in -// https://github.com/sumeerbhola/cockroach/blob/27ab4062ad1b036ab1e686a66a04723bd9f2b5a0/pkg/util/cpupool/cpu_pool.go -// either in a comment here or a separate RFC. -// - -// TODO(sumeer): update with all the recent changes. - -// Internal organization: -// -// The package is mostly structured as a set of interfaces that are meant to -// provide a general framework, and specific implementations that are -// initially quite simple in their heuristics but may become more -// sophisticated over time. The concrete abstractions: -// - Tokens and slots are the two ways admission is granted (see grantKind). -// - Categorization of kinds of work (see WorkKind), and a priority ordering -// across WorkKinds that is used to reflect their shared need for underlying -// resources. -// - The top-level GrantCoordinator which coordinates grants across these -// WorkKinds. The WorkKinds handled by an instantiation of GrantCoordinator -// will differ for single-tenant clusters, and multi-tenant clusters -// consisting of (multi-tenant) KV nodes and (single-tenant) SQL nodes. -// -// The interfaces involved: -// - requester: handles all requests for a particular WorkKind. Implemented by -// WorkQueue. The requester implementation is responsible for controlling -// the admission order within a WorkKind based on tenant fairness, -// importance of work etc. -// - granter: the counterpart to requester which grants admission tokens or -// slots. The implementations are slotGranter, tokenGranter, -// kvStoreTokenGranter. The implementation of requester interacts with the -// granter interface. -// - granterWithLockedCalls: this is an extension of granter that is used -// as part of the implementation of GrantCoordinator. This arrangement -// is partly to centralize locking in the GrantCoordinator (except for -// the lock in WorkQueue). -// - cpuOverloadIndicator: this serves as an optional additional gate on -// granting, by providing an (ideally) instantaneous signal of cpu overload. -// The kvSlotAdjuster is the concrete implementation, except for SQL -// nodes, where this will be implemented by sqlNodeCPUOverloadIndicator. -// CPULoadListener is also implemented by these structs, to listen to -// the latest CPU load information from the scheduler. -// -// Load observation and slot count or token burst adjustment: Dynamic -// adjustment is performed by kvSlotAdjuster for KVWork slots. This is because -// KVWork is expected to usually be CPU bound (due to good caching), and -// unlike SQLKVResponseWork and SQLSQLResponseWork (which are even more CPU -// bound), we have a completion indicator -- so we can expect to have a -// somewhat stable KVWork slot count even if the work sizes are extremely -// heterogeneous. -// -// There isn't token burst adjustment (except for each store -- see below), -// and the burst limits should be chosen to err on the side of fully -// saturating CPU, since we have the fallback of the cpuOverloadIndicator to -// stop granting even if tokens are available. If we figure out a way to -// dynamically tune the token burst count, or (even more ambitious) figure out -// a way to come up with a token rate, it should fit in the general framework -// that is setup here. -// - -// Partial usage example (regular cluster): -// -// var metricRegistry *metric.Registry = ... -// coord, metrics := admission.NewGrantCoordinator(admission.Options{...}) -// for i := range metrics { -// registry.AddMetricStruct(metrics[i]) -// } -// kvQueue := coord.GetWorkQueue(admission.KVWork) -// // Pass kvQueue to server.Node that implements roachpb.InternalServer. -// ... -// // Do similar things with the other WorkQueues. -// -// Usage of WorkQueue for KV: -// // Before starting some work -// if enabled, err := kvQueue.Admit(ctx, WorkInfo{TenantID: tid, ...}); err != nil { -// return err -// } -// doWork() -// if enabled { kvQueue.AdmittedWorkDone(tid) } - -// Additionally, each store has a single StoreWorkQueue and GrantCoordinator -// for writes. See kvStoreTokenGranter and how its tokens are dynamically -// adjusted based on Pebble metrics. - -package admission diff --git a/pkg/util/admission/grant_coordinator.go b/pkg/util/admission/grant_coordinator.go index 85f716fd23ce..64b10cbf1c15 100644 --- a/pkg/util/admission/grant_coordinator.go +++ b/pkg/util/admission/grant_coordinator.go @@ -210,8 +210,8 @@ func (sgc *StoreGrantCoordinators) close() { } // GrantCoordinator is the top-level object that coordinates grants across -// different WorkKinds (for more context see the comment in doc.go, and the -// comment where WorkKind is declared). Typically there will one +// different WorkKinds (for more context see the comment in admission.go, and +// the comment where WorkKind is declared). Typically there will one // GrantCoordinator in a node for CPU intensive work, and for nodes that also // have the KV layer, one GrantCoordinator per store (these are managed by // StoreGrantCoordinators) for KVWork that uses that store. See the diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 4db8ae3aab68..b4ed67a87f48 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble" - "github.com/cockroachdb/redact" ) // EnabledSoftSlotGranting can be set to false to disable soft slot granting. @@ -31,309 +30,16 @@ var EnabledSoftSlotGranting = settings.RegisterBoolSetting( true, ) -// grantChainID is the ID for a grant chain. See continueGrantChain for -// details. -type grantChainID uint64 - // noGrantChain is a sentinel value representing that the grant is not // responsible for continuing a grant chain. It is only used internally in // this file -- requester implementations do not need to concern themselves // with this value. var noGrantChain grantChainID = 0 -// requester is an interface implemented by an object that orders admission -// work for a particular WorkKind. See WorkQueue for the implementation of -// requester. -type requester interface { - // hasWaitingRequests returns whether there are any waiting/queued requests - // of this WorkKind. - hasWaitingRequests() bool - // granted is called by a granter to grant admission to a single queued - // request. It returns > 0 if the grant was accepted, else returns 0. A - // grant may not be accepted if the grant raced with request cancellation - // and there are now no waiting requests. The grantChainID is used when - // calling continueGrantChain -- see the comment with that method below. - // When accepted, the return value indicates the number of slots/tokens that - // were used. - // REQUIRES: count <= 1 for slots. - granted(grantChainID grantChainID) int64 - close() -} - type requesterClose interface { close() } -// grantKind represents the two kind of ways we grant admission: using a slot -// or a token. The slot terminology is akin to a scheduler, where a scheduling -// slot must be free for a thread to run. But unlike a scheduler, we don't -// have visibility into the fact that work execution may be blocked on IO. So -// a slot can also be viewed as a limit on concurrency of ongoing work. The -// token terminology is inspired by token buckets. In this case the token is -// handed out for admission but it is not returned (unlike a slot). Unlike a -// token bucket, which shapes the rate, the current implementation (see -// tokenGranter) limits burstiness and does not do rate shaping -- this is -// because it is hard to predict what rate is appropriate given the difference -// in sizes of the work. This lack of rate shaping may change in the future -// and is not a limitation of the interfaces. Similarly, there is no rate -// shaping applied when granting slots and that may also change in the future. -// The main difference between a slot and a token is that a slot is used when -// we can know when the work is complete. Having this extra completion -// information can be advantageous in admission control decisions, so -// WorkKinds where this information is easily available use slots. -// -// StoreGrantCoordinators and its corresponding StoreWorkQueues are a hybrid -// -- they use tokens (as explained later). However, there is useful -// completion information such as how many tokens were actually used, which -// can differ from the up front information, and is utilized to adjust the -// available tokens. -type grantKind int8 - -const ( - slot grantKind = iota - token -) - -// granter is paired with a requester in that a requester for a particular -// WorkKind will interact with a granter. See doc.go for an overview of how -// this fits into the overall structure. -type granter interface { - grantKind() grantKind - // tryGet is used by a requester to get slots/tokens for a piece of work - // that has encountered no waiting/queued work. This is the fast path that - // avoids queueing in the requester. - // - // REQUIRES: count > 0. count == 1 for slots. - tryGet(count int64) bool - // returnGrant is called for: - // - returning slots after use. - // - returning either slots or tokens when the grant raced with the work - // being canceled, and the grantee did not end up doing any work. - // - // The last case occurs despite the return value on the requester.granted - // method -- it is possible that the work was not canceled at the time when - // requester.grant was called, and hence returned a count > 0, but later - // when the goroutine doing the work noticed that it had been granted, there - // is a possibility that that raced with cancellation. - // - // Do not use this for doing store IO-related token adjustments when work is - // done -- that should be done via granterWithStoreWriteDone.storeWriteDone. - // - // REQUIRES: count > 0. count == 1 for slots. - returnGrant(count int64) - // tookWithoutPermission informs the granter that a slot or tokens were - // taken unilaterally, without permission. This is useful: - // - Slots: this is useful since KVWork is allowed to bypass admission - // control for high priority internal activities (e.g. node liveness) and - // for KVWork that generates other KVWork (like intent resolution of - // discovered intents). Not bypassing for the latter could result in - // single node or distributed deadlock, and since such work is typically - // not a major (on average) consumer of resources, we consider bypassing - // to be acceptable. - // - Tokens: this is useful when the initial estimated tokens for a unit of - // work turned out to be an underestimate. - // - // Do not use this for doing store IO-related token adjustments when work is - // done -- that should be done via granterWithStoreWriteDone.storeWriteDone. - // - // REQUIRES: count > 0. count == 1 for slots. - tookWithoutPermission(count int64) - // continueGrantChain is called by the requester at some point after grant - // was called on the requester. The expectation is that this is called by - // the grantee after its goroutine runs and notices that it has been granted - // a slot/tokens. This provides a natural throttling that reduces grant - // bursts by taking into immediate account the capability of the goroutine - // scheduler to schedule such work. - // - // In an experiment, using such grant chains reduced burstiness of grants by - // 5x and shifted ~2s of latency (at p99) from the scheduler into admission - // control (which is desirable since the latter is where we can - // differentiate between work). - // - // TODO(sumeer): the "grant chain" concept is subtle and under-documented. - // It's easy to go through most of this package thinking it has something to - // do with dependent requests (e.g. intent resolution chains on an end txn). - // It would help for a top-level comment on grantChainID or continueGrantChain - // to spell out what grant chains are, their purpose, and how they work with - // an example. - continueGrantChain(grantChainID grantChainID) -} - -// WorkKind represents various types of work that are subject to admission -// control. -type WorkKind int8 - -// The list of WorkKinds are ordered from lower level to higher level, and -// also serves as a hard-wired ordering from most important to least important -// (for details on how this ordering is enacted, see the GrantCoordinator -// code). -// -// KVWork, SQLKVResponseWork, SQLSQLResponseWork are the lower-level work -// units that are expected to be primarily CPU bound (with disk IO for KVWork, -// but cache hit rates are typically high), and expected to be where most of -// the CPU consumption happens. These are prioritized in the order -// KVWork > SQLKVResponseWork > SQLSQLResponseWork -// -// The high prioritization of KVWork reduces the likelihood that non-SQL KV -// work will be starved. SQLKVResponseWork is prioritized over -// SQLSQLResponseWork since the former includes leaf DistSQL processing and we -// would like to release memory used up in RPC responses at lower layers of -// RPC tree. We expect that if SQLSQLResponseWork is delayed, it will -// eventually reduce new work being issued, which is a desirable form of -// natural backpressure. -// -// Furthermore, SQLStatementLeafStartWork and SQLStatementRootStartWork are -// prioritized lowest with -// SQLStatementLeafStartWork > SQLStatementRootStartWork -// This follows the same idea of prioritizing lower layers above higher layers -// since it releases memory caught up in lower layers, and exerts natural -// backpressure on the higher layer. -// -// Consider the example of a less important long-running single statement OLAP -// query competing with more important small OLTP queries in a single node -// setting. Say the OLAP query starts first and uses up all the KVWork slots, -// and the OLTP queries queue up for the KVWork slots. As the OLAP query -// KVWork completes, it will queue up for SQLKVResponseWork, which will not -// start because the OLTP queries are using up all available KVWork slots. As -// this OLTP KVWork completes, their SQLKVResponseWork will queue up. The -// WorkQueue for SQLKVResponseWork, when granting tokens, will first admit -// those for the more important OLTP queries. This will prevent or slow down -// admission of further work by the OLAP query. -// -// In an ideal world with the only shared resource (across WorkKinds) being -// CPU, and control over the CPU scheduler, we could pool all work, regardless -// of WorkKind into a single queue, and would not need to rely on this -// indirect backpressure and hard-wired ordering. However, we do not have -// control over the CPU scheduler, so we cannot preempt work with widely -// different cpu consumption. Additionally, (non-preemptible) memory is also a -// shared resource, and we wouldn't want to have partially done KVWork not -// finish, due to preemption in the CPU scheduler, since it can be holding -// significant amounts of memory (e.g. in scans). -// -// The aforementioned prioritization also enables us to get instantaneous -// feedback on CPU resource overload. This instantaneous feedback for a grant -// chain (mentioned earlier) happens in two ways: -// - the chain requires the grantee's goroutine to run. -// - the cpuOverloadIndicator (see later), specifically the implementation -// provided by kvSlotAdjuster, provides instantaneous feedback (which is -// viable only because KVWork is the highest priority). -// -// Weaknesses of this strict prioritization across WorkKinds: -// - Priority inversion: Lower importance KVWork, not derived from SQL, like -// GC of MVCC versions, will happen before user-facing SQLKVResponseWork. -// This is because the backpressure, described in the example above, does -// not apply to work generated from within the KV layer. -// TODO(sumeer): introduce a KVLowPriWork and put it last in this ordering, -// to get over this limitation. -// - Insufficient competition leading to poor isolation: Putting -// SQLStatementLeafStartWork, SQLStatementRootStartWork in this list, within -// the same GrantCoordinator, does provide node overload protection, but not -// necessarily performance isolation when we have WorkKinds of different -// importance. Consider the same OLAP example above: if the KVWork slots -// being full due to the OLAP query prevents SQLStatementRootStartWork for -// the OLTP queries, the competition is starved out before it has an -// opportunity to submit any KVWork. Given that control over admitting -// SQLStatement{Leaf,Root}StartWork is not primarily about CPU control (the -// lower-level work items are where cpu is consumed), we could decouple -// these two into a separate GrantCoordinator and only gate them with (high) -// fixed slot counts that allow for enough competition, plus a memory -// overload indicator. -// TODO(sumeer): experiment with this approach. -// - Continuing the previous bullet, low priority long-lived -// {SQLStatementLeafStartWork, SQLStatementRootStartWork} could use up all -// the slots, if there was no high priority work for some period of time, -// and therefore starve admission of the high priority work when it does -// appear. The typical solution to this is to put a max on the number of -// slots low priority can use. This would be viable if we did not allow -// arbitrary int8 values to be set for Priority. - -const ( - // KVWork represents requests submitted to the KV layer, from the same node - // or a different node. They may originate from the SQL layer or the KV - // layer. - KVWork WorkKind = iota - // SQLKVResponseWork is response processing in SQL for a KV response from a - // local or remote node. This can be either leaf or root DistSQL work, i.e., - // this is inter-layer and not necessarily inter-node. - SQLKVResponseWork - // SQLSQLResponseWork is response processing in SQL, for DistSQL RPC - // responses. This is root work happening in response to leaf SQL work, - // i.e., it is inter-node. - SQLSQLResponseWork - // SQLStatementLeafStartWork represents the start of leaf-level processing - // for a SQL statement. - SQLStatementLeafStartWork - // SQLStatementRootStartWork represents the start of root-level processing - // for a SQL statement. - SQLStatementRootStartWork - numWorkKinds -) - -func workKindString(workKind WorkKind) redact.RedactableString { - switch workKind { - case KVWork: - return "kv" - case SQLKVResponseWork: - return "sql-kv-response" - case SQLSQLResponseWork: - return "sql-sql-response" - case SQLStatementLeafStartWork: - return "sql-leaf-start" - case SQLStatementRootStartWork: - return "sql-root-start" - default: - panic(errors.AssertionFailedf("unknown WorkKind")) - } -} - -type grantResult int8 - -const ( - grantSuccess grantResult = iota - // grantFailDueToSharedResource is returned when the granter is unable to - // grant because a shared resource (CPU or memory) is overloaded. For grant - // chains, this is a signal to terminate. - grantFailDueToSharedResource - // grantFailLocal is returned when the granter is unable to grant due to (a) - // a local constraint -- insufficient tokens or slots, or (b) no work is - // waiting. - grantFailLocal -) - -// granterWithLockedCalls is an encapsulation of typically one -// granter-requester pair, and for kvStoreTokenGranter of two -// granter-requester pairs (one for each workClass). It is used as an internal -// implementation detail of the GrantCoordinator. An implementer of -// granterWithLockedCalls responds to calls from its granter(s) by calling -// into the GrantCoordinator, which then calls the various *Locked() methods. -// The demuxHandle is meant to be opaque to the GrantCoordinator, and is used -// when this interface encapsulates multiple granter-requester pairs -- it is -// currently used only by kvStoreTokenGranter, where it is a workClass. The -// *Locked() methods are where the differences in slots and various kinds of -// tokens are handled. -type granterWithLockedCalls interface { - // tryGetLocked is the real implementation of tryGet from the granter - // interface. demuxHandle is an opaque handle that was passed into the - // GrantCoordinator. - tryGetLocked(count int64, demuxHandle int8) grantResult - // returnGrantLocked is the real implementation of returnGrant from the - // granter interface. demuxHandle is an opaque handle that was passed into - // the GrantCoordinator. - returnGrantLocked(count int64, demuxHandle int8) - // tookWithoutPermissionLocked is the real implementation of - // tookWithoutPermission from the granter interface. demuxHandle is an - // opaque handle that was passed into the GrantCoordinator. - tookWithoutPermissionLocked(count int64, demuxHandle int8) - - // The following methods are for direct use by GrantCoordinator. - - // requesterHasWaitingRequests returns whether some requester associated - // with the granter has waiting requests. - requesterHasWaitingRequests() bool - // tryGrantLocked is used to attempt to grant to waiting requests. - tryGrantLocked(grantChainID grantChainID) grantResult -} - // For the cpu-bound slot case we have background activities (like Pebble // compactions) that would like to utilize additional slots if available (e.g. // to do concurrent compression of ssblocks). These activities do not want to @@ -922,30 +628,6 @@ func (sg *kvStoreTokenGranter) storeWriteDone( return additionalL0TokensNeeded } -// cpuOverloadIndicator is meant to be an instantaneous indicator of cpu -// availability. Since actual scheduler stats are periodic, we prefer to use -// the KV slot availability, since it is instantaneous. The -// cpuOverloadIndicator is used to gate admission of work other than KVWork -// (KVWork only looks at slot availability). An instantaneous indicator limits -// over-admission and queueing in the scheduler, and thereby provider better -// isolation, especially in multi-tenant environments where tenants not -// responsible for a load spike expect to suffer no increase in latency. -// -// In multi-tenant settings, for single-tenant SQL nodes, which do not do KV -// work, we do not have an instantaneous indicator and instead use -// sqlNodeCPUOverloadIndicator. -type cpuOverloadIndicator interface { - isOverloaded() bool -} - -// CPULoadListener listens to the latest CPU load information. Currently we -// expect this to be called every 1ms, unless the cpu is extremely -// underloaded. If the samplePeriod is > 1ms, admission control enforcement -// for CPU is disabled. -type CPULoadListener interface { - CPULoad(runnable int, procs int, samplePeriod time.Duration) -} - // PebbleMetricsProvider provides the pebble.Metrics for all stores. type PebbleMetricsProvider interface { GetPebbleMetrics() []StoreMetrics @@ -984,46 +666,6 @@ type DiskStats struct { ProvisionedBandwidth int64 } -// granterWithIOTokens is used to abstract kvStoreTokenGranter for testing. -// The interface is used by the entity that periodically looks at load and -// computes the tokens to grant (ioLoadListener). -type granterWithIOTokens interface { - // setAvailableIOTokensLocked bounds the available tokens that can be - // granted to the value provided in the tokens parameter. This is not a - // tight bound when the callee has negative available tokens, due to the use - // of granter.tookWithoutPermission, since in that the case the callee - // increments that negative value with the value provided by tokens. This - // method needs to be called periodically. The return value is the number of - // used tokens in the interval since the prior call to this method. Note - // that tokensUsed can be negative, though that will be rare, since it is - // possible for tokens to be returned. - setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) - // setAvailableElasticDiskBandwidthTokensLocked bounds the available tokens - // that can be granted to elastic work. These tokens are based on disk - // bandwidth being a bottleneck resource. - setAvailableElasticDiskBandwidthTokensLocked(tokens int64) - // getDiskTokensUsedAndResetLocked returns the disk bandwidth tokens used - // since the last such call. - getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 - // setAdmittedDoneModelsLocked supplies the models to use when - // storeWriteDone is called, to adjust token consumption. Note that these - // models are not used for token adjustment at admission time -- that is - // handled by StoreWorkQueue and is not in scope of this granter. This - // asymmetry is due to the need to use all the functionality of WorkQueue at - // admission time. See the long explanatory comment at the beginning of - // store_token_estimation.go, regarding token estimation. - setAdmittedDoneModelsLocked(l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, - ingestLM tokensLinearModel) -} - -// granterWithStoreWriteDone is used to abstract kvStoreTokenGranter for -// testing. The interface is used by StoreWorkQueue to pass on sizing -// information provided when the work was completed. -type granterWithStoreWriteDone interface { - granter - storeWriteDone(originalTokens int64, doneInfo StoreWorkDoneInfo) (additionalTokens int64) -} - // storeAdmissionStats are stats maintained by a storeRequester. The non-test // implementation of storeRequester is StoreWorkQueue. StoreWorkQueue updates // all of these when StoreWorkQueue.AdmittedWorkDone is called, so that these @@ -1069,14 +711,6 @@ type storeRequestEstimates struct { writeTokens int64 } -// storeRequester is used to abstract *StoreWorkQueue for testing. -type storeRequester interface { - requesterClose - getRequesters() [numWorkClasses]requester - getStoreAdmissionStats() storeAdmissionStats - setStoreRequestEstimates(estimates storeRequestEstimates) -} - var ( totalSlots = metric.Metadata{ Name: "admission.granter.total_slots.kv", From 9e7460dc3a100e075a34876f42f8b6dc301bc889 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Sat, 13 Aug 2022 00:53:12 -0400 Subject: [PATCH 7/9] admission: move store{AdmissionStats,RequestEstimates}.. ..into work_queue.go, where they're primarily used. This feels more suitable than granter.go. Release note: None --- pkg/util/admission/granter.go | 45 -------------------------------- pkg/util/admission/work_queue.go | 45 ++++++++++++++++++++++++++++++++ 2 files changed, 45 insertions(+), 45 deletions(-) diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index b4ed67a87f48..dec7b0cc19d2 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -666,51 +666,6 @@ type DiskStats struct { ProvisionedBandwidth int64 } -// storeAdmissionStats are stats maintained by a storeRequester. The non-test -// implementation of storeRequester is StoreWorkQueue. StoreWorkQueue updates -// all of these when StoreWorkQueue.AdmittedWorkDone is called, so that these -// cumulative values are mutually consistent. -type storeAdmissionStats struct { - // Total requests that called AdmittedWorkDone or BypassedWorkDone. - admittedCount uint64 - // Sum of StoreWorkDoneInfo.WriteBytes. - // - // TODO(sumeer): writeAccountedBytes and ingestedAccountedBytes are not - // actually comparable, since the former is uncompressed. We may need to fix - // this inaccuracy if it turns out to be an issue. - writeAccountedBytes uint64 - // Sum of StoreWorkDoneInfo.IngestedBytes. - ingestedAccountedBytes uint64 - // statsToIgnore represents stats that we should exclude from token - // consumption, and estimation of per-work-tokens. Currently, this is - // limited to range snapshot ingestion. These are likely to usually land in - // levels lower than L0, so may not fit the existing per-work-tokens model - // well. Additionally, we do not want large range snapshots to consume a - // huge number of tokens (see - // https://github.com/cockroachdb/cockroach/pull/80914 for justification -- - // that PR is closer to the final solution, and this is a step in that - // direction). - statsToIgnore struct { - pebble.IngestOperationStats - } - // aux represents additional information carried for informational purposes - // (e.g. for logging). - aux struct { - // These bypassed numbers are already included in the corresponding - // {admittedCount, writeAccountedBytes, ingestedAccountedBytes}. - bypassedCount uint64 - writeBypassedAccountedBytes uint64 - ingestedBypassedAccountedBytes uint64 - } -} - -// storeRequestEstimates are estimates that the storeRequester should use for -// its future requests. -type storeRequestEstimates struct { - // writeTokens is the tokens to request at admission time. Must be > 0. - writeTokens int64 -} - var ( totalSlots = metric.Metadata{ Name: "admission.granter.total_slots.kv", diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index c51e42167bc4..0c038e92a994 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -1557,6 +1557,51 @@ type StoreWriteWorkInfo struct { // information at proposal time, and may be able to use it fruitfully. } +// storeAdmissionStats are stats maintained by a storeRequester. The non-test +// implementation of storeRequester is StoreWorkQueue. StoreWorkQueue updates +// all of these when StoreWorkQueue.AdmittedWorkDone is called, so that these +// cumulative values are mutually consistent. +type storeAdmissionStats struct { + // Total requests that called AdmittedWorkDone or BypassedWorkDone. + admittedCount uint64 + // Sum of StoreWorkDoneInfo.WriteBytes. + // + // TODO(sumeer): writeAccountedBytes and ingestedAccountedBytes are not + // actually comparable, since the former is uncompressed. We may need to fix + // this inaccuracy if it turns out to be an issue. + writeAccountedBytes uint64 + // Sum of StoreWorkDoneInfo.IngestedBytes. + ingestedAccountedBytes uint64 + // statsToIgnore represents stats that we should exclude from token + // consumption, and estimation of per-work-tokens. Currently, this is + // limited to range snapshot ingestion. These are likely to usually land in + // levels lower than L0, so may not fit the existing per-work-tokens model + // well. Additionally, we do not want large range snapshots to consume a + // huge number of tokens (see + // https://github.com/cockroachdb/cockroach/pull/80914 for justification -- + // that PR is closer to the final solution, and this is a step in that + // direction). + statsToIgnore struct { + pebble.IngestOperationStats + } + // aux represents additional information carried for informational purposes + // (e.g. for logging). + aux struct { + // These bypassed numbers are already included in the corresponding + // {admittedCount, writeAccountedBytes, ingestedAccountedBytes}. + bypassedCount uint64 + writeBypassedAccountedBytes uint64 + ingestedBypassedAccountedBytes uint64 + } +} + +// storeRequestEstimates are estimates that the storeRequester should use for +// its future requests. +type storeRequestEstimates struct { + // writeTokens is the tokens to request at admission time. Must be > 0. + writeTokens int64 +} + // StoreWorkQueue is responsible for admission to a store. type StoreWorkQueue struct { q [numWorkClasses]WorkQueue From d186c592e6c1ea69a2cf96af6dd9464de12c8db7 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 19 Aug 2022 10:13:55 -0400 Subject: [PATCH 8/9] admission: move store{AdmissionStatsRequestEstimates} Pure code movement. They're part of the interface definition, so better placed in the top-level admission.go housing the remaining interfaces. Release note: None --- pkg/util/admission/admission.go | 46 ++++++++++++++++++++++++++++++++ pkg/util/admission/work_queue.go | 45 ------------------------------- 2 files changed, 46 insertions(+), 45 deletions(-) diff --git a/pkg/util/admission/admission.go b/pkg/util/admission/admission.go index 01ea2e7e4ce7..ba1c78318806 100644 --- a/pkg/util/admission/admission.go +++ b/pkg/util/admission/admission.go @@ -132,6 +132,7 @@ import ( "time" "github.com/cockroachdb/errors" + "github.com/cockroachdb/pebble" "github.com/cockroachdb/redact" ) @@ -498,3 +499,48 @@ func workKindString(workKind WorkKind) redact.RedactableString { panic(errors.AssertionFailedf("unknown WorkKind")) } } + +// storeAdmissionStats are stats maintained by a storeRequester. The non-test +// implementation of storeRequester is StoreWorkQueue. StoreWorkQueue updates +// all of these when StoreWorkQueue.AdmittedWorkDone is called, so that these +// cumulative values are mutually consistent. +type storeAdmissionStats struct { + // Total requests that called AdmittedWorkDone or BypassedWorkDone. + admittedCount uint64 + // Sum of StoreWorkDoneInfo.WriteBytes. + // + // TODO(sumeer): writeAccountedBytes and ingestedAccountedBytes are not + // actually comparable, since the former is uncompressed. We may need to fix + // this inaccuracy if it turns out to be an issue. + writeAccountedBytes uint64 + // Sum of StoreWorkDoneInfo.IngestedBytes. + ingestedAccountedBytes uint64 + // statsToIgnore represents stats that we should exclude from token + // consumption, and estimation of per-work-tokens. Currently, this is + // limited to range snapshot ingestion. These are likely to usually land in + // levels lower than L0, so may not fit the existing per-work-tokens model + // well. Additionally, we do not want large range snapshots to consume a + // huge number of tokens (see + // https://github.com/cockroachdb/cockroach/pull/80914 for justification -- + // that PR is closer to the final solution, and this is a step in that + // direction). + statsToIgnore struct { + pebble.IngestOperationStats + } + // aux represents additional information carried for informational purposes + // (e.g. for logging). + aux struct { + // These bypassed numbers are already included in the corresponding + // {admittedCount, writeAccountedBytes, ingestedAccountedBytes}. + bypassedCount uint64 + writeBypassedAccountedBytes uint64 + ingestedBypassedAccountedBytes uint64 + } +} + +// storeRequestEstimates are estimates that the storeRequester should use for +// its future requests. +type storeRequestEstimates struct { + // writeTokens is the tokens to request at admission time. Must be > 0. + writeTokens int64 +} diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index 0c038e92a994..c51e42167bc4 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -1557,51 +1557,6 @@ type StoreWriteWorkInfo struct { // information at proposal time, and may be able to use it fruitfully. } -// storeAdmissionStats are stats maintained by a storeRequester. The non-test -// implementation of storeRequester is StoreWorkQueue. StoreWorkQueue updates -// all of these when StoreWorkQueue.AdmittedWorkDone is called, so that these -// cumulative values are mutually consistent. -type storeAdmissionStats struct { - // Total requests that called AdmittedWorkDone or BypassedWorkDone. - admittedCount uint64 - // Sum of StoreWorkDoneInfo.WriteBytes. - // - // TODO(sumeer): writeAccountedBytes and ingestedAccountedBytes are not - // actually comparable, since the former is uncompressed. We may need to fix - // this inaccuracy if it turns out to be an issue. - writeAccountedBytes uint64 - // Sum of StoreWorkDoneInfo.IngestedBytes. - ingestedAccountedBytes uint64 - // statsToIgnore represents stats that we should exclude from token - // consumption, and estimation of per-work-tokens. Currently, this is - // limited to range snapshot ingestion. These are likely to usually land in - // levels lower than L0, so may not fit the existing per-work-tokens model - // well. Additionally, we do not want large range snapshots to consume a - // huge number of tokens (see - // https://github.com/cockroachdb/cockroach/pull/80914 for justification -- - // that PR is closer to the final solution, and this is a step in that - // direction). - statsToIgnore struct { - pebble.IngestOperationStats - } - // aux represents additional information carried for informational purposes - // (e.g. for logging). - aux struct { - // These bypassed numbers are already included in the corresponding - // {admittedCount, writeAccountedBytes, ingestedAccountedBytes}. - bypassedCount uint64 - writeBypassedAccountedBytes uint64 - ingestedBypassedAccountedBytes uint64 - } -} - -// storeRequestEstimates are estimates that the storeRequester should use for -// its future requests. -type storeRequestEstimates struct { - // writeTokens is the tokens to request at admission time. Must be > 0. - writeTokens int64 -} - // StoreWorkQueue is responsible for admission to a store. type StoreWorkQueue struct { q [numWorkClasses]WorkQueue From 88ccf28bdccf9a2c9dd42842404777169906363a Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 19 Aug 2022 10:24:20 -0400 Subject: [PATCH 9/9] admission: segment test files by central type Pure code movement change. Carve out {io_load_listener,tokens_linear_model}_test.go using the same non-test partitioning scheme as earlier commits. Within granter_test.go, move the supporting types further below in the file. Release note: None --- pkg/util/admission/BUILD.bazel | 2 + pkg/util/admission/granter_test.go | 578 +++--------------- pkg/util/admission/io_load_listener_test.go | 419 +++++++++++++ .../admission/store_token_estimation_test.go | 59 -- .../admission/tokens_linear_model_test.go | 82 +++ 5 files changed, 595 insertions(+), 545 deletions(-) create mode 100644 pkg/util/admission/io_load_listener_test.go create mode 100644 pkg/util/admission/tokens_linear_model_test.go diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index a866798b45ec..905630f4263a 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -40,7 +40,9 @@ go_test( srcs = [ "disk_bandwidth_test.go", "granter_test.go", + "io_load_listener_test.go", "store_token_estimation_test.go", + "tokens_linear_model_test.go", "work_queue_test.go", ], data = glob(["testdata/**"]), diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index cd41ac5450fe..78033c6e59d9 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -13,9 +13,6 @@ package admission import ( "context" "fmt" - "math" - "math/rand" - "path/filepath" "sort" "strings" "testing" @@ -25,100 +22,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/datadriven" "github.com/cockroachdb/pebble" - "github.com/cockroachdb/redact" "github.com/stretchr/testify/require" ) -type testRequester struct { - workKind WorkKind - additionalID string - granter granter - usesTokens bool - buf *strings.Builder - - waitingRequests bool - returnValueFromGranted int64 - grantChainID grantChainID -} - -var _ requester = &testRequester{} - -func (tr *testRequester) hasWaitingRequests() bool { - return tr.waitingRequests -} - -func (tr *testRequester) granted(grantChainID grantChainID) int64 { - fmt.Fprintf(tr.buf, "%s%s: granted in chain %d, and returning %d\n", - workKindString(tr.workKind), tr.additionalID, - grantChainID, tr.returnValueFromGranted) - tr.grantChainID = grantChainID - return tr.returnValueFromGranted -} - -func (tr *testRequester) close() {} - -func (tr *testRequester) tryGet(count int64) { - rv := tr.granter.tryGet(count) - fmt.Fprintf(tr.buf, "%s%s: tryGet(%d) returned %t\n", workKindString(tr.workKind), - tr.additionalID, count, rv) -} - -func (tr *testRequester) returnGrant(count int64) { - fmt.Fprintf(tr.buf, "%s%s: returnGrant(%d)\n", workKindString(tr.workKind), tr.additionalID, - count) - tr.granter.returnGrant(count) -} - -func (tr *testRequester) tookWithoutPermission(count int64) { - fmt.Fprintf(tr.buf, "%s%s: tookWithoutPermission(%d)\n", workKindString(tr.workKind), - tr.additionalID, count) - tr.granter.tookWithoutPermission(count) -} - -func (tr *testRequester) continueGrantChain() { - fmt.Fprintf(tr.buf, "%s%s: continueGrantChain\n", workKindString(tr.workKind), - tr.additionalID) - tr.granter.continueGrantChain(tr.grantChainID) -} - -type storeTestRequester struct { - requesters [numWorkClasses]*testRequester -} - -var _ storeRequester = &storeTestRequester{} - -func (str *storeTestRequester) getRequesters() [numWorkClasses]requester { - var rv [numWorkClasses]requester - for i := range str.requesters { - rv[i] = str.requesters[i] - } - return rv -} - -func (str *storeTestRequester) close() {} - -func (str *storeTestRequester) getStoreAdmissionStats() storeAdmissionStats { - // Only used by ioLoadListener, so don't bother. - return storeAdmissionStats{} -} - -func (str *storeTestRequester) setStoreRequestEstimates(estimates storeRequestEstimates) { - // Only used by ioLoadListener, so don't bother. -} - -// setModerateSlotsClamp is used in testing to force a value for kvsa.moderateSlotsClamp. -func (kvsa *kvSlotAdjuster) setModerateSlotsClamp(val int) { - kvsa.moderateSlotsClampOverride = val - kvsa.moderateSlotsClamp = val -} - // TestGranterBasic is a datadriven test with the following commands: // // init-grant-coordinator min-cpu= max-cpu= sql-kv-tokens= @@ -240,9 +151,9 @@ func TestGranterBasic(t *testing.T) { workQueueMetrics: makeWorkQueueMetrics(""), disableTickerForTesting: true, } - var testMetricsProvider testMetricsProvider - testMetricsProvider.setMetricsForStores([]int32{1}, pebble.Metrics{}) - storeCoordinators.SetPebbleMetricsProvider(context.Background(), &testMetricsProvider, &testMetricsProvider) + var metricsProvider testMetricsProvider + metricsProvider.setMetricsForStores([]int32{1}, pebble.Metrics{}) + storeCoordinators.SetPebbleMetricsProvider(context.Background(), &metricsProvider, &metricsProvider) unsafeGranter, ok := storeCoordinators.gcMap.Load(int64(1)) require.True(t, ok) coord = (*GrantCoordinator)(unsafeGranter) @@ -367,49 +278,6 @@ func TestGranterBasic(t *testing.T) { }) } -func scanWorkKind(t *testing.T, d *datadriven.TestData) int8 { - var kindStr string - d.ScanArgs(t, "work", &kindStr) - switch kindStr { - case "kv": - return int8(KVWork) - case "sql-kv-response": - return int8(SQLKVResponseWork) - case "sql-sql-response": - return int8(SQLSQLResponseWork) - case "sql-leaf-start": - return int8(SQLStatementLeafStartWork) - case "sql-root-start": - return int8(SQLStatementRootStartWork) - case "kv-elastic": - return int8(numWorkKinds) - } - panic("unknown WorkKind") -} - -type testMetricsProvider struct { - metrics []StoreMetrics -} - -func (m *testMetricsProvider) GetPebbleMetrics() []StoreMetrics { - return m.metrics -} - -func (m *testMetricsProvider) UpdateIOThreshold( - id roachpb.StoreID, threshold *admissionpb.IOThreshold, -) { -} - -func (m *testMetricsProvider) setMetricsForStores(stores []int32, metrics pebble.Metrics) { - m.metrics = m.metrics[:0] - for _, s := range stores { - m.metrics = append(m.metrics, StoreMetrics{ - StoreID: s, - Metrics: &metrics, - }) - } -} - // TestStoreCoordinators tests only the setup of GrantCoordinators per store. // Testing of IO load functionality happens in TestIOLoadListener. func TestStoreCoordinators(t *testing.T) { @@ -493,394 +361,132 @@ func TestStoreCoordinators(t *testing.T) { coords.Close() } -type testRequesterForIOLL struct { - stats storeAdmissionStats - buf strings.Builder -} - -var _ storeRequester = &testRequesterForIOLL{} - -func (r *testRequesterForIOLL) close() {} +type testRequester struct { + workKind WorkKind + additionalID string + granter granter + usesTokens bool + buf *strings.Builder -func (r *testRequesterForIOLL) getRequesters() [numWorkClasses]requester { - panic("unimplemented") + waitingRequests bool + returnValueFromGranted int64 + grantChainID grantChainID } -func (r *testRequesterForIOLL) getStoreAdmissionStats() storeAdmissionStats { - return r.stats -} +var _ requester = &testRequester{} -func (r *testRequesterForIOLL) setStoreRequestEstimates(estimates storeRequestEstimates) { - fmt.Fprintf(&r.buf, "store-request-estimates: writeTokens: %d", estimates.writeTokens) +func (tr *testRequester) hasWaitingRequests() bool { + return tr.waitingRequests } -type testGranterWithIOTokens struct { - buf strings.Builder - allTokensUsed bool - diskBandwidthTokensUsed [numWorkClasses]int64 +func (tr *testRequester) granted(grantChainID grantChainID) int64 { + fmt.Fprintf(tr.buf, "%s%s: granted in chain %d, and returning %d\n", + workKindString(tr.workKind), tr.additionalID, + grantChainID, tr.returnValueFromGranted) + tr.grantChainID = grantChainID + return tr.returnValueFromGranted } -var _ granterWithIOTokens = &testGranterWithIOTokens{} +func (tr *testRequester) close() {} -func (g *testGranterWithIOTokens) setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) { - fmt.Fprintf(&g.buf, "setAvailableIOTokens: %s", tokensForTokenTickDurationToString(tokens)) - if g.allTokensUsed { - return tokens * 2 - } - return 0 +func (tr *testRequester) tryGet(count int64) { + rv := tr.granter.tryGet(count) + fmt.Fprintf(tr.buf, "%s%s: tryGet(%d) returned %t\n", workKindString(tr.workKind), + tr.additionalID, count, rv) } -func (g *testGranterWithIOTokens) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) { - fmt.Fprintf(&g.buf, " setAvailableElasticDiskTokens: %s", - tokensForTokenTickDurationToString(tokens)) +func (tr *testRequester) returnGrant(count int64) { + fmt.Fprintf(tr.buf, "%s%s: returnGrant(%d)\n", workKindString(tr.workKind), tr.additionalID, + count) + tr.granter.returnGrant(count) } -func (g *testGranterWithIOTokens) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 { - return g.diskBandwidthTokensUsed +func (tr *testRequester) tookWithoutPermission(count int64) { + fmt.Fprintf(tr.buf, "%s%s: tookWithoutPermission(%d)\n", workKindString(tr.workKind), + tr.additionalID, count) + tr.granter.tookWithoutPermission(count) } -func (g *testGranterWithIOTokens) setAdmittedDoneModelsLocked( - l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, -) { - fmt.Fprintf(&g.buf, "setAdmittedDoneModelsLocked: l0-write-lm: ") - printLinearModel(&g.buf, l0WriteLM) - fmt.Fprintf(&g.buf, " l0-ingest-lm: ") - printLinearModel(&g.buf, l0IngestLM) - fmt.Fprintf(&g.buf, " ingest-lm: ") - printLinearModel(&g.buf, ingestLM) - fmt.Fprintf(&g.buf, "\n") +func (tr *testRequester) continueGrantChain() { + fmt.Fprintf(tr.buf, "%s%s: continueGrantChain\n", workKindString(tr.workKind), + tr.additionalID) + tr.granter.continueGrantChain(tr.grantChainID) } -func tokensForTokenTickDurationToString(tokens int64) string { - if tokens >= unlimitedTokens/ticksInAdjustmentInterval { - return "unlimited" - } - return fmt.Sprintf("%d", tokens) +type storeTestRequester struct { + requesters [numWorkClasses]*testRequester } -type rawTokenResult adjustTokensResult - -// TestIOLoadListener is a datadriven test with the following command that -// sets the state for token calculation and then ticks adjustmentInterval -// times to cause tokens to be set in the testGranterWithIOTokens: -// set-state admitted= l0-bytes= l0-added= l0-files= l0-sublevels= ... -func TestIOLoadListener(t *testing.T) { - req := &testRequesterForIOLL{} - kvGranter := &testGranterWithIOTokens{} - var ioll *ioLoadListener - ctx := context.Background() - st := cluster.MakeTestingClusterSettings() - datadriven.RunTest(t, testutils.TestDataPath(t, "io_load_listener"), - func(t *testing.T, d *datadriven.TestData) string { - switch d.Cmd { - case "init": - ioll = &ioLoadListener{ - settings: st, - kvRequester: req, - perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), - diskBandwidthLimiter: makeDiskBandwidthLimiter(), - } - // The mutex is needed by ioLoadListener but is not useful in this - // test -- the channels provide synchronization and prevent this - // test code and the ioLoadListener from being concurrently - // active. - ioll.mu.Mutex = &syncutil.Mutex{} - ioll.mu.kvGranter = kvGranter - return "" - - case "prep-admission-stats": - req.stats = storeAdmissionStats{ - admittedCount: 0, - writeAccountedBytes: 0, - ingestedAccountedBytes: 0, - } - d.ScanArgs(t, "admitted", &req.stats.admittedCount) - if d.HasArg("write-bytes") { - d.ScanArgs(t, "write-bytes", &req.stats.writeAccountedBytes) - } - if d.HasArg("ingested-bytes") { - d.ScanArgs(t, "ingested-bytes", &req.stats.ingestedAccountedBytes) - } - return fmt.Sprintf("%+v", req.stats) - - case "set-min-flush-util": - var percent int - d.ScanArgs(t, "percent", &percent) - MinFlushUtilizationFraction.Override(ctx, &st.SV, float64(percent)/100) - return "" - - // TODO(sumeer): the output printed by set-state is hard to follow. It - // prints the internal fields which are hard to interpret, and it prints - // a properly formatted ioLoadListenerState. The latter is supposed to - // be easier to understand, but reviewers have noted that it is still - // challenging to understand whether the output is correct. Come up with - // more easily consumable output. Additionally, the input uses - // cumulative values, so one has to look at the preceding testdata -- we - // could instead accept the interval delta as input. - case "set-state": - // Setup state used as input for token adjustment. - var metrics pebble.Metrics - var l0Bytes uint64 - d.ScanArgs(t, "l0-bytes", &l0Bytes) - metrics.Levels[0].Size = int64(l0Bytes) - var l0AddedWrite, l0AddedIngested uint64 - d.ScanArgs(t, "l0-added-write", &l0AddedWrite) - metrics.Levels[0].BytesFlushed = l0AddedWrite - if d.HasArg("l0-added-ingested") { - d.ScanArgs(t, "l0-added-ingested", &l0AddedIngested) - } - metrics.Levels[0].BytesIngested = l0AddedIngested - var l0Files int - d.ScanArgs(t, "l0-files", &l0Files) - metrics.Levels[0].NumFiles = int64(l0Files) - var l0SubLevels int - d.ScanArgs(t, "l0-sublevels", &l0SubLevels) - metrics.Levels[0].Sublevels = int32(l0SubLevels) - var flushBytes, flushWorkSec, flushIdleSec int - if d.HasArg("flush-bytes") { - d.ScanArgs(t, "flush-bytes", &flushBytes) - d.ScanArgs(t, "flush-work-sec", &flushWorkSec) - d.ScanArgs(t, "flush-idle-sec", &flushIdleSec) - } - flushMetric := pebble.ThroughputMetric{ - Bytes: int64(flushBytes), - WorkDuration: time.Duration(flushWorkSec) * time.Second, - IdleDuration: time.Duration(flushIdleSec) * time.Second, - } - im := &pebble.InternalIntervalMetrics{} - im.Flush.WriteThroughput = flushMetric - var writeStallCount int - if d.HasArg("write-stall-count") { - d.ScanArgs(t, "write-stall-count", &writeStallCount) - } - var allTokensUsed bool - if d.HasArg("all-tokens-used") { - d.ScanArgs(t, "all-tokens-used", &allTokensUsed) - } - kvGranter.allTokensUsed = allTokensUsed - var provisionedBandwidth, bytesRead, bytesWritten int - if d.HasArg("provisioned-bandwidth") { - d.ScanArgs(t, "provisioned-bandwidth", &provisionedBandwidth) - } - if d.HasArg("bytes-read") { - d.ScanArgs(t, "bytes-read", &bytesRead) - } - if d.HasArg("bytes-written") { - d.ScanArgs(t, "bytes-written", &bytesWritten) - } - if d.HasArg("disk-bw-tokens-used") { - var regularTokensUsed, elasticTokensUsed int - d.ScanArgs(t, "disk-bw-tokens-used", ®ularTokensUsed, &elasticTokensUsed) - kvGranter.diskBandwidthTokensUsed[regularWorkClass] = int64(regularTokensUsed) - kvGranter.diskBandwidthTokensUsed[elasticWorkClass] = int64(elasticTokensUsed) - } else { - kvGranter.diskBandwidthTokensUsed[regularWorkClass] = 0 - kvGranter.diskBandwidthTokensUsed[elasticWorkClass] = 0 - } - var printOnlyFirstTick bool - if d.HasArg("print-only-first-tick") { - d.ScanArgs(t, "print-only-first-tick", &printOnlyFirstTick) - } - ioll.pebbleMetricsTick(ctx, StoreMetrics{ - Metrics: &metrics, - WriteStallCount: int64(writeStallCount), - InternalIntervalMetrics: im, - DiskStats: DiskStats{ - BytesRead: uint64(bytesRead), - BytesWritten: uint64(bytesWritten), - ProvisionedBandwidth: int64(provisionedBandwidth), - }, - }) - var buf strings.Builder - // Do the ticks until just before next adjustment. - res := ioll.adjustTokensResult - fmt.Fprintln(&buf, redact.StringWithoutMarkers(&res)) - res.ioThreshold = nil // avoid nondeterminism - fmt.Fprintf(&buf, "%+v\n", (rawTokenResult)(res)) - if req.buf.Len() > 0 { - fmt.Fprintf(&buf, "%s\n", req.buf.String()) - req.buf.Reset() - } - for i := 0; i < ticksInAdjustmentInterval; i++ { - ioll.allocateTokensTick() - if i == 0 || !printOnlyFirstTick { - fmt.Fprintf(&buf, "tick: %d, %s\n", i, kvGranter.buf.String()) - } - kvGranter.buf.Reset() - } - return buf.String() - - default: - return fmt.Sprintf("unknown command: %s", d.Cmd) - } - }) -} +var _ storeRequester = &storeTestRequester{} -func TestIOLoadListenerOverflow(t *testing.T) { - req := &testRequesterForIOLL{} - kvGranter := &testGranterWithIOTokens{} - ctx := context.Background() - st := cluster.MakeTestingClusterSettings() - ioll := ioLoadListener{ - settings: st, - kvRequester: req, - } - ioll.mu.Mutex = &syncutil.Mutex{} - ioll.mu.kvGranter = kvGranter - // Bug 1: overflow when totalNumByteTokens is too large. - for i := int64(0); i < adjustmentInterval; i++ { - // Override the totalNumByteTokens manually to trigger the overflow bug. - ioll.totalNumByteTokens = math.MaxInt64 - i - ioll.byteTokensAllocated = 0 - for j := 0; j < ticksInAdjustmentInterval; j++ { - ioll.allocateTokensTick() - } - } - // Bug2: overflow when bytes added delta is 0. - m := pebble.Metrics{} - m.Levels[0] = pebble.LevelMetrics{ - Sublevels: 100, - NumFiles: 10000, +func (str *storeTestRequester) getRequesters() [numWorkClasses]requester { + var rv [numWorkClasses]requester + for i := range str.requesters { + rv[i] = str.requesters[i] } - ioll.pebbleMetricsTick(ctx, - StoreMetrics{Metrics: &m, InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}}) - ioll.pebbleMetricsTick(ctx, - StoreMetrics{Metrics: &m, InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}}) - ioll.allocateTokensTick() -} - -type testGranterNonNegativeTokens struct { - t *testing.T + return rv } -var _ granterWithIOTokens = &testGranterNonNegativeTokens{} - -func (g *testGranterNonNegativeTokens) setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) { - require.LessOrEqual(g.t, int64(0), tokens) - return 0 -} +func (str *storeTestRequester) close() {} -func (g *testGranterNonNegativeTokens) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) { - require.LessOrEqual(g.t, int64(0), tokens) +func (str *storeTestRequester) getStoreAdmissionStats() storeAdmissionStats { + // Only used by ioLoadListener, so don't bother. + return storeAdmissionStats{} } -func (g *testGranterNonNegativeTokens) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 { - return [numWorkClasses]int64{} +func (str *storeTestRequester) setStoreRequestEstimates(estimates storeRequestEstimates) { + // Only used by ioLoadListener, so don't bother. } -func (g *testGranterNonNegativeTokens) setAdmittedDoneModelsLocked( - l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, -) { - require.LessOrEqual(g.t, 0.5, l0WriteLM.multiplier) - require.LessOrEqual(g.t, int64(0), l0WriteLM.constant) - require.Less(g.t, 0.0, l0IngestLM.multiplier) - require.LessOrEqual(g.t, int64(0), l0IngestLM.constant) - require.LessOrEqual(g.t, 0.5, ingestLM.multiplier) - require.LessOrEqual(g.t, int64(0), ingestLM.constant) +// setModerateSlotsClamp is used in testing to force a value for kvsa.moderateSlotsClamp. +func (kvsa *kvSlotAdjuster) setModerateSlotsClamp(val int) { + kvsa.moderateSlotsClampOverride = val + kvsa.moderateSlotsClamp = val } -// TODO(sumeer): we now do more work outside adjustTokensInner, so the parts -// of the adjustTokensResult computed by adjustTokensInner has become a subset -// of what is logged below, and the rest is logged with 0 values. Expand this -// test to call adjustTokens. -func TestAdjustTokensInnerAndLogging(t *testing.T) { - const mb = 12 + 1<<20 - tests := []struct { - name redact.SafeString - prev ioLoadListenerState - l0Metrics pebble.LevelMetrics - }{ - { - name: "zero", - }, - { - name: "real-numbers", - prev: ioLoadListenerState{ - cumL0AddedBytes: 1402 * mb, - curL0Bytes: 400 * mb, - cumWriteStallCount: 10, - smoothedIntL0CompactedBytes: 47 * mb, - smoothedCompactionByteTokens: 201 * mb, - totalNumByteTokens: int64(201 * mb), - }, - l0Metrics: pebble.LevelMetrics{ - Sublevels: 27, - NumFiles: 195, - Size: 900 * mb, - BytesIngested: 1801 * mb, - BytesFlushed: 178 * mb, - }, - }, - } - ctx := context.Background() - var buf redact.StringBuilder - for _, tt := range tests { - buf.Printf("%s:\n", tt.name) - res := (*ioLoadListener)(nil).adjustTokensInner( - ctx, tt.prev, tt.l0Metrics, 12, &pebble.InternalIntervalMetrics{}, - 100, 10, 0.50) - buf.Printf("%s\n", res) +func scanWorkKind(t *testing.T, d *datadriven.TestData) int8 { + var kindStr string + d.ScanArgs(t, "work", &kindStr) + switch kindStr { + case "kv": + return int8(KVWork) + case "sql-kv-response": + return int8(SQLKVResponseWork) + case "sql-sql-response": + return int8(SQLSQLResponseWork) + case "sql-leaf-start": + return int8(SQLStatementLeafStartWork) + case "sql-root-start": + return int8(SQLStatementRootStartWork) + case "kv-elastic": + return int8(numWorkKinds) } - echotest.Require(t, string(redact.Sprint(buf)), filepath.Join(testutils.TestDataPath(t, "format_adjust_tokens_stats.txt"))) + panic("unknown WorkKind") } // TODO(sumeer): // - Test metrics // - Test GrantCoordinator with multi-tenant configurations -// TestBadIOLoadListenerStats tests that bad stats (non-monotonic cumulative -// stats and negative values) don't cause panics or tokens to be negative. -func TestBadIOLoadListenerStats(t *testing.T) { - var m pebble.Metrics - var d DiskStats - req := &testRequesterForIOLL{} - ctx := context.Background() - - randomValues := func() { - // Use uints, and cast so that we get bad negative values. - m.Levels[0].Sublevels = int32(rand.Uint32()) - m.Levels[0].NumFiles = int64(rand.Uint64()) - m.Levels[0].Size = int64(rand.Uint64()) - m.Levels[0].BytesFlushed = rand.Uint64() - for i := range m.Levels { - m.Levels[i].BytesIngested = rand.Uint64() - } - d.BytesRead = rand.Uint64() - d.BytesWritten = rand.Uint64() - d.ProvisionedBandwidth = 1 << 20 - req.stats.admittedCount = rand.Uint64() - req.stats.writeAccountedBytes = rand.Uint64() - req.stats.ingestedAccountedBytes = rand.Uint64() - req.stats.statsToIgnore.Bytes = rand.Uint64() - req.stats.statsToIgnore.ApproxIngestedIntoL0Bytes = rand.Uint64() - } - kvGranter := &testGranterNonNegativeTokens{t: t} - st := cluster.MakeTestingClusterSettings() - ioll := ioLoadListener{ - settings: st, - kvRequester: req, - perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), - diskBandwidthLimiter: makeDiskBandwidthLimiter(), - } - ioll.mu.Mutex = &syncutil.Mutex{} - ioll.mu.kvGranter = kvGranter - for i := 0; i < 100; i++ { - randomValues() - ioll.pebbleMetricsTick(ctx, StoreMetrics{ - Metrics: &m, - InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}, - DiskStats: d, +type testMetricsProvider struct { + metrics []StoreMetrics +} + +func (m *testMetricsProvider) GetPebbleMetrics() []StoreMetrics { + return m.metrics +} + +func (m *testMetricsProvider) UpdateIOThreshold( + id roachpb.StoreID, threshold *admissionpb.IOThreshold, +) { +} + +func (m *testMetricsProvider) setMetricsForStores(stores []int32, metrics pebble.Metrics) { + m.metrics = m.metrics[:0] + for _, s := range stores { + m.metrics = append(m.metrics, StoreMetrics{ + StoreID: s, + Metrics: &metrics, }) - for j := 0; j < ticksInAdjustmentInterval; j++ { - ioll.allocateTokensTick() - require.LessOrEqual(t, int64(0), ioll.smoothedIntL0CompactedBytes) - require.LessOrEqual(t, float64(0), ioll.smoothedCompactionByteTokens) - require.LessOrEqual(t, float64(0), ioll.smoothedNumFlushTokens) - require.LessOrEqual(t, float64(0), ioll.flushUtilTargetFraction) - require.LessOrEqual(t, int64(0), ioll.totalNumByteTokens) - require.LessOrEqual(t, int64(0), ioll.byteTokensAllocated) - require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokens) - require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokensAllocated) - } } } diff --git a/pkg/util/admission/io_load_listener_test.go b/pkg/util/admission/io_load_listener_test.go new file mode 100644 index 000000000000..98042822a3c0 --- /dev/null +++ b/pkg/util/admission/io_load_listener_test.go @@ -0,0 +1,419 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package admission + +import ( + "context" + "fmt" + "math" + "math/rand" + "path/filepath" + "strings" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/pebble" + "github.com/cockroachdb/redact" + "github.com/stretchr/testify/require" +) + +// TestIOLoadListener is a datadriven test with the following command that +// sets the state for token calculation and then ticks adjustmentInterval +// times to cause tokens to be set in the testGranterWithIOTokens: +// set-state admitted= l0-bytes= l0-added= l0-files= l0-sublevels= ... +func TestIOLoadListener(t *testing.T) { + req := &testRequesterForIOLL{} + kvGranter := &testGranterWithIOTokens{} + var ioll *ioLoadListener + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + datadriven.RunTest(t, testutils.TestDataPath(t, "io_load_listener"), + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + ioll = &ioLoadListener{ + settings: st, + kvRequester: req, + perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), + diskBandwidthLimiter: makeDiskBandwidthLimiter(), + } + // The mutex is needed by ioLoadListener but is not useful in this + // test -- the channels provide synchronization and prevent this + // test code and the ioLoadListener from being concurrently + // active. + ioll.mu.Mutex = &syncutil.Mutex{} + ioll.mu.kvGranter = kvGranter + return "" + + case "prep-admission-stats": + req.stats = storeAdmissionStats{ + admittedCount: 0, + writeAccountedBytes: 0, + ingestedAccountedBytes: 0, + } + d.ScanArgs(t, "admitted", &req.stats.admittedCount) + if d.HasArg("write-bytes") { + d.ScanArgs(t, "write-bytes", &req.stats.writeAccountedBytes) + } + if d.HasArg("ingested-bytes") { + d.ScanArgs(t, "ingested-bytes", &req.stats.ingestedAccountedBytes) + } + return fmt.Sprintf("%+v", req.stats) + + case "set-min-flush-util": + var percent int + d.ScanArgs(t, "percent", &percent) + MinFlushUtilizationFraction.Override(ctx, &st.SV, float64(percent)/100) + return "" + + // TODO(sumeer): the output printed by set-state is hard to follow. It + // prints the internal fields which are hard to interpret, and it prints + // a properly formatted ioLoadListenerState. The latter is supposed to + // be easier to understand, but reviewers have noted that it is still + // challenging to understand whether the output is correct. Come up with + // more easily consumable output. Additionally, the input uses + // cumulative values, so one has to look at the preceding testdata -- we + // could instead accept the interval delta as input. + case "set-state": + // Setup state used as input for token adjustment. + var metrics pebble.Metrics + var l0Bytes uint64 + d.ScanArgs(t, "l0-bytes", &l0Bytes) + metrics.Levels[0].Size = int64(l0Bytes) + var l0AddedWrite, l0AddedIngested uint64 + d.ScanArgs(t, "l0-added-write", &l0AddedWrite) + metrics.Levels[0].BytesFlushed = l0AddedWrite + if d.HasArg("l0-added-ingested") { + d.ScanArgs(t, "l0-added-ingested", &l0AddedIngested) + } + metrics.Levels[0].BytesIngested = l0AddedIngested + var l0Files int + d.ScanArgs(t, "l0-files", &l0Files) + metrics.Levels[0].NumFiles = int64(l0Files) + var l0SubLevels int + d.ScanArgs(t, "l0-sublevels", &l0SubLevels) + metrics.Levels[0].Sublevels = int32(l0SubLevels) + var flushBytes, flushWorkSec, flushIdleSec int + if d.HasArg("flush-bytes") { + d.ScanArgs(t, "flush-bytes", &flushBytes) + d.ScanArgs(t, "flush-work-sec", &flushWorkSec) + d.ScanArgs(t, "flush-idle-sec", &flushIdleSec) + } + flushMetric := pebble.ThroughputMetric{ + Bytes: int64(flushBytes), + WorkDuration: time.Duration(flushWorkSec) * time.Second, + IdleDuration: time.Duration(flushIdleSec) * time.Second, + } + im := &pebble.InternalIntervalMetrics{} + im.Flush.WriteThroughput = flushMetric + var writeStallCount int + if d.HasArg("write-stall-count") { + d.ScanArgs(t, "write-stall-count", &writeStallCount) + } + var allTokensUsed bool + if d.HasArg("all-tokens-used") { + d.ScanArgs(t, "all-tokens-used", &allTokensUsed) + } + kvGranter.allTokensUsed = allTokensUsed + var provisionedBandwidth, bytesRead, bytesWritten int + if d.HasArg("provisioned-bandwidth") { + d.ScanArgs(t, "provisioned-bandwidth", &provisionedBandwidth) + } + if d.HasArg("bytes-read") { + d.ScanArgs(t, "bytes-read", &bytesRead) + } + if d.HasArg("bytes-written") { + d.ScanArgs(t, "bytes-written", &bytesWritten) + } + if d.HasArg("disk-bw-tokens-used") { + var regularTokensUsed, elasticTokensUsed int + d.ScanArgs(t, "disk-bw-tokens-used", ®ularTokensUsed, &elasticTokensUsed) + kvGranter.diskBandwidthTokensUsed[regularWorkClass] = int64(regularTokensUsed) + kvGranter.diskBandwidthTokensUsed[elasticWorkClass] = int64(elasticTokensUsed) + } else { + kvGranter.diskBandwidthTokensUsed[regularWorkClass] = 0 + kvGranter.diskBandwidthTokensUsed[elasticWorkClass] = 0 + } + var printOnlyFirstTick bool + if d.HasArg("print-only-first-tick") { + d.ScanArgs(t, "print-only-first-tick", &printOnlyFirstTick) + } + ioll.pebbleMetricsTick(ctx, StoreMetrics{ + Metrics: &metrics, + WriteStallCount: int64(writeStallCount), + InternalIntervalMetrics: im, + DiskStats: DiskStats{ + BytesRead: uint64(bytesRead), + BytesWritten: uint64(bytesWritten), + ProvisionedBandwidth: int64(provisionedBandwidth), + }, + }) + var buf strings.Builder + // Do the ticks until just before next adjustment. + res := ioll.adjustTokensResult + fmt.Fprintln(&buf, redact.StringWithoutMarkers(&res)) + res.ioThreshold = nil // avoid nondeterminism + fmt.Fprintf(&buf, "%+v\n", (rawTokenResult)(res)) + if req.buf.Len() > 0 { + fmt.Fprintf(&buf, "%s\n", req.buf.String()) + req.buf.Reset() + } + for i := 0; i < ticksInAdjustmentInterval; i++ { + ioll.allocateTokensTick() + if i == 0 || !printOnlyFirstTick { + fmt.Fprintf(&buf, "tick: %d, %s\n", i, kvGranter.buf.String()) + } + kvGranter.buf.Reset() + } + return buf.String() + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) +} + +func TestIOLoadListenerOverflow(t *testing.T) { + req := &testRequesterForIOLL{} + kvGranter := &testGranterWithIOTokens{} + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + ioll := ioLoadListener{ + settings: st, + kvRequester: req, + } + ioll.mu.Mutex = &syncutil.Mutex{} + ioll.mu.kvGranter = kvGranter + // Bug 1: overflow when totalNumByteTokens is too large. + for i := int64(0); i < adjustmentInterval; i++ { + // Override the totalNumByteTokens manually to trigger the overflow bug. + ioll.totalNumByteTokens = math.MaxInt64 - i + ioll.byteTokensAllocated = 0 + for j := 0; j < ticksInAdjustmentInterval; j++ { + ioll.allocateTokensTick() + } + } + // Bug2: overflow when bytes added delta is 0. + m := pebble.Metrics{} + m.Levels[0] = pebble.LevelMetrics{ + Sublevels: 100, + NumFiles: 10000, + } + ioll.pebbleMetricsTick(ctx, + StoreMetrics{Metrics: &m, InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}}) + ioll.pebbleMetricsTick(ctx, + StoreMetrics{Metrics: &m, InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}}) + ioll.allocateTokensTick() +} + +// TODO(sumeer): we now do more work outside adjustTokensInner, so the parts +// of the adjustTokensResult computed by adjustTokensInner has become a subset +// of what is logged below, and the rest is logged with 0 values. Expand this +// test to call adjustTokens. +func TestAdjustTokensInnerAndLogging(t *testing.T) { + const mb = 12 + 1<<20 + tests := []struct { + name redact.SafeString + prev ioLoadListenerState + l0Metrics pebble.LevelMetrics + }{ + { + name: "zero", + }, + { + name: "real-numbers", + prev: ioLoadListenerState{ + cumL0AddedBytes: 1402 * mb, + curL0Bytes: 400 * mb, + cumWriteStallCount: 10, + smoothedIntL0CompactedBytes: 47 * mb, + smoothedCompactionByteTokens: 201 * mb, + totalNumByteTokens: int64(201 * mb), + }, + l0Metrics: pebble.LevelMetrics{ + Sublevels: 27, + NumFiles: 195, + Size: 900 * mb, + BytesIngested: 1801 * mb, + BytesFlushed: 178 * mb, + }, + }, + } + ctx := context.Background() + var buf redact.StringBuilder + for _, tt := range tests { + buf.Printf("%s:\n", tt.name) + res := (*ioLoadListener)(nil).adjustTokensInner( + ctx, tt.prev, tt.l0Metrics, 12, &pebble.InternalIntervalMetrics{}, + 100, 10, 0.50) + buf.Printf("%s\n", res) + } + echotest.Require(t, string(redact.Sprint(buf)), filepath.Join(testutils.TestDataPath(t, "format_adjust_tokens_stats.txt"))) +} + +// TestBadIOLoadListenerStats tests that bad stats (non-monotonic cumulative +// stats and negative values) don't cause panics or tokens to be negative. +func TestBadIOLoadListenerStats(t *testing.T) { + var m pebble.Metrics + var d DiskStats + req := &testRequesterForIOLL{} + ctx := context.Background() + + randomValues := func() { + // Use uints, and cast so that we get bad negative values. + m.Levels[0].Sublevels = int32(rand.Uint32()) + m.Levels[0].NumFiles = int64(rand.Uint64()) + m.Levels[0].Size = int64(rand.Uint64()) + m.Levels[0].BytesFlushed = rand.Uint64() + for i := range m.Levels { + m.Levels[i].BytesIngested = rand.Uint64() + } + d.BytesRead = rand.Uint64() + d.BytesWritten = rand.Uint64() + d.ProvisionedBandwidth = 1 << 20 + req.stats.admittedCount = rand.Uint64() + req.stats.writeAccountedBytes = rand.Uint64() + req.stats.ingestedAccountedBytes = rand.Uint64() + req.stats.statsToIgnore.Bytes = rand.Uint64() + req.stats.statsToIgnore.ApproxIngestedIntoL0Bytes = rand.Uint64() + } + kvGranter := &testGranterNonNegativeTokens{t: t} + st := cluster.MakeTestingClusterSettings() + ioll := ioLoadListener{ + settings: st, + kvRequester: req, + perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), + diskBandwidthLimiter: makeDiskBandwidthLimiter(), + } + ioll.mu.Mutex = &syncutil.Mutex{} + ioll.mu.kvGranter = kvGranter + for i := 0; i < 100; i++ { + randomValues() + ioll.pebbleMetricsTick(ctx, StoreMetrics{ + Metrics: &m, + InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}, + DiskStats: d, + }) + for j := 0; j < ticksInAdjustmentInterval; j++ { + ioll.allocateTokensTick() + require.LessOrEqual(t, int64(0), ioll.smoothedIntL0CompactedBytes) + require.LessOrEqual(t, float64(0), ioll.smoothedCompactionByteTokens) + require.LessOrEqual(t, float64(0), ioll.smoothedNumFlushTokens) + require.LessOrEqual(t, float64(0), ioll.flushUtilTargetFraction) + require.LessOrEqual(t, int64(0), ioll.totalNumByteTokens) + require.LessOrEqual(t, int64(0), ioll.byteTokensAllocated) + require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokens) + require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokensAllocated) + } + } +} + +type testRequesterForIOLL struct { + stats storeAdmissionStats + buf strings.Builder +} + +var _ storeRequester = &testRequesterForIOLL{} + +func (r *testRequesterForIOLL) close() {} + +func (r *testRequesterForIOLL) getRequesters() [numWorkClasses]requester { + panic("unimplemented") +} + +func (r *testRequesterForIOLL) getStoreAdmissionStats() storeAdmissionStats { + return r.stats +} + +func (r *testRequesterForIOLL) setStoreRequestEstimates(estimates storeRequestEstimates) { + fmt.Fprintf(&r.buf, "store-request-estimates: writeTokens: %d", estimates.writeTokens) +} + +type testGranterWithIOTokens struct { + buf strings.Builder + allTokensUsed bool + diskBandwidthTokensUsed [numWorkClasses]int64 +} + +var _ granterWithIOTokens = &testGranterWithIOTokens{} + +func (g *testGranterWithIOTokens) setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) { + fmt.Fprintf(&g.buf, "setAvailableIOTokens: %s", tokensForTokenTickDurationToString(tokens)) + if g.allTokensUsed { + return tokens * 2 + } + return 0 +} + +func (g *testGranterWithIOTokens) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) { + fmt.Fprintf(&g.buf, " setAvailableElasticDiskTokens: %s", + tokensForTokenTickDurationToString(tokens)) +} + +func (g *testGranterWithIOTokens) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 { + return g.diskBandwidthTokensUsed +} + +func (g *testGranterWithIOTokens) setAdmittedDoneModelsLocked( + l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, +) { + fmt.Fprintf(&g.buf, "setAdmittedDoneModelsLocked: l0-write-lm: ") + printLinearModel(&g.buf, l0WriteLM) + fmt.Fprintf(&g.buf, " l0-ingest-lm: ") + printLinearModel(&g.buf, l0IngestLM) + fmt.Fprintf(&g.buf, " ingest-lm: ") + printLinearModel(&g.buf, ingestLM) + fmt.Fprintf(&g.buf, "\n") +} + +func tokensForTokenTickDurationToString(tokens int64) string { + if tokens >= unlimitedTokens/ticksInAdjustmentInterval { + return "unlimited" + } + return fmt.Sprintf("%d", tokens) +} + +type rawTokenResult adjustTokensResult + +type testGranterNonNegativeTokens struct { + t *testing.T +} + +var _ granterWithIOTokens = &testGranterNonNegativeTokens{} + +func (g *testGranterNonNegativeTokens) setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) { + require.LessOrEqual(g.t, int64(0), tokens) + return 0 +} + +func (g *testGranterNonNegativeTokens) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) { + require.LessOrEqual(g.t, int64(0), tokens) +} + +func (g *testGranterNonNegativeTokens) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 { + return [numWorkClasses]int64{} +} + +func (g *testGranterNonNegativeTokens) setAdmittedDoneModelsLocked( + l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, +) { + require.LessOrEqual(g.t, 0.5, l0WriteLM.multiplier) + require.LessOrEqual(g.t, int64(0), l0WriteLM.constant) + require.Less(g.t, 0.0, l0IngestLM.multiplier) + require.LessOrEqual(g.t, int64(0), l0IngestLM.constant) + require.LessOrEqual(g.t, 0.5, ingestLM.multiplier) + require.LessOrEqual(g.t, int64(0), ingestLM.constant) +} diff --git a/pkg/util/admission/store_token_estimation_test.go b/pkg/util/admission/store_token_estimation_test.go index 958da9b7b992..c5d078e35f76 100644 --- a/pkg/util/admission/store_token_estimation_test.go +++ b/pkg/util/admission/store_token_estimation_test.go @@ -12,7 +12,6 @@ package admission import ( "fmt" - "strconv" "strings" "testing" @@ -24,64 +23,6 @@ import ( "github.com/stretchr/testify/require" ) -func scanFloat(t *testing.T, d *datadriven.TestData, key string) float64 { - var vstring string - d.ScanArgs(t, key, &vstring) - v, err := strconv.ParseFloat(vstring, 64) - require.NoError(t, err) - return v -} - -func printLinearModel(b *strings.Builder, m tokensLinearModel) { - fmt.Fprintf(b, "%.2fx+%d", m.multiplier, m.constant) -} - -func printLinearModelFitter(b *strings.Builder, fitter tokensLinearModelFitter) { - fmt.Fprintf(b, "int: ") - printLinearModel(b, fitter.intLinearModel) - fmt.Fprintf(b, " smoothed: ") - printLinearModel(b, fitter.smoothedLinearModel) - fmt.Fprintf(b, " per-work-accounted: %d\n", fitter.smoothedPerWorkAccountedBytes) -} - -func TestTokensLinearModelFitter(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - var fitter tokensLinearModelFitter - fitterToString := func() string { - var b strings.Builder - printLinearModelFitter(&b, fitter) - return b.String() - } - datadriven.RunTest(t, testutils.TestDataPath(t, "tokens_linear_model_fitter"), - func(t *testing.T, d *datadriven.TestData) string { - switch d.Cmd { - case "init": - multMin := scanFloat(t, d, "mult-min") - multMax := scanFloat(t, d, "mult-max") - updateWithZeroActualNonZeroAccounted := false - if d.HasArg("ingested-model") { - d.ScanArgs(t, "ingested-model", &updateWithZeroActualNonZeroAccounted) - } - fitter = makeTokensLinearModelFitter(multMin, multMax, updateWithZeroActualNonZeroAccounted) - return fitterToString() - - case "update": - var accountedBytes, actualBytes, workCount int - d.ScanArgs(t, "accounted-bytes", &accountedBytes) - d.ScanArgs(t, "actual-bytes", &actualBytes) - d.ScanArgs(t, "work-count", &workCount) - fitter.updateModelUsingIntervalStats( - int64(accountedBytes), int64(actualBytes), int64(workCount)) - return fitterToString() - - default: - return fmt.Sprintf("unknown command: %s", d.Cmd) - } - }) -} - func TestStorePerWorkTokenEstimator(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/util/admission/tokens_linear_model_test.go b/pkg/util/admission/tokens_linear_model_test.go new file mode 100644 index 000000000000..351c86b0f18b --- /dev/null +++ b/pkg/util/admission/tokens_linear_model_test.go @@ -0,0 +1,82 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package admission + +import ( + "fmt" + "strconv" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/datadriven" + "github.com/stretchr/testify/require" +) + +func scanFloat(t *testing.T, d *datadriven.TestData, key string) float64 { + var vstring string + d.ScanArgs(t, key, &vstring) + v, err := strconv.ParseFloat(vstring, 64) + require.NoError(t, err) + return v +} + +func printLinearModel(b *strings.Builder, m tokensLinearModel) { + fmt.Fprintf(b, "%.2fx+%d", m.multiplier, m.constant) +} + +func printLinearModelFitter(b *strings.Builder, fitter tokensLinearModelFitter) { + fmt.Fprintf(b, "int: ") + printLinearModel(b, fitter.intLinearModel) + fmt.Fprintf(b, " smoothed: ") + printLinearModel(b, fitter.smoothedLinearModel) + fmt.Fprintf(b, " per-work-accounted: %d\n", fitter.smoothedPerWorkAccountedBytes) +} + +func TestTokensLinearModelFitter(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var fitter tokensLinearModelFitter + fitterToString := func() string { + var b strings.Builder + printLinearModelFitter(&b, fitter) + return b.String() + } + datadriven.RunTest(t, testutils.TestDataPath(t, "tokens_linear_model_fitter"), + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + multMin := scanFloat(t, d, "mult-min") + multMax := scanFloat(t, d, "mult-max") + updateWithZeroActualNonZeroAccounted := false + if d.HasArg("ingested-model") { + d.ScanArgs(t, "ingested-model", &updateWithZeroActualNonZeroAccounted) + } + fitter = makeTokensLinearModelFitter(multMin, multMax, updateWithZeroActualNonZeroAccounted) + return fitterToString() + + case "update": + var accountedBytes, actualBytes, workCount int + d.ScanArgs(t, "accounted-bytes", &accountedBytes) + d.ScanArgs(t, "actual-bytes", &actualBytes) + d.ScanArgs(t, "work-count", &workCount) + fitter.updateModelUsingIntervalStats( + int64(accountedBytes), int64(actualBytes), int64(workCount)) + return fitterToString() + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) +}