diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index 11e528f3b374..905630f4263a 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -4,10 +4,15 @@ 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", + "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", @@ -35,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/admission.go b/pkg/util/admission/admission.go new file mode 100644 index 000000000000..ba1c78318806 --- /dev/null +++ b/pkg/util/admission/admission.go @@ -0,0 +1,546 @@ +// 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/pebble" + "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")) + } +} + +// 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/doc.go b/pkg/util/admission/doc.go deleted file mode 100644 index 62cdfdd55fdf..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 new file mode 100644 index 000000000000..64b10cbf1c15 --- /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 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 +// 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 89c1369af0b7..dec7b0cc19d2 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -11,53 +11,17 @@ package admission import ( - "context" - "math" "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/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" ) -// 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) - -// 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,348 +30,16 @@ 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 - // 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 @@ -996,1106 +628,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() - }() -} - -// 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, - 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 -// 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) -} - -// 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 @@ -2134,860 +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 -// 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 -} - -// storeRequester is used to abstract *StoreWorkQueue for testing. -type storeRequester interface { - requesterClose - getRequesters() [numWorkClasses]requester - getStoreAdmissionStats() storeAdmissionStats - 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{} - -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", @@ -3031,38 +709,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..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= @@ -207,7 +118,7 @@ func TestGranterBasic(t *testing.T) { case "init-store-grant-coordinator": clearRequesterAndCoord() - metrics := makeGranterMetrics() + metrics := makeGrantCoordinatorMetrics() storeCoordinators := &StoreGrantCoordinators{ settings: settings, makeStoreRequesterFunc: func( @@ -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.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/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/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 +} 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 +} diff --git a/pkg/util/admission/store_token_estimation.go b/pkg/util/admission/store_token_estimation.go index caad156e7f50..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 @@ -399,3 +247,10 @@ func (e *storePerWorkTokenEstimator) getModelsAtAdmittedDone() ( e.atDoneL0IngestTokensLinearModel.smoothedLinearModel, e.atDoneIngestTokensLinearModel.smoothedLinearModel } + +func max(i, j int64) int64 { + if i < j { + return j + } + return i +} 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.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)) +} 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) + } + }) +}