diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index cfc73b42dff8..c5eb2cefe3b6 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -209,6 +209,9 @@ ALL_TESTS = [ "//pkg/kv/kvserver/idalloc:idalloc_test", "//pkg/kv/kvserver/intentresolver:intentresolver_test", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller:kvflowcontroller_test", + "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:kvflowhandle_test", + "//pkg/kv/kvserver/kvflowcontrol/kvflowsimulator:kvflowsimulator_test", + "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:kvflowtokentracker_test", "//pkg/kv/kvserver/kvstorage:kvstorage_test", "//pkg/kv/kvserver/liveness:liveness_test", "//pkg/kv/kvserver/logstore:logstore_test", @@ -1234,6 +1237,11 @@ GO_TARGETS = [ "//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller:kvflowcontroller", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller:kvflowcontroller_test", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb:kvflowcontrolpb", + "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:kvflowhandle", + "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:kvflowhandle_test", + "//pkg/kv/kvserver/kvflowcontrol/kvflowsimulator:kvflowsimulator_test", + "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:kvflowtokentracker", + "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:kvflowtokentracker_test", "//pkg/kv/kvserver/kvflowcontrol:kvflowcontrol", "//pkg/kv/kvserver/kvserverbase:kvserverbase", "//pkg/kv/kvserver/kvserverpb:kvserverpb", @@ -2643,6 +2651,9 @@ GET_X_DATA_TARGETS = [ "//pkg/kv/kvserver/kvflowcontrol:get_x_data", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller:get_x_data", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb:get_x_data", + "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:get_x_data", + "//pkg/kv/kvserver/kvflowcontrol/kvflowsimulator:get_x_data", + "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:get_x_data", "//pkg/kv/kvserver/kvserverbase:get_x_data", "//pkg/kv/kvserver/kvserverpb:get_x_data", "//pkg/kv/kvserver/kvstorage:get_x_data", diff --git a/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel index 4817b09ce5a9..23b2271aa510 100644 --- a/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel @@ -6,10 +6,12 @@ go_library( srcs = [ "doc.go", "kvflowcontrol.go", + "testing_knobs.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/roachpb", "//pkg/util/admission/admissionpb", diff --git a/pkg/kv/kvserver/kvflowcontrol/doc.go b/pkg/kv/kvserver/kvflowcontrol/doc.go index 285cb16cce1e..60abec7421fe 100644 --- a/pkg/kv/kvserver/kvflowcontrol/doc.go +++ b/pkg/kv/kvserver/kvflowcontrol/doc.go @@ -307,6 +307,26 @@ package kvflowcontrol // stream is set to the raft log position of the command removing the replica, // so stale AdmittedRaftLogEntries messages can be discarded. // +// I11. What happens when a node is restarted and is being caught up rapidly +// through raft log appends? We know of cases where the initial log appends +// and subsequent state machine application be large enough to invert the +// LSM[^9]. Imagine large block writes with a uniform key distribution; we +// may persist log entries rapidly across many replicas (without inverting +// the LSM, so follower pausing is also of no help) and during state +// machine application, create lots of overlapping files/sublevels in L0. +// - We want to pace the initial rate of log appends while factoring in the +// effect of the subsequent state machine application on L0 (modulo [^9]). We +// can use flow tokens for this too. In I3a we outlined how for quorum writes +// that includes a replica on some recently re-started node, we need to wait +// for it to be sufficiently caught before deducting/blocking for flow tokens. +// Until that point we can use flow tokens on sender nodes that wish to send +// catchup MsgApps to the newly-restarted node. Similar to the steady state, +// flow tokens are only be returned once log entries are logically admitted +// (which takes into account any apply-time write amplification, modulo [^9]). +// Once the node is sufficiently caught up with respect to all its raft logs, +// it can transition into the mode described in I3a where we deduct/block for +// flow tokens for subsequent quorum writes. +// // --- // // [^1]: kvserverpb.RaftMessageRequest is the unit of what's sent @@ -315,9 +335,8 @@ package kvflowcontrol // [^2]: Over which we're dispatching kvflowcontrolpb.AdmittedRaftLogEntries. // [^3]: kvflowcontrol.DispatchReader implementations do this as part of // PendingDispatchFor. -// [^4]: Using DeductedTokensUpto + ReturnAllTokensUpto on -// kvflowcontrol.Handler. -// [^5]: Using ReturnAllTokensUpto on kvflowcontrol.Handler. +// [^4]: Using DisconnectStream on kvflowcontrol.Handler. +// [^5]: Using ConnectStream on kvflowcontrol.Handler. // [^6]: DeductTokens on kvflowcontrol.Controller returns whether the deduction // was done. // [^7]: When a node is crashed, instead of ignoring the underlying flow token @@ -331,6 +350,29 @@ package kvflowcontrol // Admit(), or (ii) don't DeductTokens (Admit() is rendered a no-op), // we're being somewhat optimistic, which is fine. // [^8]: Using ReturnTokensUpto on kvflowcontrol.Handle. +// [^9]: With async raft storage writes (#17500, etcd-io/raft#8), we can +// decouple raft log appends and state machine application (see #94854 and +// #94853). So we could append at a higher rate than applying. Since +// application can be arbitrarily deferred, we cause severe LSM +// inversions. Do we want some form of pacing of log appends then, +// relative to observed state machine application? Perhaps specifically in +// cases where we're more likely to append faster than apply, like node +// restarts. We're likely to defeat AC's IO control otherwise. +// - For what it's worth, this "deferred application with high read-amp" +// was also a problem before async raft storage writes. Consider many +// replicas on an LSM, all of which appended a few raft log entries +// without applying, and at apply time across all those replicas, we end +// up inverting the LSM. +// - Since we don't want to wait below raft, one way bound the lag between +// appended entries and applied ones is to only release flow tokens for +// an entry at position P once the applied state position >= P - delta. +// We'd have to be careful, if we're not applying due to quorum loss +// (as a result of remote node failure(s)), we don't want to deplete +// flow tokens and cause interference on other ranges. +// - If this all proves too complicated, we could just not let state +// machine application get significantly behind due to local scheduling +// reasons by using the same goroutine to do both async raft log writes +// and state machine application. // // TODO(irfansharif): These descriptions are too high-level, imprecise and // possibly wrong. Fix that. After implementing these interfaces and integrating diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go index 635b946f6b14..3f0a51ef3831 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go @@ -33,6 +33,17 @@ type Stream struct { StoreID roachpb.StoreID } +// ConnectedStream models a stream over which we're actively replicating data +// traffic. The embedded channel is signaled when the stream is disconnected, +// for example when (i) the remote node has crashed, (ii) bidirectional gRPC +// streams break, (iii) we've paused replication traffic to it, (iv) truncated +// our raft log ahead it, and more. Whenever that happens, we unblock inflight +// requests waiting for flow tokens. +type ConnectedStream interface { + Stream() Stream + Disconnected() <-chan struct{} +} + // Tokens represent the finite capacity of a given stream, expressed in bytes // for data we're looking to replicate. Use of replication streams are // predicated on tokens being available. @@ -46,12 +57,13 @@ type Tokens int64 type Controller interface { // Admit seeks admission to replicate data, regardless of size, for work // with the given priority, create-time, and over the given stream. This - // blocks until there are flow tokens available. - Admit(context.Context, admissionpb.WorkPriority, time.Time, Stream) error + // blocks until there are flow tokens available or the stream disconnects, + // subject to context cancellation. + Admit(context.Context, admissionpb.WorkPriority, time.Time, ConnectedStream) error // DeductTokens deducts (without blocking) flow tokens for replicating work // with given priority over the given stream. Requests are expected to // have been Admit()-ed first. - DeductTokens(context.Context, admissionpb.WorkPriority, Tokens, Stream) (deducted bool) + DeductTokens(context.Context, admissionpb.WorkPriority, Tokens, Stream) // ReturnTokens returns flow tokens for the given stream. These tokens are // expected to have been deducted earlier with the same priority provided // here. @@ -67,7 +79,8 @@ type Controller interface { // Handle is used to interface with replication flow control; it's typically // backed by a node-level kvflowcontrol.Controller. Handles are held on replicas // initiating replication traffic, i.e. are both the leaseholder and raft -// leader, and manage multiple Streams (one per active replica) underneath. +// leader, and manage multiple streams underneath (typically one per active +// member of the raft group). // // When replicating log entries, these replicas choose the log position // (term+index) the data is to end up at, and use this handle to track the token @@ -79,34 +92,45 @@ type Controller interface { type Handle interface { // Admit seeks admission to replicate data, regardless of size, for work // with the given priority and create-time. This blocks until there are - // flow tokens available. - Admit(context.Context, admissionpb.WorkPriority, time.Time) + // flow tokens available for all connected streams. + Admit(context.Context, admissionpb.WorkPriority, time.Time) error // DeductTokensFor deducts (without blocking) flow tokens for replicating - // work with given priority to members of the raft group. The deduction, - // if successful, is tracked with respect to the specific raft log position - // it's expecting it to end up in. Requests are assumed to have been - // Admit()-ed first. + // work with given priority along connected streams. The deduction is + // tracked with respect to the specific raft log position it's expecting it + // to end up in, log positions that monotonically increase. Requests are + // assumed to have been Admit()-ed first. DeductTokensFor(context.Context, admissionpb.WorkPriority, kvflowcontrolpb.RaftLogPosition, Tokens) - // DeductedTokensUpto returns the highest log position for which we've - // deducted flow tokens for, over the given stream. - DeductedTokensUpto(context.Context, Stream) kvflowcontrolpb.RaftLogPosition // ReturnTokensUpto returns all previously deducted tokens of a given // priority for all log positions less than or equal to the one specified. // It does for the specific stream. Once returned, subsequent attempts to - // return tokens upto the same position or lower are no-ops. + // return tokens upto the same position or lower are no-ops. It's used when + // entries at specific log positions have been admitted below-raft. + // + // NB: Another use is during successive lease changes (out and back) within + // the same raft term -- we want to both free up tokens from when we lost + // the lease, and also ensure we discard attempts to return them (on hearing + // about AdmittedRaftLogEntries replicated under the earlier lease). ReturnTokensUpto(context.Context, admissionpb.WorkPriority, kvflowcontrolpb.RaftLogPosition, Stream) - // ReturnAllTokensUpto is like ReturnTokensUpto but does so across all - // priorities. + // ConnectStream connects a stream (typically pointing to an active member + // of the raft group) to the handle. Subsequent calls to Admit() will block + // until flow tokens are available for the stream, or for it to be + // disconnected via DisconnectStream. DeductTokensFor will also deduct + // tokens for all connected streams. The log position is used as a lower + // bound, beneath which all token deductions/returns are rendered no-ops. + ConnectStream(context.Context, kvflowcontrolpb.RaftLogPosition, Stream) + // DisconnectStream disconnects a stream from the handle. When disconnecting + // a stream, (a) all previously held flow tokens are released and (b) we + // unblock all requests waiting in Admit() for this stream's flow tokens in + // particular. // - // NB: This is used when a replica on the other end of a stream gets caught - // up via snapshot (say, after a log truncation), where we then don't expect - // dispatches for the individual AdmittedRaftLogEntries between what it - // admitted last and its latest RaftLogPosition. Another use is during - // successive lease changes (out and back) within the same raft term -- we - // want to both free up tokens from when we lost the lease, and also ensure - // that attempts to return them (on hearing about AdmittedRaftLogEntries - // replicated under the earlier lease), we discard the attempts. - ReturnAllTokensUpto(context.Context, kvflowcontrolpb.RaftLogPosition, Stream) + // This is typically used when we're no longer replicating data to a member + // of the raft group, because (a) it crashed, (b) it's no longer part of the + // raft group, (c) we've decided to pause it, (d) we've truncated the raft + // log ahead of it and expect it to be caught up via snapshot, and more. In + // all these cases we don't expect dispatches for individual + // AdmittedRaftLogEntries between what it admitted last and its latest + // RaftLogPosition. + DisconnectStream(context.Context, Stream) // Close closes the handle and returns all held tokens back to the // underlying controller. Typically used when the replica loses its lease // and/or raft leadership, or ends up getting GC-ed (if it's being @@ -134,7 +158,7 @@ type DispatchWriter interface { // piggybacking) has not taken place. // // NB: PendingDispatchFor is expected to remove dispatches from the pending -// list. If the GRPC stream we're sending it over happens to break, we drop +// list. If the gRPC stream we're sending it over happens to break, we drop // these dispatches. The node waiting these dispatches is expected to react to // the stream breaking by freeing up all held tokens. type DispatchReader interface { diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel index eaf9f5c6311b..1c2ac6ecb8d0 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel @@ -25,10 +25,7 @@ go_library( go_test( name = "kvflowcontroller_test", - srcs = [ - "kvflowcontrol_token_adjustment_test.go", - "kvflowcontroller_simulation_test.go", - ], + srcs = ["kvflowcontrol_token_adjustment_test.go"], args = ["-test.timeout=295s"], data = glob(["testdata/**"]), embed = [":kvflowcontroller"], @@ -36,18 +33,13 @@ go_test( "//pkg/kv/kvserver/kvflowcontrol", "//pkg/roachpb", "//pkg/settings/cluster", - "//pkg/testutils/datapathutils", "//pkg/util/admission/admissionpb", - "//pkg/util/asciitsdb", "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/metric", - "//pkg/util/timeutil", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_dustin_go_humanize//:go-humanize", - "@com_github_guptarohit_asciigraph//:asciigraph", - "@com_github_mkungla_bexp_v3//:bexp", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go index 69a760d47c59..f9ab872e29df 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go @@ -119,7 +119,10 @@ func New(registry *metric.Registry, settings *cluster.Settings, clock *hlc.Clock // there are flow tokens available for replication over the given stream for // work of the given priority. func (c *Controller) Admit( - ctx context.Context, pri admissionpb.WorkPriority, _ time.Time, stream kvflowcontrol.Stream, + ctx context.Context, + pri admissionpb.WorkPriority, + _ time.Time, + connection kvflowcontrol.ConnectedStream, ) error { class := admissionpb.WorkClassFromPri(pri) c.metrics.onWaiting(class) @@ -128,14 +131,14 @@ func (c *Controller) Admit( tstart := c.clock.PhysicalTime() for { c.mu.Lock() - b := c.getBucketLocked(stream) + b := c.getBucketLocked(connection.Stream()) tokens := b.tokens[class] c.mu.Unlock() if tokens > 0 { if log.ExpensiveLogEnabled(ctx, 2) { log.Infof(ctx, "flow tokens available (pri=%s stream=%s tokens=%s wait-duration=%s)", - pri, stream, tokens, c.clock.PhysicalTime().Sub(tstart)) + pri, connection.Stream(), tokens, c.clock.PhysicalTime().Sub(tstart)) } // TODO(irfansharif): Right now we continue forwarding admission @@ -163,12 +166,15 @@ func (c *Controller) Admit( if !logged && log.ExpensiveLogEnabled(ctx, 2) { log.Infof(ctx, "waiting for flow tokens (pri=%s stream=%s tokens=%s)", - pri, stream, tokens) + pri, connection.Stream(), tokens) logged = true } select { case <-b.wait(): // wait for a signal + case <-connection.Disconnected(): + c.metrics.onBypassed(class) + return nil case <-ctx.Done(): if ctx.Err() != nil { c.metrics.onErrored(class) @@ -187,13 +193,12 @@ func (c *Controller) DeductTokens( pri admissionpb.WorkPriority, tokens kvflowcontrol.Tokens, stream kvflowcontrol.Stream, -) bool { +) { if tokens < 0 { log.Fatalf(ctx, "malformed argument: -ve tokens deducted (pri=%s tokens=%s stream=%s)", pri, tokens, stream) } c.adjustTokens(ctx, pri, -tokens, stream) - return true } // ReturnTokens is part of the kvflowcontrol.Controller interface. @@ -204,10 +209,13 @@ func (c *Controller) ReturnTokens( stream kvflowcontrol.Stream, ) { if tokens < 0 { - log.Fatalf(ctx, "malformed argument: -ve tokens deducted (pri=%s tokens=%s stream=%s)", + log.Fatalf(ctx, "malformed argument: -ve tokens returned (pri=%s tokens=%s stream=%s)", pri, tokens, stream) } - c.adjustTokens(ctx, pri, tokens, stream) + if tokens == 0 { + return // nothing to do + } + c.adjustTokens(ctx, pri, +tokens, stream) } func (c *Controller) adjustTokens( @@ -369,7 +377,7 @@ func (c *Controller) testingGetLimit() tokensPerWorkClass { return c.mu.limit } -// testingNonBlockingAdmit is a non-blocking alternative to Admit() for use in +// TestingNonBlockingAdmit is a non-blocking alternative to Admit() for use in // tests. // - it checks if we have a non-zero number of flow tokens // - if we do, we return immediately with admitted=true @@ -378,16 +386,28 @@ func (c *Controller) testingGetLimit() tokensPerWorkClass { // admitting again; // - admit, which can be used to try and admit again. If still not admitted, // callers are to wait until they're signaled again. -func (c *Controller) testingNonBlockingAdmit( - pri admissionpb.WorkPriority, stream kvflowcontrol.Stream, +// +// TODO(irfansharif): Fold in ctx cancelation into this non-blocking interface +// (signaled return true if ctx is canceled), and admit can increment the right +// errored metric underneath. We'll have to plumb this to the (test) caller too +// to prevent it from deducting tokens for canceled requests. +func (c *Controller) TestingNonBlockingAdmit( + pri admissionpb.WorkPriority, connection kvflowcontrol.ConnectedStream, ) (admitted bool, signaled func() bool, admit func() bool) { class := admissionpb.WorkClassFromPri(pri) c.metrics.onWaiting(class) tstart := c.clock.PhysicalTime() admit = func() bool { + select { + case <-connection.Disconnected(): + c.metrics.onBypassed(class) + return true + default: + } + c.mu.Lock() - b := c.getBucketLocked(stream) + b := c.getBucketLocked(connection.Stream()) tokens := b.tokens[class] c.mu.Unlock() @@ -404,8 +424,23 @@ func (c *Controller) testingNonBlockingAdmit( return true, nil, nil } - b := c.testingGetBucket(stream) - return false, b.testingSignaled, admit + b := c.testingGetBucket(connection.Stream()) + return false, b.testingSignaled(connection), admit +} + +// TestingAdjustTokens exports adjustTokens for testing purposes. +func (c *Controller) TestingAdjustTokens( + ctx context.Context, + pri admissionpb.WorkPriority, + delta kvflowcontrol.Tokens, + stream kvflowcontrol.Stream, +) { + c.adjustTokens(ctx, pri, delta, stream) +} + +// TestingMetrics returns the underlying metrics struct for testing purposes. +func (c *Controller) TestingMetrics() interface{} { + return c.metrics } func (c *Controller) testingGetBucket(stream kvflowcontrol.Stream) bucket { @@ -414,11 +449,15 @@ func (c *Controller) testingGetBucket(stream kvflowcontrol.Stream) bucket { return c.getBucketLocked(stream) } -func (b *bucket) testingSignaled() bool { - select { - case <-b.wait(): // check if signaled - return true - default: - return false +func (b *bucket) testingSignaled(connection kvflowcontrol.ConnectedStream) func() bool { + return func() bool { + select { + case <-connection.Disconnected(): + return true + case <-b.wait(): // check if signaled + return true + default: + return false + } } } diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_metrics.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_metrics.go index 9a1eb4f1c40f..f6fbeb1f2ba0 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_metrics.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_metrics.go @@ -85,6 +85,13 @@ var ( Unit: metric.Unit_COUNT, } + requestsBypassed = metric.Metadata{ + Name: "kvadmission.flow_controller.%s_requests_bypassed", + Help: "Number of %s waiting requests that bypassed the flow controller due to disconnecting streams", + Measurement: "Requests", + Unit: metric.Unit_COUNT, + } + waitDuration = metric.Metadata{ Name: "kvadmission.flow_controller.%s_wait_duration", Help: "Latency histogram for time %s requests spent waiting for flow tokens", @@ -126,6 +133,7 @@ type metrics struct { RequestsWaiting [admissionpb.NumWorkClasses]*metric.Gauge RequestsAdmitted [admissionpb.NumWorkClasses]*metric.Counter RequestsErrored [admissionpb.NumWorkClasses]*metric.Counter + RequestsBypassed [admissionpb.NumWorkClasses]*metric.Counter WaitDuration [admissionpb.NumWorkClasses]metric.IHistogram TotalStreamCount [admissionpb.NumWorkClasses]*metric.Gauge BlockedStreamCount [admissionpb.NumWorkClasses]*metric.Gauge @@ -167,6 +175,9 @@ func newMetrics(c *Controller) *metrics { m.RequestsAdmitted[wc] = metric.NewCounter( annotateMetricTemplateWithWorkClass(wc, requestsAdmitted), ) + m.RequestsBypassed[wc] = metric.NewCounter( + annotateMetricTemplateWithWorkClass(wc, requestsBypassed), + ) m.RequestsErrored[wc] = metric.NewCounter( annotateMetricTemplateWithWorkClass(wc, requestsErrored), ) @@ -214,6 +225,10 @@ func (m *metrics) onAdmitted(class admissionpb.WorkClass, dur time.Duration) { m.WaitDuration[class].RecordValue(dur.Nanoseconds()) } +func (m *metrics) onBypassed(class admissionpb.WorkClass) { + m.RequestsBypassed[class].Inc(1) +} + func (m *metrics) onErrored(class admissionpb.WorkClass) { m.RequestsErrored[class].Inc(1) } diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_simulation_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_simulation_test.go deleted file mode 100644 index 103460d943d8..000000000000 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_simulation_test.go +++ /dev/null @@ -1,522 +0,0 @@ -// Copyright 2023 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 kvflowcontroller - -import ( - "context" - "fmt" - "strconv" - "strings" - "testing" - "time" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" - "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" - "github.com/cockroachdb/cockroach/pkg/util/asciitsdb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/metric" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/datadriven" - "github.com/dustin/go-humanize" - "github.com/guptarohit/asciigraph" - "github.com/mkungla/bexp/v3" - "github.com/stretchr/testify/require" -) - -// TestUsingSimulation is a data-driven test for kvflowcontrol.Controller. -// It gives package authors a way to understand how flow tokens are maintained -// for individual replication streams, how write bandwidth is shaped by these -// tokens, and how requests queue/dequeue internally. We provide the following -// syntax: -// -// - "init" -// Initialize the flow controller and test simulator. -// -// - "timeline" -// start= end= class={regular,elastic} \ -// stream=t/s adjust={+,-}/s rate=/s \ -// [deduction-delay=] -// .... -// Creates a "thread" that operates between t='start' to (non-inclusive) -// t='end', issuing the specified 'rate' of requests of the given work -// 'class', over the given 'stream', where the flow tokens are -// {deducted,returned} with the given bandwidth. The 'rate' controls the -// granularity of token adjustment, i.e. if adjust=+100bytes/s and -// rate=5/s, then each return adjusts by +100/5 = +20bytes. If flow tokens -// are being deducted (-ve 'adjust'), they go through Admit() followed by -// DeductTokens(). If they're being returned (+ve 'adjust'), they simply go -// through ReturnTokens(). The optional 'deduction-delay' parameter controls -// the number of ticks between each request being granted admission and it -// deducting the corresponding flow tokens. -// -// - "simulate" [end=] -// Simulate timelines until the optionally specified timestamp. If no -// timestamp is specified, the largest end time of all registered timelines -// is used instead. -// -// - "plot" [height=] [width=] [precision=] \ -// [start=] [end=] -// unit= [rate=true] -// .... -// Plot the flow controller specified metrics (and optionally its rate of -// change) with the specified units. The following metrics are supported: -// a. kvadmission.flow_controller.{regular,elastic}_tokens_{available,deducted,returned} -// b. kvadmission.flow_controller.{regular,elastic}_requests_{waiting,admitted,errored} -// c. kvadmission.flow_controller.{regular,elastic}{,_blocked}_stream_count -// d. kvadmission.flow_controller.{regular,elastic}_wait_duration -// To overlay metrics onto the same plot, the selector supports curly brace -// expansion. If the unit is one of {MiB,MB,KB,KiB,s,ms,us,μs}, or the -// bandwidth equivalents (/s), the y-axis is automatically -// converted. -// -// Internally we make use of a test-only non-blocking interface for the flow -// controller in order to enable deterministic simulation. -func TestUsingSimulation(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - datadriven.Walk(t, datapathutils.TestDataPath(t, "simulation"), func(t *testing.T, path string) { - var ( - controller *Controller - simulator *simulator - tsdb *asciitsdb.TSDB - mtime *timeutil.ManualTime - ) - - ctx := context.Background() - datadriven.RunTest(t, path, - func(t *testing.T, d *datadriven.TestData) string { - switch d.Cmd { - case "init": - registry := metric.NewRegistry() - tsdb = asciitsdb.New(t, registry) - mtime = timeutil.NewManualTime(tstart) - controller = New( - registry, - cluster.MakeTestingClusterSettings(), - hlc.NewClockForTesting(mtime), - ) - tsdb.Register(controller.metrics) - simulator = newSimulator(t, controller, tsdb, mtime) - return "" - - case "timeline": - require.NotNilf(t, controller, "uninitialized flow controller (did you use 'init'?)") - require.NotNilf(t, simulator, "uninitialized simulator (did you use 'init'?)") - - for _, line := range strings.Split(d.Input, "\n") { - parts := strings.Fields(line) - require.True(t, len(parts) >= 6, `expected form: - start= end= class={regular,elastic} \ - stream=t/s adjust={+,-}/s rate=/s \ - [deduction-delay=] -`) - - var ( - start, end time.Time - pri admissionpb.WorkPriority - stream kvflowcontrol.Stream - delta kvflowcontrol.Tokens - rate int - ) - - for i := range parts { - parts[i] = strings.TrimSpace(parts[i]) - } - - require.True(t, strings.HasPrefix(parts[0], "start=")) - require.True(t, strings.HasPrefix(parts[1], "end=")) - require.True(t, strings.HasPrefix(parts[2], "class=")) - require.True(t, strings.HasPrefix(parts[3], "stream=")) - require.True(t, strings.HasPrefix(parts[4], "adjust=")) - require.True(t, strings.HasPrefix(parts[5], "rate=")) - - for i := range parts[:6] { - inner := strings.Split(parts[i], "=") - require.Len(t, inner, 2) - parts[i] = strings.TrimSpace(inner[1]) - } - - // Parse start=. - dur, err := time.ParseDuration(parts[0]) - require.NoError(t, err) - start = mtime.Now().Add(dur) - - // Parse end=. - dur, err = time.ParseDuration(parts[1]) - require.NoError(t, err) - end = mtime.Now().Add(dur) - - // Parse class={regular,elastic}. - switch parts[2] { - case "regular": - pri = admissionpb.NormalPri - case "elastic": - pri = admissionpb.BulkNormalPri - default: - t.Fatalf("unexpected class: %s", parts[1]) - } - - { // Parse stream=t/s. - inner := strings.Split(parts[3], "/") - require.Len(t, inner, 2) - - ti, err := strconv.Atoi(strings.TrimPrefix(inner[0], "t")) - require.NoError(t, err) - - si, err := strconv.Atoi(strings.TrimPrefix(inner[1], "s")) - require.NoError(t, err) - - stream = kvflowcontrol.Stream{ - TenantID: roachpb.MustMakeTenantID(uint64(ti)), - StoreID: roachpb.StoreID(si), - } - } - - // Parse adjust={+,-}/s. - isPositive := strings.Contains(parts[4], "+") - parts[4] = strings.TrimPrefix(parts[4], "+") - parts[4] = strings.TrimPrefix(parts[4], "-") - bytes, err := humanize.ParseBytes(strings.TrimSuffix(parts[4], "/s")) - require.NoError(t, err) - delta = kvflowcontrol.Tokens(int64(bytes)) - if !isPositive { - delta = -delta - } - - // Parse rate=/s. - rate, err = strconv.Atoi(strings.TrimSuffix(parts[5], "/s")) - require.NoError(t, err) - - deductionDelay := 0 - if len(parts) > 6 { - for _, part := range parts[6:] { - part = strings.TrimSpace(part) - if strings.HasPrefix(part, "deduction-delay=") { - part = strings.TrimPrefix(part, "deduction-delay=") - dur, err := time.ParseDuration(part) - require.NoError(t, err) - deductionDelay = int(dur.Nanoseconds() / tick.Nanoseconds()) - } - } - } - simulator.timeline(start, end, pri, stream, delta, rate, deductionDelay) - } - return "" - - case "simulate": - require.NotNilf(t, simulator, "uninitialized simulator (did you use 'init'?)") - var end time.Time - if d.HasArg("end") { - // Parse end=. - var endStr string - d.ScanArgs(t, "end", &endStr) - dur, err := time.ParseDuration(endStr) - require.NoError(t, err) - end = mtime.Now().Add(dur) - } - simulator.simulate(ctx, end) - return "" - - case "plot": - var h, w, p = 15, 40, 1 - if d.HasArg("height") { - d.ScanArgs(t, "height", &h) - } - if d.HasArg("width") { - d.ScanArgs(t, "width", &w) - } - if d.HasArg("precision") { - d.ScanArgs(t, "precision", &p) - } - - var buf strings.Builder - for i, line := range strings.Split(d.Input, "\n") { - var ( - selector, unit string - rated bool - ) - parts := strings.Fields(line) - for i, part := range parts { - part = strings.TrimSpace(part) - if i == 0 { - selector = part - continue - } - - if strings.HasPrefix(part, "rate=") { - var err error - rated, err = strconv.ParseBool(strings.TrimPrefix(part, "rate=")) - require.NoError(t, err) - } - - if strings.HasPrefix(part, "unit=") { - unit = strings.TrimPrefix(part, "unit=") - } - } - - caption := strings.TrimPrefix(selector, "kvadmission.flow_controller.") - if rated { - caption = fmt.Sprintf("rate(%s)", caption) - } - caption = fmt.Sprintf("%s (%s)", caption, unit) - - options := []asciitsdb.Option{ - asciitsdb.WithGraphOptions( - asciigraph.Height(h), - asciigraph.Width(w), - asciigraph.Precision(uint(p)), - asciigraph.Caption(caption), - ), - } - if rated { - options = append(options, asciitsdb.WithRate(int(time.Second/metricTick))) - } - switch unit { - case "μs", "us", "microseconds": - options = append(options, asciitsdb.WithDivisor(float64(time.Microsecond.Nanoseconds())) /* ns => μs conversion */) - case "ms", "milliseconds": - options = append(options, asciitsdb.WithDivisor(float64(time.Millisecond.Nanoseconds())) /* ns => μs conversion */) - case "s", "seconds": - options = append(options, asciitsdb.WithDivisor(float64(time.Second.Nanoseconds())) /* ns => μs conversion */) - case "MiB", "MiB/s": - options = append(options, asciitsdb.WithDivisor(humanize.MiByte) /* 1 MiB */) - case "MB", "MB/s": - options = append(options, asciitsdb.WithDivisor(humanize.MByte) /* 1 MB */) - case "KiB", "KiB/s": - options = append(options, asciitsdb.WithDivisor(humanize.KiByte) /* 1 KiB */) - case "KB", "KB/s": - options = append(options, asciitsdb.WithDivisor(humanize.KByte) /* 1 KB */) - default: - } - - start := tstart - if d.HasArg("start") { - // Parse start=. - var startStr string - d.ScanArgs(t, "start", &startStr) - dur, err := time.ParseDuration(startStr) - require.NoError(t, err) - start = tstart.Add(dur) - options = append(options, asciitsdb.WithOffset(start.Sub(tstart).Nanoseconds()/metricTick.Nanoseconds())) - } - - if d.HasArg("end") { - // Parse end=. - var endStr string - d.ScanArgs(t, "end", &endStr) - dur, err := time.ParseDuration(endStr) - require.NoError(t, err) - end := tstart.Add(dur) - options = append(options, asciitsdb.WithLimit(end.Sub(start).Nanoseconds()/metricTick.Nanoseconds())) - } - - if i > 0 { - buf.WriteString("\n\n\n") - } - metrics := bexp.Parse(strings.TrimSpace(selector)) - buf.WriteString(tsdb.Plot(metrics, options...)) - } - return buf.String() - - default: - return fmt.Sprintf("unknown command: %s", d.Cmd) - } - }, - ) - }) -} - -var tstart = timeutil.Unix(0, 0) - -const tick = time.Millisecond -const metricTick = 100 * tick - -type simulator struct { - t *testing.T - - controller *Controller - ticker []ticker - tsdb *asciitsdb.TSDB - mtime *timeutil.ManualTime -} - -func newSimulator( - t *testing.T, controller *Controller, tsdb *asciitsdb.TSDB, mtime *timeutil.ManualTime, -) *simulator { - return &simulator{ - t: t, - tsdb: tsdb, - controller: controller, - mtime: mtime, - } -} - -func (s *simulator) timeline( - start, end time.Time, - pri admissionpb.WorkPriority, - stream kvflowcontrol.Stream, - delta kvflowcontrol.Tokens, - rate, deductionDelay int, -) { - if rate == 0 { - return // nothing to do - } - s.ticker = append(s.ticker, ticker{ - t: s.t, - controller: s.controller, - - start: start, - end: end, - pri: pri, - stream: stream, - - deductionDelay: deductionDelay, - deduct: make(map[time.Time][]func()), - waitHandles: make(map[time.Time]waitHandle), - - // Using the parameters above, we figure out two things: - // - On which ticks do we adjust flow tokens? - // - How much by, each time? - // - // If the request rate we're simulating is: - // - 1000/sec, we adjust flow tokens every tick(=1ms). - // - 500/sec, we adjust flow tokens every 2 ticks (=2ms). - // - .... - // - // How much do we adjust by each time? Given we're making 'rate' requests - // per second, and have to deduct 'delta' tokens per second, each request - // just deducts delta/rate. - mod: int(time.Second/tick) / rate, - delta: kvflowcontrol.Tokens(int(delta) / rate), - }) -} - -func (s *simulator) simulate(ctx context.Context, end time.Time) { - s.mtime.Backwards(s.mtime.Since(tstart)) // reset to tstart - s.tsdb.Clear() - for i := range s.ticker { - s.ticker[i].reset() - if s.ticker[i].end.After(end) { - end = s.ticker[i].end - } - } - - for { - t := s.mtime.Now() - if t.After(end) || t.Equal(end) { - break - } - for i := range s.ticker { - s.ticker[i].tick(ctx, t) - } - if t.UnixNano()%metricTick.Nanoseconds() == 0 { - s.tsdb.Scrape(ctx) - } - s.mtime.Advance(tick) - } -} - -type waitHandle struct { - ctx context.Context - signaled func() bool - admit func() bool -} - -type ticker struct { - t *testing.T - start, end time.Time - pri admissionpb.WorkPriority - stream kvflowcontrol.Stream - delta kvflowcontrol.Tokens - controller *Controller - mod, ticks int // used to control the ticks at which we interact with the controller - - deduct map[time.Time][]func() - waitHandles map[time.Time]waitHandle - - deductionDelay int -} - -func (ti *ticker) tick(ctx context.Context, t time.Time) { - if ds, ok := ti.deduct[t]; ok { - for _, deduct := range ds { - deduct() - } - delete(ti.deduct, t) - } - for key, handle := range ti.waitHandles { - // Process all waiting requests from earlier. Do this even if t > - // ti.end since these requests could've been generated earlier. - if handle.ctx.Err() != nil { - delete(ti.waitHandles, key) - continue - } - if !handle.signaled() { - continue - } - if handle.admit() { - if ti.deductionDelay == 0 { - ti.controller.adjustTokens(ctx, ti.pri, ti.delta, ti.stream) - } else { - future := t.Add(tick * time.Duration(ti.deductionDelay)) - ti.deduct[future] = append(ti.deduct[future], func() { - ti.controller.adjustTokens(ctx, ti.pri, ti.delta, ti.stream) - }) - } - delete(ti.waitHandles, key) - return - } - } - - if t.Before(ti.start) || (t.After(ti.end) || t.Equal(ti.end)) { - return // we're outside our [ti.start, ti.end), there's nothing left to do - } - - defer func() { ti.ticks += 1 }() - if ti.ticks%ti.mod != 0 { - return // nothing to do in this tick - } - - if ti.delta >= 0 { // return tokens - ti.controller.adjustTokens(ctx, ti.pri, ti.delta, ti.stream) - return - } - - admitted, signaled, admit := ti.controller.testingNonBlockingAdmit(ti.pri, ti.stream) - if admitted { - if ti.deductionDelay == 0 { - ti.controller.adjustTokens(ctx, ti.pri, ti.delta, ti.stream) - } else { - future := t.Add(tick * time.Duration(ti.deductionDelay)) - ti.deduct[future] = append(ti.deduct[future], func() { - ti.controller.adjustTokens(ctx, ti.pri, ti.delta, ti.stream) - }) - } - return - } - ti.waitHandles[t] = waitHandle{ - ctx: ctx, - signaled: signaled, - admit: admit, - } -} - -func (ti *ticker) reset() { - ti.ticks = 0 - ti.deduct = make(map[time.Time][]func()) - ti.waitHandles = make(map[time.Time]waitHandle) -} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/raft_log_position.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/raft_log_position.go index c147618928fb..c363012cc24d 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/raft_log_position.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/raft_log_position.go @@ -12,13 +12,13 @@ package kvflowcontrolpb import "github.com/cockroachdb/redact" -func (p *RaftLogPosition) String() string { +func (p RaftLogPosition) String() string { return redact.StringWithoutMarkers(p) } // SafeFormat implements the redact.SafeFormatter interface. -func (p *RaftLogPosition) SafeFormat(w redact.SafePrinter, _ rune) { - w.Printf("position=%d/%d", p.Term, p.Index) +func (p RaftLogPosition) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("log-position=%d/%d", p.Term, p.Index) } // Equal returns whether the two raft log positions are identical. diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel new file mode 100644 index 000000000000..1a4fc2aa86fe --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel @@ -0,0 +1,42 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "kvflowhandle", + srcs = [ + "connected_stream.go", + "kvflowhandle.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowhandle", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker", + "//pkg/util/admission/admissionpb", + "//pkg/util/log", + "//pkg/util/syncutil", + ], +) + +go_test( + name = "kvflowhandle_test", + srcs = ["kvflowhandle_test.go"], + args = ["-test.timeout=295s"], + deps = [ + ":kvflowhandle", + "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/roachpb", + "//pkg/settings/cluster", + "//pkg/util/admission/admissionpb", + "//pkg/util/hlc", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/metric", + "@com_github_stretchr_testify//require", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/connected_stream.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/connected_stream.go new file mode 100644 index 000000000000..02380bd658a8 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/connected_stream.go @@ -0,0 +1,53 @@ +// Copyright 2023 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 kvflowhandle + +import ( + "sync/atomic" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" +) + +// connectedStream is a concrete implementation of the +// kvflowcontrol.ConnectedStream interface. It's used to unblock requests +// waiting for flow tokens over a stream that has since disconnected. +type connectedStream struct { + stream kvflowcontrol.Stream + ch chan struct{} + disconnected int32 +} + +var _ kvflowcontrol.ConnectedStream = &connectedStream{} + +func newConnectedStream(stream kvflowcontrol.Stream) *connectedStream { + return &connectedStream{ + stream: stream, + ch: make(chan struct{}), + } +} + +// Stream is part of the kvflowcontrol.ConnectedStream interface. +func (b *connectedStream) Stream() kvflowcontrol.Stream { + return b.stream +} + +// Disconnected is part of the kvflowcontrol.ConnectedStream interface. +func (b *connectedStream) Disconnected() <-chan struct{} { + return b.ch +} + +// Disconnect is used to disconnect the underlying replication stream, +// unblocking all waiting requests. +func (b *connectedStream) Disconnect() { + if atomic.CompareAndSwapInt32(&b.disconnected, 0, 1) { + close(b.ch) + } +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go new file mode 100644 index 000000000000..51b9bd56949d --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go @@ -0,0 +1,297 @@ +// Copyright 2023 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 kvflowhandle + +import ( + "context" + "sort" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" +) + +// Handle is a concrete implementation of the kvflowcontrol.Handle +// interface. It's held on replicas initiating replication traffic, managing +// multiple Streams (one per active replica) underneath. +type Handle struct { + controller kvflowcontrol.Controller + + mu struct { + syncutil.Mutex + connections []*connectedStream + // perStreamTokenTracker tracks flow token deductions for each stream. + // It's used to release tokens back to the controller once log entries + // (identified by their log positions) have been admitted below-raft, + // streams disconnect, or the handle closed entirely. + perStreamTokenTracker map[kvflowcontrol.Stream]*kvflowtokentracker.Tracker + closed bool + } +} + +// New constructs a new Handle. +func New(controller kvflowcontrol.Controller) *Handle { + h := &Handle{ + controller: controller, + } + h.mu.perStreamTokenTracker = map[kvflowcontrol.Stream]*kvflowtokentracker.Tracker{} + return h +} + +var _ kvflowcontrol.Handle = &Handle{} + +// Admit is part of the kvflowcontrol.Handle interface. +func (h *Handle) Admit(ctx context.Context, pri admissionpb.WorkPriority, ct time.Time) error { + if h == nil { + // TODO(irfansharif): This can happen if we're proposing immediately on + // a newly split off RHS that doesn't know it's a leader yet (so we + // haven't initialized a handle). We don't want to deduct/track flow + // tokens for it; the handle only has a lifetime while we explicitly + // know that we're the leaseholder+leader. It's ok for the caller to + // later invoke ReturnTokensUpto even with a no-op DeductTokensFor since + // it can only return what has been actually been deducted. + // + // As for cluster settings that disable flow control entirely or only + // for regular traffic, that can be dealt with at the caller by not + // calling .Admit() and ensuring we use the right raft entry encodings. + return nil + } + + h.mu.Lock() + if h.mu.closed { + log.Errorf(ctx, "operating on a closed handle") + return nil + } + connections := h.mu.connections + h.mu.Unlock() + for _, c := range connections { + if err := h.controller.Admit(ctx, pri, ct, c); err != nil { + return err + } + } + return nil +} + +// DeductTokensFor is part of the kvflowcontrol.Handle interface. +func (h *Handle) DeductTokensFor( + ctx context.Context, + pri admissionpb.WorkPriority, + pos kvflowcontrolpb.RaftLogPosition, + tokens kvflowcontrol.Tokens, +) { + if h == nil { + // TODO(irfansharif): See TODO around nil receiver check in Admit(). + return + } + + _ = h.deductTokensForInner(ctx, pri, pos, tokens) +} + +func (h *Handle) deductTokensForInner( + ctx context.Context, + pri admissionpb.WorkPriority, + pos kvflowcontrolpb.RaftLogPosition, + tokens kvflowcontrol.Tokens, +) (streams []kvflowcontrol.Stream) { + h.mu.Lock() + defer h.mu.Unlock() + if h.mu.closed { + log.Errorf(ctx, "operating on a closed handle") + return nil // unused return value in production code + } + + for _, c := range h.mu.connections { + h.controller.DeductTokens(ctx, pri, tokens, c.Stream()) + h.mu.perStreamTokenTracker[c.Stream()].Track(ctx, pri, tokens, pos) + streams = append(streams, c.Stream()) + } + return streams +} + +// ReturnTokensUpto is part of the kvflowcontrol.Handle interface. +func (h *Handle) ReturnTokensUpto( + ctx context.Context, + pri admissionpb.WorkPriority, + upto kvflowcontrolpb.RaftLogPosition, + stream kvflowcontrol.Stream, +) { + if h == nil { + // We're trying to release tokens to a handle that no longer exists, + // likely because we've lost the lease and/or raft leadership since + // we acquired flow tokens originally. At that point the handle was + // closed, and all flow tokens were returned back to the controller. + // There's nothing left for us to do here. + // + // NB: It's possible to have reacquired leadership and re-initialize a + // handle. We still want to ignore token returns from earlier + // terms/leases (which were already returned to the controller). To that + // end, we rely on the handle being re-initialized with an empty tracker + // -- there's simply nothing to double return. Also, when connecting + // streams on fresh handles, we specify a lower-bound raft log position. + // The log position corresponds to when the lease/leadership was + // acquired (whichever comes after). This is used to assert against + // regressions in token deductions (i.e. deducting tokens for indexes + // lower than the current term/lease). + return + } + + h.mu.Lock() + defer h.mu.Unlock() + if h.mu.closed { + log.Errorf(ctx, "operating on a closed handle") + return + } + + tokens := h.mu.perStreamTokenTracker[stream].Untrack(ctx, pri, upto) + h.controller.ReturnTokens(ctx, pri, tokens, stream) +} + +// ConnectStream is part of the kvflowcontrol.Handle interface. +func (h *Handle) ConnectStream( + ctx context.Context, pos kvflowcontrolpb.RaftLogPosition, stream kvflowcontrol.Stream, +) { + h.mu.Lock() + defer h.mu.Unlock() + if h.mu.closed { + log.Errorf(ctx, "operating on a closed handle") + return + } + + if _, ok := h.mu.perStreamTokenTracker[stream]; ok { + log.Fatalf(ctx, "reconnecting already connected stream: %s", stream) + } + h.mu.connections = append(h.mu.connections, newConnectedStream(stream)) + sort.Slice(h.mu.connections, func(i, j int) bool { + // Sort connections based on store IDs (this is the order in which we + // invoke Controller.Admit) for predictability. If in the future we use + // flow tokens for raft log catchup (see I11 and [^9] in + // kvflowcontrol/doc.go), we may want to introduce an Admit-variant that + // both blocks and deducts tokens before sending catchup MsgApps. In + // that case, this sorting will help avoid deadlocks. + return h.mu.connections[i].Stream().StoreID < h.mu.connections[j].Stream().StoreID + }) + h.mu.perStreamTokenTracker[stream] = kvflowtokentracker.New(pos, nil /* knobs */) +} + +// DisconnectStream is part of the kvflowcontrol.Handle interface. +func (h *Handle) DisconnectStream(ctx context.Context, stream kvflowcontrol.Stream) { + h.mu.Lock() + defer h.mu.Unlock() + h.disconnectStreamLocked(ctx, stream) +} + +func (h *Handle) disconnectStreamLocked(ctx context.Context, stream kvflowcontrol.Stream) { + if h.mu.closed { + log.Errorf(ctx, "operating on a closed handle") + return + } + if _, ok := h.mu.perStreamTokenTracker[stream]; !ok { + log.Fatalf(ctx, "disconnecting non-existent stream: %s", stream) + } + + h.mu.perStreamTokenTracker[stream].Iter(ctx, + func(pri admissionpb.WorkPriority, tokens kvflowcontrol.Tokens) { + h.controller.ReturnTokens(ctx, pri, tokens, stream) + }, + ) + delete(h.mu.perStreamTokenTracker, stream) + + streamIdx := -1 + for i := range h.mu.connections { + if h.mu.connections[i].Stream() == stream { + streamIdx = i + break + } + } + connection := h.mu.connections[streamIdx] + connection.Disconnect() + h.mu.connections = append(h.mu.connections[:streamIdx], h.mu.connections[streamIdx+1:]...) + + // TODO(irfansharif): Optionally record lower bound raft log positions for + // disconnected streams to guard against regressions when (re-)connecting -- + // it must be done with higher positions. +} + +// Close is part of the kvflowcontrol.Handle interface. +func (h *Handle) Close(ctx context.Context) { + if h == nil { + return // nothing to do + } + + h.mu.Lock() + defer h.mu.Unlock() + if h.mu.closed { + log.Errorf(ctx, "operating on a closed handle") + return + } + + for _, connection := range h.mu.connections { + h.disconnectStreamLocked(ctx, connection.Stream()) + } + h.mu.closed = true +} + +// TestingNonBlockingAdmit is a non-blocking alternative to Admit() for use in +// tests. +// - it checks if we have a non-zero number of flow tokens for all connected +// streams; +// - if we do, we return immediately with admitted=true; +// - if we don't, we return admitted=false and two sets of callbacks: +// (i) signaled, which can be polled to check whether we're ready to try and +// admitting again. There's one per underlying stream. +// (ii) admit, which can be used to try and admit again. If still not +// admitted, callers are to wait until they're signaled again. There's one +// per underlying stream. +func (h *Handle) TestingNonBlockingAdmit( + ctx context.Context, pri admissionpb.WorkPriority, +) (admitted bool, signaled []func() bool, admit []func() bool) { + h.mu.Lock() + if h.mu.closed { + log.Fatalf(ctx, "operating on a closed handle") + } + connections := h.mu.connections + h.mu.Unlock() + + type testingNonBlockingController interface { + TestingNonBlockingAdmit( + pri admissionpb.WorkPriority, connection kvflowcontrol.ConnectedStream, + ) (admitted bool, signaled func() bool, admit func() bool) + } + + admitted = true + testingController := h.controller.(testingNonBlockingController) + for _, c := range connections { + connectionAdmitted, connectionSignaled, connectionAdmit := testingController.TestingNonBlockingAdmit(pri, c) + if connectionAdmitted { + continue + } + + admitted = false + signaled = append(signaled, connectionSignaled) + admit = append(admit, connectionAdmit) + } + return admitted, signaled, admit +} + +// TestingDeductTokensForInner exposes deductTokensForInner for testing +// purposes. +func (h *Handle) TestingDeductTokensForInner( + ctx context.Context, + pri admissionpb.WorkPriority, + pos kvflowcontrolpb.RaftLogPosition, + tokens kvflowcontrol.Tokens, +) []kvflowcontrol.Stream { + return h.deductTokensForInner(ctx, pri, pos, tokens) +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go new file mode 100644 index 000000000000..cf6ccfcbd676 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go @@ -0,0 +1,105 @@ +// Copyright 2023 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 kvflowhandle_test + +import ( + "context" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowhandle" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/stretchr/testify/require" +) + +// TestHandleAdmit tests the blocking behavior of Handle.Admit(): +// - we block until there are flow tokens available; +// - we unblock when streams without flow tokens are disconnected; +// - we unblock when the handle is closed. +func TestHandleAdmit(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + stream := kvflowcontrol.Stream{TenantID: roachpb.MustMakeTenantID(42), StoreID: roachpb.StoreID(42)} + pos := func(d uint64) kvflowcontrolpb.RaftLogPosition { + return kvflowcontrolpb.RaftLogPosition{Term: 1, Index: d} + } + + for _, tc := range []struct { + name string + unblockFn func(context.Context, kvflowcontrol.Handle) + }{ + { + name: "blocks-for-tokens", + unblockFn: func(ctx context.Context, handle kvflowcontrol.Handle) { + // Return tokens tied to pos=1 (16MiB worth); the call to + // .Admit() should unblock. + handle.ReturnTokensUpto(ctx, admissionpb.NormalPri, pos(1), stream) + }, + }, + { + name: "unblocked-when-stream-disconnects", + unblockFn: func(ctx context.Context, handle kvflowcontrol.Handle) { + // Disconnect the stream; the call to .Admit() should unblock. + handle.DisconnectStream(ctx, stream) + }, + }, + { + name: "unblocked-when-closed", + unblockFn: func(ctx context.Context, handle kvflowcontrol.Handle) { + // Close the handle; the call to .Admit() should unblock. + handle.Close(ctx) + }, + }, + } { + t.Run(tc.name, func(t *testing.T) { + controller := kvflowcontroller.New(metric.NewRegistry(), cluster.MakeTestingClusterSettings(), hlc.NewClockForTesting(nil)) + handle := kvflowhandle.New(controller) + + // Connect a single stream at pos=0 and deplete all 16MiB of regular + // tokens at pos=1. + handle.ConnectStream(ctx, pos(0), stream) + handle.DeductTokensFor(ctx, admissionpb.NormalPri, pos(1), kvflowcontrol.Tokens(16<<20 /* 16MiB */)) + + // Invoke .Admit() in a separate goroutine, and test below whether + // the goroutine is blocked. + admitCh := make(chan struct{}) + go func() { + require.NoError(t, handle.Admit(ctx, admissionpb.NormalPri, time.Time{})) + close(admitCh) + }() + + select { + case <-admitCh: + t.Fatalf("unexpectedly admitted") + case <-time.After(10 * time.Millisecond): + } + + tc.unblockFn(ctx, handle) + + select { + case <-admitCh: + case <-time.After(5 * time.Second): + t.Fatalf("didn't get admitted") + } + }) + } +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/BUILD.bazel new file mode 100644 index 000000000000..826ca4c1f9aa --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/BUILD.bazel @@ -0,0 +1,33 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "kvflowsimulator_test", + srcs = ["simulation_test.go"], + args = ["-test.timeout=295s"], + data = glob(["testdata/**"]), + deps = [ + "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle", + "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker", + "//pkg/roachpb", + "//pkg/settings/cluster", + "//pkg/testutils/datapathutils", + "//pkg/util/admission/admissionpb", + "//pkg/util/asciitsdb", + "//pkg/util/hlc", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/metric", + "//pkg/util/timeutil", + "@com_github_cockroachdb_datadriven//:datadriven", + "@com_github_dustin_go_humanize//:go-humanize", + "@com_github_guptarohit_asciigraph//:asciigraph", + "@com_github_mkungla_bexp_v3//:bexp", + "@com_github_stretchr_testify//require", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go new file mode 100644 index 000000000000..0d37b35f06c8 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go @@ -0,0 +1,1020 @@ +// Copyright 2023 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 kvflowsimulator + +import ( + "context" + "fmt" + "strconv" + "strings" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowhandle" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/asciitsdb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/datadriven" + "github.com/dustin/go-humanize" + "github.com/guptarohit/asciigraph" + "github.com/mkungla/bexp/v3" + "github.com/stretchr/testify/require" +) + +// TestUsingSimulation is a data-driven test for kvflowcontrol.Controller and +// kvflowcontrol.Handle. It gives package authors a way to understand how flow +// tokens are maintained for individual replication streams, how write bandwidth +// is shaped by these tokens, and how requests queue/dequeue internally. We +// provide the following syntax: +// +// - "init" +// [handle=] +// .... +// Initialize the flow controller and test simulator. Optionally, initialize +// named handles for subsequent use. +// +// ----------------------------------------------------------------------------- +// +// - "timeline" +// t=[,) class={regular,elastic} \ +// stream=t/s adjust={+,-}/s rate=/s \ +// [deduction-delay=] (A) +// .... +// t=[,) handle= class={regular,elastic} \ +// adjust={+,-}/s rate=/s [stream=t/s] \ +// [deduction-delay=] (B) +// .... +// t= handle= op=connect stream=t/s \ +// log-position=/ (C) +// .... +// t= handle= op=disconnect stream=t/s (D) +// .... +// t= handle= op={snapshot,close} (E) +// .... +// +// Set up timelines to simulate. There are a few forms: +// +// A. Creates a "thread" that operates during the given time range +// t=[tstart,tend), issuing the specified 'rate' of requests of the given +// work 'class', over the given 'stream', where the flow tokens are +// {deducted,returned} with the given bandwidth. The 'rate' controls the +// granularity of token adjustment, i.e. if adjust=+100bytes/s and +// rate=5/s, then each return adjusts by +100/5 = +20bytes. If flow +// tokens are being deducted (-ve 'adjust'), they go through Admit() +// followed by DeductTokens(). If they're being returned (+ve 'adjust'), +// they simply go through ReturnTokens(). The optional 'deduction-delay' +// parameter controls the number of ticks between each request being +// granted admission and it deducting the corresponding flow tokens. +// +// B. Similar to A except using a named handle instead, which internally +// deducts tokens from all connected streams or if returning tokens, does so +// for the named stream. Token deductions from a handle are tied to +// monotonically increasing raft log positions starting from position the +// underlying stream was connected to (using C). When returning tokens, we +// translate the byte token value to the corresponding raft log prefix +// (token returns with handles are in terms of raft log positions). +// +// C. Connects the named handle to the specific stream, starting at the +// given log position. Subsequent token deductions using the handle will +// deduct from the given stream. +// +// D. Disconnects the specific stream from the named handle. All deducted +// flow tokens (using the named handle) from that specific stream are +// released. Future token deductions/returns (when using the named handle) +// don't deduct from/return to the stream. +// +// E. Close or snapshot the named handle. When closing a handle, all +// deducted flow tokens are released and subsequent operations are noops. +// Snapshots record the internal state (what tokens have been +// deducted-but-not-returned, and for what log positions). This can later be +// rendered using the "snapshot" directive. +// +// ----------------------------------------------------------------------------- +// +// - "simulate" [t=[,)] +// Simulate timelines until the optionally specified timestamp (the start +// time is ignored). If no t is specified, the largest end time of all +// registered timelines is used instead. +// +// ----------------------------------------------------------------------------- +// +// - "plot" [height=] [width=] [precision=] \ +// [t=[,)] +// unit= [rate=true] +// .... +// Plot the flow controller specified metrics (and optionally its rate of +// change) with the specified units. The following metrics are supported: +// a. kvadmission.flow_controller.{regular,elastic}_tokens_{available,deducted,returned} +// b. kvadmission.flow_controller.{regular,elastic}_requests_{waiting,admitted,errored} +// c. kvadmission.flow_controller.{regular,elastic}{,_blocked}_stream_count +// d. kvadmission.flow_controller.{regular,elastic}_wait_duration +// To overlay metrics onto the same plot, the selector supports curly brace +// expansion. If the unit is one of {MiB,MB,KB,KiB,s,ms,us,μs}, or the +// bandwidth equivalents (/s), the y-axis is automatically +// converted. +// +// ----------------------------------------------------------------------------- +// +// - "snapshots" handle= +// Render any captured "snapshots" (what tokens were +// deducted-but-not-returned as of some timestamp, for what log positions) +// for the named handle. +// +// Internally we make use of a test-only non-blocking interface for the flow +// controller to support deterministic simulation. +func TestUsingSimulation(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) { + var ( + controller *kvflowcontroller.Controller + simulator *simulator + tsdb *asciitsdb.TSDB + mtime *timeutil.ManualTime + replicaHandles map[string]*replicaHandle + ) + + ctx := context.Background() + replicaHandles = map[string]*replicaHandle{} + datadriven.RunTest(t, path, + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + registry := metric.NewRegistry() + tsdb = asciitsdb.New(t, registry) + mtime = timeutil.NewManualTime(tzero) + controller = kvflowcontroller.New( + registry, + cluster.MakeTestingClusterSettings(), + hlc.NewClockForTesting(mtime), + ) + tsdb.Register(controller.TestingMetrics()) + simulator = newSimulator(t, controller, tsdb, mtime) + for _, line := range strings.Split(d.Input, "\n") { + name := strings.TrimPrefix(strings.TrimSpace(line), "handle=") + replicaHandles[name] = &replicaHandle{ + handle: kvflowhandle.New(controller), + deductionTracker: make(map[kvflowcontrol.Stream]*kvflowtokentracker.Tracker), + outstandingReturns: make(map[kvflowcontrol.Stream]kvflowcontrol.Tokens), + snapshots: make([]snapshot, 0), + } + } + return "" + + case "timeline": + require.NotNilf(t, controller, "uninitialized flow controller (did you use 'init'?)") + require.NotNilf(t, simulator, "uninitialized simulator (did you use 'init'?)") + + for _, line := range strings.Split(d.Input, "\n") { + parts := strings.Fields(line) + + var tl timeline + for i := range parts { + parts[i] = strings.TrimSpace(parts[i]) + inner := strings.Split(parts[i], "=") + require.Len(t, inner, 2) + arg := strings.TrimSpace(inner[1]) + + switch { + case strings.HasPrefix(parts[i], "t="): + // Parse t={,[,)}. + ranged := strings.HasPrefix(arg, "[") + if ranged { + arg = strings.TrimSuffix(strings.TrimPrefix(arg, "["), ")") + args := strings.Split(arg, ",") + dur, err := time.ParseDuration(args[0]) + require.NoError(t, err) + tl.tstart = mtime.Now().Add(dur) + dur, err = time.ParseDuration(args[1]) + require.NoError(t, err) + tl.tend = mtime.Now().Add(dur) + } else { + dur, err := time.ParseDuration(arg) + require.NoError(t, err) + tl.tstart = mtime.Now().Add(dur) + } + + case strings.HasPrefix(parts[i], "class="): + // Parse class={regular,elastic}. + switch arg { + case "regular": + tl.pri = admissionpb.NormalPri + case "elastic": + tl.pri = admissionpb.BulkNormalPri + default: + t.Fatalf("unexpected class: %s", parts[i]) + } + + case strings.HasPrefix(parts[i], "stream="): + // Parse stream=t/s. + inner := strings.Split(arg, "/") + require.Len(t, inner, 2) + ti, err := strconv.Atoi(strings.TrimPrefix(inner[0], "t")) + require.NoError(t, err) + si, err := strconv.Atoi(strings.TrimPrefix(inner[1], "s")) + require.NoError(t, err) + tl.stream = kvflowcontrol.Stream{ + TenantID: roachpb.MustMakeTenantID(uint64(ti)), + StoreID: roachpb.StoreID(si), + } + + case strings.HasPrefix(parts[i], "adjust="): + // Parse adjust={+,-}/s. + isPositive := strings.Contains(arg, "+") + arg = strings.TrimPrefix(arg, "+") + arg = strings.TrimPrefix(arg, "-") + bytes, err := humanize.ParseBytes(strings.TrimSuffix(arg, "/s")) + require.NoError(t, err) + tl.delta = kvflowcontrol.Tokens(int64(bytes)) + if !isPositive { + tl.delta = -tl.delta + } + + case strings.HasPrefix(parts[i], "rate="): + // Parse rate=/s. + var err error + tl.rate, err = strconv.Atoi(strings.TrimSuffix(arg, "/s")) + require.NoError(t, err) + + case strings.HasPrefix(parts[i], "deduction-delay="): + // Parse deduction-delay=. + dur, err := time.ParseDuration(arg) + require.NoError(t, err) + tl.deductionDelay = int(dur.Nanoseconds() / tick.Nanoseconds()) + + case strings.HasPrefix(parts[i], "handle="): + // Parse handle=. + var ok bool + tl.replicaHandle, ok = replicaHandles[arg] + require.True(t, ok, "expected to find named handle %q, was it initialized?", arg) + + case strings.HasPrefix(parts[i], "op="): + // Parse op=. + require.True(t, arg == "connect" || arg == "disconnect" || + arg == "close" || arg == "snapshot") + tl.handleOp = arg + + case strings.HasPrefix(parts[i], "log-position="): + // Parse log-position=/. + inner := strings.Split(arg, "/") + require.Len(t, inner, 2) + term, err := strconv.Atoi(inner[0]) + require.NoError(t, err) + index, err := strconv.Atoi(inner[1]) + require.NoError(t, err) + tl.position = kvflowcontrolpb.RaftLogPosition{ + Term: uint64(term), + Index: uint64(index), + } + + default: + t.Fatalf("unrecognized prefix: %s", parts[i]) + } + } + + simulator.timeline(tl) + } + return "" + + case "simulate": + require.NotNilf(t, simulator, "uninitialized simulator (did you use 'init'?)") + var end time.Time + if d.HasArg("t") { + // Parse t=[,), but ignoring the + // start time. + var tstr string + d.ScanArgs(t, "t", &tstr) + tstr = strings.TrimSuffix(strings.TrimPrefix(tstr, "["), ")") + args := strings.Split(tstr, ",") + dur, err := time.ParseDuration(args[1]) + require.NoError(t, err) + end = mtime.Now().Add(dur) + } + simulator.simulate(ctx, end) + return "" + + case "plot": + var h, w, p = 15, 40, 1 + if d.HasArg("height") { + d.ScanArgs(t, "height", &h) + } + if d.HasArg("width") { + d.ScanArgs(t, "width", &w) + } + if d.HasArg("precision") { + d.ScanArgs(t, "precision", &p) + } + + var buf strings.Builder + for i, line := range strings.Split(d.Input, "\n") { + var ( + selector, unit string + rated bool + ) + parts := strings.Fields(line) + for i, part := range parts { + part = strings.TrimSpace(part) + if i == 0 { + selector = part + continue + } + + if strings.HasPrefix(part, "rate=") { + var err error + rated, err = strconv.ParseBool(strings.TrimPrefix(part, "rate=")) + require.NoError(t, err) + } + + if strings.HasPrefix(part, "unit=") { + unit = strings.TrimPrefix(part, "unit=") + } + } + + caption := strings.TrimPrefix(selector, "kvadmission.flow_controller.") + if rated { + caption = fmt.Sprintf("rate(%s)", caption) + } + caption = fmt.Sprintf("%s (%s)", caption, unit) + + options := []asciitsdb.Option{ + asciitsdb.WithGraphOptions( + asciigraph.Height(h), + asciigraph.Width(w), + asciigraph.Precision(uint(p)), + asciigraph.Caption(caption), + ), + } + if rated { + options = append(options, asciitsdb.WithRate(int(time.Second/metricTick))) + } + switch unit { + case "μs", "us", "microseconds": + options = append(options, asciitsdb.WithDivisor(float64(time.Microsecond.Nanoseconds())) /* ns => μs conversion */) + case "ms", "milliseconds": + options = append(options, asciitsdb.WithDivisor(float64(time.Millisecond.Nanoseconds())) /* ns => μs conversion */) + case "s", "seconds": + options = append(options, asciitsdb.WithDivisor(float64(time.Second.Nanoseconds())) /* ns => μs conversion */) + case "MiB", "MiB/s": + options = append(options, asciitsdb.WithDivisor(humanize.MiByte) /* 1 MiB */) + case "MB", "MB/s": + options = append(options, asciitsdb.WithDivisor(humanize.MByte) /* 1 MB */) + case "KiB", "KiB/s": + options = append(options, asciitsdb.WithDivisor(humanize.KiByte) /* 1 KiB */) + case "KB", "KB/s": + options = append(options, asciitsdb.WithDivisor(humanize.KByte) /* 1 KB */) + default: + } + + if d.HasArg("t") { + // Parse t=[,). + var tstr string + d.ScanArgs(t, "t", &tstr) + tstr = strings.TrimSuffix(strings.TrimPrefix(tstr, "["), ")") + args := strings.Split(tstr, ",") + + dur, err := time.ParseDuration(args[0]) + require.NoError(t, err) + start := tzero.Add(dur) + options = append(options, asciitsdb.WithOffset(start.Sub(tzero).Nanoseconds()/metricTick.Nanoseconds())) + + dur, err = time.ParseDuration(args[1]) + require.NoError(t, err) + end := tzero.Add(dur) + options = append(options, asciitsdb.WithLimit(end.Sub(start).Nanoseconds()/metricTick.Nanoseconds())) + } + if i > 0 { + buf.WriteString("\n\n\n") + } + metrics := bexp.Parse(strings.TrimSpace(selector)) + buf.WriteString(tsdb.Plot(metrics, options...)) + } + return buf.String() + + case "snapshots": + var name string + d.ScanArgs(t, "handle", &name) + replicaHandle, ok := replicaHandles[name] + require.True(t, ok, "expected to find named handle %q, was it initialized?", name) + var buf strings.Builder + for i, s := range replicaHandle.snapshots { + if i > 0 { + buf.WriteString("\n") + } + buf.WriteString(fmt.Sprintf("t=%s stream=%s\n", s.time.Sub(tzero), s.stream)) + buf.WriteString(fmt.Sprintf(" %s", s.data)) + } + return buf.String() + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }, + ) + }) +} + +// tick is the smallest time interval that we simulate (1ms). +const tick = time.Millisecond + +// metricTick is the time interval over which we scrape metrics for ASCII plots. +const metricTick = 100 * tick + +// simulator for various flow control components. It can be annotated with +// various timelines +type simulator struct { + t *testing.T + controller *kvflowcontroller.Controller + ticker []ticker + tsdb *asciitsdb.TSDB + mtime *timeutil.ManualTime +} + +func newSimulator( + t *testing.T, + controller *kvflowcontroller.Controller, + tsdb *asciitsdb.TSDB, + mtime *timeutil.ManualTime, +) *simulator { + return &simulator{ + t: t, + tsdb: tsdb, + controller: controller, + mtime: mtime, + } +} + +func (s *simulator) timeline(tl timeline) { + // See commentary on the timeline type for its various forms; we do the + // validation here, in-line. For each timeline, we construct an appropriate + // ticker that's ticked during the simulation. + + if tl.replicaHandle == nil { + // Form A, interacting with the kvflowcontrol.Controller directly. + + if tl.rate == 0 { + return // nothing to do + } + + require.NotZero(s.t, tl.tend) + require.NotZero(s.t, tl.stream) + require.LessOrEqual(s.t, tl.rate, 1000) + + s.ticker = append(s.ticker, &controllerTicker{ + t: s.t, + controller: s.controller, + + tstart: tl.tstart, + tend: tl.tend, + pri: tl.pri, + stream: tl.stream, + + deductionDelay: tl.deductionDelay, + deduct: make(map[time.Time][]func()), + waitingRequests: make(map[time.Time]waitingRequestInController), + + // Using the parameters above, we figure out two things: + // - On which ticks do we adjust flow tokens? + // - How much by, each time? + // + // If the request rate we're simulating is: + // - 1000/sec, we adjust flow tokens every tick(=1ms). + // - 500/sec, we adjust flow tokens every 2 ticks (=2ms). + // - .... + // + // How much do we adjust by each time? Given we're making 'rate' requests + // per second, and have to deduct 'delta' tokens per second, each request + // just deducts delta/rate. + mod: int(time.Second/tick) / tl.rate, + delta: kvflowcontrol.Tokens(int(tl.delta) / tl.rate), + }) + return + } + + // Forms B-E, using the kvflowcontrol.Handle instead. + require.NotNil(s.t, tl.replicaHandle) + + if tl.handleOp != "" { + // Forms C-E, where we're either connecting/disconnecting a named + // stream, or closing/snapshotting a handle. + require.Zero(s.t, tl.tend) + if tl.handleOp == "connect" { + // Form C. + require.NotZero(s.t, tl.stream) + require.NotZero(s.t, tl.position) + } + if tl.handleOp == "disconnect" { + // Form D. + require.NotZero(s.t, tl.stream) + } + + s.ticker = append(s.ticker, &handleOpTicker{ + t: s.t, + tstart: tl.tstart, + replicaHandle: tl.replicaHandle, + op: tl.handleOp, + stream: tl.stream, + position: tl.position, + }) + return + } + + // Form B, where we're deducting/returning flow tokens using + // kvflowcontrol.Handle. + if tl.rate == 0 { + return // nothing to do + } + + require.NotZero(s.t, tl.tend) + require.Zero(s.t, tl.position) + s.ticker = append(s.ticker, &handleTicker{ + t: s.t, + + tstart: tl.tstart, + tend: tl.tend, + pri: tl.pri, + replicaHandle: tl.replicaHandle, + stream: tl.stream, + + deductionDelay: tl.deductionDelay, + deduct: make(map[time.Time][]func()), + waitingRequests: make(map[time.Time]waitingRequestInHandle), + + // See commentary on the controllerTicker construction above. + mod: int(time.Second/tick) / tl.rate, + delta: kvflowcontrol.Tokens(int(tl.delta) / tl.rate), + }) +} + +// tzero represents the t=0, the earliest possible time. All other +// t={,[,)) is relative to this time. +var tzero = timeutil.Unix(0, 0) + +func (s *simulator) simulate(ctx context.Context, tend time.Time) { + s.mtime.Backwards(s.mtime.Since(tzero)) // reset to tzero + s.tsdb.Clear() + for i := range s.ticker { + s.ticker[i].reset() + if s.ticker[i].end().After(tend) { + tend = s.ticker[i].end() + } + } + + for { + t := s.mtime.Now() + if t.After(tend) || t.Equal(tend) { + break + } + for i := range s.ticker { + s.ticker[i].tick(ctx, t) + } + if t.UnixNano()%metricTick.Nanoseconds() == 0 { + s.tsdb.Scrape(ctx) + } + s.mtime.Advance(tick) + } +} + +// replicaHandle is used to simulate a leaseholder+leader replica that embeds a +// kvflowcontrol.Handle to do flow control. +type replicaHandle struct { + handle *kvflowhandle.Handle + closed bool + // The current raft log position. This is incremented on every flow token + // deduction, to simulate a command being proposed to raft. + quorumLogPosition kvflowcontrolpb.RaftLogPosition + // deductionTracker is used to track flow token deductions and the + // corresponding quorumLogPosition in the simulator. When returning flow + // tokens, we specify a bandwidth (+2MiB/s); we use this per-stream tracker + // to figure out what raft log positions to free up tokens upto. + deductionTracker map[kvflowcontrol.Stream]*kvflowtokentracker.Tracker + outstandingReturns map[kvflowcontrol.Stream]kvflowcontrol.Tokens + snapshots []snapshot +} + +type snapshot struct { + time time.Time + stream kvflowcontrol.Stream + data string +} + +func (h *replicaHandle) deductTokens( + ctx context.Context, pri admissionpb.WorkPriority, tokens kvflowcontrol.Tokens, +) { + if h.closed { + return + } + // Increment the quorum log position -- all token deductions are bound to + // incrementing log positions. + h.quorumLogPosition.Index += 1 + streams := h.handle.TestingDeductTokensForInner(ctx, pri, h.quorumLogPosition, tokens) + for _, stream := range streams { + h.deductionTracker[stream].Track(ctx, pri, tokens, h.quorumLogPosition) + } +} + +func (h *replicaHandle) returnTokens( + ctx context.Context, + pri admissionpb.WorkPriority, + tokens kvflowcontrol.Tokens, + stream kvflowcontrol.Stream, +) { + if h.closed { + return + } + + // Iterate through tracked deductions of the given priority until we've + // found the log index that accumulates to the # of tokens we want to + // return. Track any leftovers to consider the next time. + h.outstandingReturns[stream] += tokens + + returnUpto := kvflowcontrolpb.RaftLogPosition{} + h.deductionTracker[stream].TestingIter( + func(priority admissionpb.WorkPriority, tokens kvflowcontrol.Tokens, pos kvflowcontrolpb.RaftLogPosition) bool { + if pri != priority { + return true + } + + if (h.outstandingReturns[stream] - tokens) >= 0 { + h.outstandingReturns[stream] -= tokens + returnUpto = pos + return true + } + return false + }, + ) + + if !returnUpto.Equal(kvflowcontrolpb.RaftLogPosition{}) { + h.handle.ReturnTokensUpto(ctx, pri, returnUpto, stream) + _ = h.deductionTracker[stream].Untrack(ctx, pri, returnUpto) + } +} + +// timeline is a sequence of events being simulated. It comes in the following +// forms: +// +// A. t=[,) class={regular,elastic} \ +// stream=t/s adjust={+,-}/s rate=/s \ +// [deduction-delay=] +// B. t=[,) class={regular,elastic} handle= \ +// adjust={+,-}/s rate=/s [deduction-delay=] +// C. t= handle= op=connect stream=t/s \ +// log-position=/ +// D. t= handle= op=disconnect stream=t/s +// E. t= handle= op={snapshot,close} +type timeline struct { + // Start and (optional) end time for action being simulated. + tstart, tend time.Time + // Priority (if applicable) of work on behalf of which we're + // deducting/returning flow tokens through kvflowcontrol.Controller or + // kvflowcontrol.Handle. + pri admissionpb.WorkPriority + // Stream over which we're deducting/returning flow tokens (form A) or the + // stream we're connecting to/disconnecting from a given + // kvflowcontrol.Handle. + stream kvflowcontrol.Stream + // The number of tokens either being deducted or returned over + // [tstart,tend). Only applicable to forms A and B. + delta kvflowcontrol.Tokens + // The rate at which we adjust flow tokens, controlling the granularity at + // which 'delta' is adjusted. Only applicable to forms A and B. + rate int + // The # of ticks post-Admit() when we actually deduct tokens. Only + // applicable to forms A and B. + deductionDelay int + // Scoped replica handle. Only applicable to forms B-E, when we're not + // dealing with the kvflowcontrol.Controller directly. + replicaHandle *replicaHandle + // The specific operation to run on a kvflowcontrol.Handle. Only applicable + // to forms C-E. + handleOp string + // The log position at which we start issuing writes/deducting tokens (form + // B) or the position at which we connect a given stream (form C). + position kvflowcontrolpb.RaftLogPosition +} + +type ticker interface { + tick(ctx context.Context, t time.Time) + reset() + end() time.Time +} + +// controllerTicker is a concrete implementation of the ticker interface, used +// to simulate token adjustments on the kvflowcontrol.Controller directly. +type controllerTicker struct { + t *testing.T + tstart, tend time.Time + pri admissionpb.WorkPriority + stream kvflowcontrol.Stream + delta kvflowcontrol.Tokens + controller *kvflowcontroller.Controller + mod, ticks int // used to control the ticks at which we interact with the controller + deductionDelay int + + deduct map[time.Time][]func() + waitingRequests map[time.Time]waitingRequestInController +} + +var _ ticker = &controllerTicker{} + +// tick is part of the ticker interface. +func (ct *controllerTicker) tick(ctx context.Context, t time.Time) { + if ds, ok := ct.deduct[t]; ok { + for _, deduct := range ds { + deduct() + } + delete(ct.deduct, t) + } + for key, waitingRequest := range ct.waitingRequests { + // Process all waiting requests from earlier. Do this even if t > + // ct.tend since these requests could've been generated earlier. + if waitingRequest.ctx.Err() != nil { + delete(ct.waitingRequests, key) + continue + } + if !waitingRequest.signaled() { + continue + } + if waitingRequest.admit() { + // Request admitted; proceed with token deductions. + if ct.deductionDelay == 0 { + ct.controller.TestingAdjustTokens(ctx, ct.pri, ct.delta, ct.stream) + } else { + future := t.Add(tick * time.Duration(ct.deductionDelay)) + ct.deduct[future] = append(ct.deduct[future], func() { + ct.controller.TestingAdjustTokens(ctx, ct.pri, ct.delta, ct.stream) + }) + } + delete(ct.waitingRequests, key) + return + } + } + + if t.Before(ct.tstart) || (t.After(ct.tend) || t.Equal(ct.tend)) { + return // we're outside our [ct.tstart, ct.tend), there's nothing left to do + } + + defer func() { ct.ticks += 1 }() + if ct.ticks%ct.mod != 0 { + return // nothing to do in this tick + } + + if ct.delta >= 0 { // return tokens + ct.controller.TestingAdjustTokens(ctx, ct.pri, ct.delta, ct.stream) + return + } + + admitted, signaled, admit := ct.controller.TestingNonBlockingAdmit(ct.pri, &unbreakableStream{ct.stream}) + if admitted { + // Request admitted; proceed with token deductions. + if ct.deductionDelay == 0 { + ct.controller.TestingAdjustTokens(ctx, ct.pri, ct.delta, ct.stream) + } else { + future := t.Add(tick * time.Duration(ct.deductionDelay)) + ct.deduct[future] = append(ct.deduct[future], func() { + ct.controller.TestingAdjustTokens(ctx, ct.pri, ct.delta, ct.stream) + }) + } + return + } + // Track waiting request. + ct.waitingRequests[t] = waitingRequestInController{ + ctx: ctx, + signaled: signaled, + admit: admit, + } +} + +// reset is part of the ticker interface. +func (ct *controllerTicker) reset() { + ct.ticks = 0 + ct.deduct = make(map[time.Time][]func()) + ct.waitingRequests = make(map[time.Time]waitingRequestInController) +} + +// end is part of the ticker interface. +func (ct *controllerTicker) end() time.Time { + return ct.tend +} + +// waitingRequestInController represents a request waiting for admission (due to +// unavailable flow tokens) when interacting directly with the +// kvflowcontrol.Controller. +type waitingRequestInController struct { + ctx context.Context + signaled func() bool // whether the request has been signaled + admit func() bool // invoked once signaled, returns whether the request has been admitted +} + +// handleTicker is a concrete implementation of the ticker interface, used +// to simulate tokens adjustments using a kvflowcontrol.Handle. +type handleTicker struct { + t *testing.T + tstart, tend time.Time + pri admissionpb.WorkPriority + delta kvflowcontrol.Tokens + replicaHandle *replicaHandle + stream kvflowcontrol.Stream + mod, ticks int // used to control the ticks at which we interact with the handle + deductionDelay int + + deduct map[time.Time][]func() + waitingRequests map[time.Time]waitingRequestInHandle +} + +var _ ticker = &handleTicker{} + +// tick is part of the ticker interface. +func (ht *handleTicker) tick(ctx context.Context, t time.Time) { + if ds, ok := ht.deduct[t]; ok { + for _, deduct := range ds { + deduct() + } + delete(ht.deduct, t) + } + for key, waitingRequest := range ht.waitingRequests { + // Process all waiting requests from earlier. Do this even if t > + // ht.tend since these requests could've been generated earlier. + if waitingRequest.ctx.Err() != nil { + delete(ht.waitingRequests, key) + continue + } + for i := range waitingRequest.signaled { + if !waitingRequest.signaled[i]() { + continue + } + if !waitingRequest.admit[i]() { + continue + } + + // Specific stream is unblocked (either because tokens were + // available, or it disconnected). Stop tracking it. + waitingRequest.remove(i) + break + + // TODO(irfansharif): Are we introducing non-determinism in this + // test by potentially allowing multiple (i) streams of a single + // request, and (ii) requests getting admitted depending on + // (non-deterministic) channel delivery? + } + + if len(waitingRequest.signaled) == 0 { + // All underlying streams have been unblocked; proceed with token + // deductions. + if ht.deductionDelay == 0 { + ht.replicaHandle.deductTokens(ctx, ht.pri, -ht.delta) + } else { + future := t.Add(tick * time.Duration(ht.deductionDelay)) + ht.deduct[future] = append(ht.deduct[future], func() { + ht.replicaHandle.deductTokens(ctx, ht.pri, -ht.delta) + }) + } + delete(ht.waitingRequests, key) + } + } + + if t.Before(ht.tstart) || (t.After(ht.tend) || t.Equal(ht.tend)) { + return // we're outside our [ht.tstart, ht.tend), there's nothing left to do + } + + defer func() { ht.ticks += 1 }() + if ht.ticks%ht.mod != 0 { + return // nothing to do in this tick + } + + if ht.delta >= 0 { // return tokens + ht.replicaHandle.returnTokens(ctx, ht.pri, ht.delta, ht.stream) + return + } + + admitted, signaled, admit := ht.replicaHandle.handle.TestingNonBlockingAdmit(ctx, ht.pri) + if admitted { + // Request admitted; proceed with token deductions. + if ht.deductionDelay == 0 { + ht.replicaHandle.deductTokens(ctx, ht.pri, -ht.delta) + } else { + future := t.Add(tick * time.Duration(ht.deductionDelay)) + ht.deduct[future] = append(ht.deduct[future], func() { + ht.replicaHandle.deductTokens(ctx, ht.pri, -ht.delta) + }) + } + return + } + + // Track waiting request. + ht.waitingRequests[t] = waitingRequestInHandle{ + ctx: ctx, + signaled: signaled, + admit: admit, + } +} + +// reset is part of the ticker interface. +func (ht *handleTicker) reset() { + ht.ticks = 0 + ht.deduct = make(map[time.Time][]func()) + ht.waitingRequests = make(map[time.Time]waitingRequestInHandle) +} + +// end is part of the ticker interface. +func (ht *handleTicker) end() time.Time { + return ht.tend +} + +// waitingRequestInHandle represents a request waiting for admission (due to +// unavailable flow tokens) when interacting directly with the +// kvflowcontrol.Handle. +type waitingRequestInHandle struct { + ctx context.Context + signaled []func() bool // whether the request has been signaled (for each underlying stream) + admit []func() bool // invoked once signaled, returns whether the request has been admitted (for each underlying stream) +} + +func (w *waitingRequestInHandle) remove(i int) { + w.signaled = append(w.signaled[:i], w.signaled[i+1:]...) + w.admit = append(w.admit[:i], w.admit[i+1:]...) +} + +// handleOpTicker is a concrete implementation of the ticker interface, used +// to simulate timed operations on a kvflowcontrol.Handle (taking snapshots, +// connecting/disconnecting streams, closing it entirely). +type handleOpTicker struct { + t *testing.T + tstart time.Time + replicaHandle *replicaHandle + position kvflowcontrolpb.RaftLogPosition + stream kvflowcontrol.Stream + op string + + done bool +} + +var _ ticker = &handleOpTicker{} + +// tick is part of the ticker interface. +func (ht *handleOpTicker) tick(ctx context.Context, t time.Time) { + if ht.done || t.Before(ht.tstart) { + return // nothing to do + } + switch ht.op { + case "close": + ht.replicaHandle.handle.Close(ctx) + ht.replicaHandle.closed = true + case "disconnect": + ht.replicaHandle.handle.DisconnectStream(ctx, ht.stream) + case "connect": + ht.replicaHandle.quorumLogPosition = ht.position + ht.replicaHandle.handle.ConnectStream(ctx, ht.position, ht.stream) + ht.replicaHandle.deductionTracker[ht.stream] = kvflowtokentracker.New(ht.position, nil /* knobs */) + case "snapshot": + ht.replicaHandle.snapshots = append(ht.replicaHandle.snapshots, snapshot{ + time: t, + stream: ht.stream, + data: ht.replicaHandle.deductionTracker[ht.stream].TestingPrintIter(), + }) + } + ht.done = true +} + +// reset is part of the ticker interface. +func (ht *handleOpTicker) reset() {} + +// end is part of the ticker interface. +func (ht *handleOpTicker) end() time.Time { + return ht.tstart.Add(time.Second) +} + +type unbreakableStream struct { + stream kvflowcontrol.Stream +} + +var _ kvflowcontrol.ConnectedStream = &unbreakableStream{} + +// Stream is part of the ConnectedStream interface. +func (u *unbreakableStream) Stream() kvflowcontrol.Stream { + return u.stream +} + +// Disconnected is part of the ConnectedStream interface. +func (u *unbreakableStream) Disconnected() <-chan struct{} { + return nil +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/admitting_waiting_choppy b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_admitting_waiting_choppy similarity index 90% rename from pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/admitting_waiting_choppy rename to pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_admitting_waiting_choppy index 371f31f87dc3..fdccca3135ad 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/admitting_waiting_choppy +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_admitting_waiting_choppy @@ -8,22 +8,22 @@ init # Set up an open-loop thread issuing 2MiB/s of regular writes from t=0s to # t=25s. timeline -start=0s end=25s class=regular stream=t1/s1 adjust=-2MiB/s rate=10/s +t=[0s,25s) class=regular stream=t1/s1 adjust=-2MiB/s rate=10/s ---- # Set up choppy flow token returns starting at t=15s. The average rate of # returns is lower than 2MiB/s, so we should always have some waiting work. timeline -start=15s end=16s class=regular stream=t1/s1 adjust=+0.9MiB/s rate=10/s -start=16s end=17s class=regular stream=t1/s1 adjust=+2.1MiB/s rate=10/s -start=17s end=18s class=regular stream=t1/s1 adjust=+0.1MiB/s rate=10/s -start=18s end=19s class=regular stream=t1/s1 adjust=+0.9MiB/s rate=10/s -start=19s end=20s class=regular stream=t1/s1 adjust=+2.1MiB/s rate=10/s -start=20s end=21s class=regular stream=t1/s1 adjust=+0.1MiB/s rate=10/s -start=21s end=22s class=regular stream=t1/s1 adjust=+0.9MiB/s rate=10/s -start=22s end=23s class=regular stream=t1/s1 adjust=+2.1MiB/s rate=10/s -start=23s end=24s class=regular stream=t1/s1 adjust=+0.1MiB/s rate=10/s -start=24s end=25s class=regular stream=t1/s1 adjust=+0.9MiB/s rate=10/s +t=[15s,16s) class=regular stream=t1/s1 adjust=+0.9MiB/s rate=10/s +t=[16s,17s) class=regular stream=t1/s1 adjust=+2.1MiB/s rate=10/s +t=[17s,18s) class=regular stream=t1/s1 adjust=+0.1MiB/s rate=10/s +t=[18s,19s) class=regular stream=t1/s1 adjust=+0.9MiB/s rate=10/s +t=[19s,20s) class=regular stream=t1/s1 adjust=+2.1MiB/s rate=10/s +t=[20s,21s) class=regular stream=t1/s1 adjust=+0.1MiB/s rate=10/s +t=[21s,22s) class=regular stream=t1/s1 adjust=+0.9MiB/s rate=10/s +t=[22s,23s) class=regular stream=t1/s1 adjust=+2.1MiB/s rate=10/s +t=[23s,24s) class=regular stream=t1/s1 adjust=+0.1MiB/s rate=10/s +t=[24s,25s) class=regular stream=t1/s1 adjust=+0.9MiB/s rate=10/s ---- simulate @@ -102,7 +102,7 @@ kvadmission.flow_controller.regular_tokens_available unit=MiB # are being returned choppily. Given the average rate of returns is lower # than what's being requested (2MiB/s), the total flow tokens available bounces # off of zero. -plot start=15s end=30s +plot t=[15s,30s) kvadmission.flow_controller.regular_tokens_available unit=MiB ---- 0.2 ┤ ╭╮ @@ -126,7 +126,7 @@ kvadmission.flow_controller.regular_tokens_available unit=MiB # Note again the mirroring between token returns which immediately allows # admission, followed by token deductions. -plot start=15s end=30s +plot t=[15s,30s) kvadmission.flow_controller.regular_tokens_{deducted,returned} unit=MiB rate=true ---- 2.1 ┤ ╭╮ @@ -151,7 +151,7 @@ kvadmission.flow_controller.regular_tokens_{deducted,returned} unit=MiB rate=tr # So we're still admitting work choppily, and observe corresponding deductions # in the waiting request rate. But given the open-loop thread above, the # of # waiting request is still growing unboundedly. -plot start=15s end=30s +plot t=[15s,30s) kvadmission.flow_controller.regular_requests_{admitted,waiting} unit=reqs/s rate=true kvadmission.flow_controller.regular_requests_waiting unit=reqs ---- @@ -195,4 +195,4 @@ kvadmission.flow_controller.regular_requests_waiting unit=reqs ---- ---- -# vim:ft=sh +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/over_admission b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_over_admission similarity index 92% rename from pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/over_admission rename to pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_over_admission index 39b74c681351..f63cd750fb51 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/over_admission +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_over_admission @@ -6,22 +6,22 @@ init # Take away all 16MiB of regular flow tokens; we want a buildup of waiting # requests to over-admit from. timeline -start=0s end=1s class=regular stream=t1/s1 adjust=-16MiB/s rate=1/s +t=[0s,1s) class=regular stream=t1/s1 adjust=-16MiB/s rate=1/s ---- # Queue 10/s*10s=100 requests for admission, asking for 10*4MiB=40MiB of # tokens. For these requests, induce a 2s delay between Admit() and # DeductTokens(). timeline -start=10s end=20s class=regular stream=t1/s1 adjust=-4MiB/s rate=10/s deduction-delay=2s +t=[10s,20s) class=regular stream=t1/s1 adjust=-4MiB/s rate=10/s deduction-delay=2s ---- # Return 1KiB of flow tokens at t=30. timeline -start=30s end=31s class=regular stream=t1/s1 adjust=+1KiB/s rate=1/s +t=[30s,31s) class=regular stream=t1/s1 adjust=+1KiB/s rate=1/s ---- -simulate end=40s +simulate t=[0s,40s) ---- # Observe how the single 1KiB flow token return ends up admitting all 100 @@ -30,7 +30,7 @@ simulate end=40s # TODO(irfansharif): Introduce a "tentative deducted counter" on a per-stream # basis, to prevent this kind of over-admission. It's likely to occur any time # there's AC queueing due to CPU control, waiting on locks/latches, etc. -plot start=0s end=40s +plot t=[0s,40s) kvadmission.flow_controller.regular_tokens_available unit=MiB kvadmission.flow_controller.regular_requests_admitted unit=reqs/s rate=true kvadmission.flow_controller.regular_requests_waiting unit=reqs @@ -94,4 +94,4 @@ kvadmission.flow_controller.regular_requests_waiting unit=reqs ---- ---- -# vim:ft=sh +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/overview b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_overview similarity index 94% rename from pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/overview rename to pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_overview index a70ee8cd31d4..6bd8e95b15dd 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/overview +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_overview @@ -1,4 +1,4 @@ -# Walk through the basics of the datadriven syntax. +# Walk through the basics of the datadriven syntax for flow controller tests. init ---- @@ -7,17 +7,17 @@ init # it to get depleted in 8s. The remaining 8s of incoming work ends up getting # queued. timeline -start=0s end=16s class=regular stream=t1/s1 adjust=-2MiB/s rate=100/s -start=0s end=16s class=regular stream=t1/s2 adjust=-2MiB/s rate=100/s +t=[0s,16s) class=regular stream=t1/s1 adjust=-2MiB/s rate=100/s +t=[0s,16s) class=regular stream=t1/s2 adjust=-2MiB/s rate=100/s ---- # Schedule the return of flow tokens at t=20s in similar increments. timeline -start=20s end=36s class=regular stream=t1/s1 adjust=+2MiB/s rate=100/s -start=20s end=36s class=regular stream=t1/s2 adjust=+2MiB/s rate=100/s +t=[20s,36s) class=regular stream=t1/s1 adjust=+2MiB/s rate=100/s +t=[20s,36s) class=regular stream=t1/s2 adjust=+2MiB/s rate=100/s ---- -simulate end=40s +simulate t=[0s,40s) ---- # There two replication streams, so we initially have 16*2=32MiB of regular @@ -27,7 +27,7 @@ simulate end=40s # prioritization -- those tokens get depleted at t=4s. Both are deducted at # 4MiB/s (2MiB/s for each stream) until we have 0MiB regular tokens and -16MiB # elastic tokens. -plot start=0s end=20s +plot t=[0s,20s) kvadmission.flow_controller.{regular,elastic}_tokens_available unit=MiB kvadmission.flow_controller.{regular,elastic}_tokens_deducted unit=MiB/s rate=true ---- @@ -76,7 +76,7 @@ kvadmission.flow_controller.{regular,elastic}_tokens_deducted unit=MiB/s rate=t # 100 requests/s for two streams, we observe an aggregate admission rate of # ~200/s, and then later 200/s of waiting requests growth. There are no # errors. -plot start=0s end=20s +plot t=[0s,20s) kvadmission.flow_controller.regular_requests_{admitted,waiting} unit=reqs/s rate=true kvadmission.flow_controller.{regular,elastic}_requests_errored unit=reqs/s rate=true ---- @@ -107,7 +107,7 @@ kvadmission.flow_controller.{regular,elastic}_requests_errored unit=reqs/s rate # Confirm that there are two streams underneath, both of which eventually get # blocked for {regular,elastic} traffic, with the latter happening first. -plot start=0s end=20s +plot t=[0s,20s) kvadmission.flow_controller.regular_stream_count unit=streams kvadmission.flow_controller.{regular,elastic}_blocked_stream_count unit=streams ---- @@ -140,7 +140,7 @@ kvadmission.flow_controller.{regular,elastic}_blocked_stream_count unit=streams # at ~200/s, which matches the rate at which we're reducing the number of # waiting requests. By t=36s we'll have returned all {regular,elastic} flow # tokens, including for the requests that had to wait for admission. -plot start=18s end=40s +plot t=[18s,40s) kvadmission.flow_controller.regular_requests_{admitted,waiting} unit=reqs/s rate=true kvadmission.flow_controller.{regular,elastic}_tokens_available unit=MiB ---- @@ -185,4 +185,4 @@ kvadmission.flow_controller.{regular,elastic}_tokens_available unit=MiB ---- ---- -# vim:ft=sh +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/regular_elastic_prioritization b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_regular_elastic_prioritization similarity index 93% rename from pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/regular_elastic_prioritization rename to pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_regular_elastic_prioritization index 43df08beebc0..a7392943e76f 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/testdata/simulation/regular_elastic_prioritization +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/flow_controller_regular_elastic_prioritization @@ -6,13 +6,10 @@ init # Kick off two threads, one for each work class, each writing at 1MiB/s over # across reqs/s. timeline -start=0s end=16s class=regular stream=t1/s1 adjust=-1MiB/s rate=100/s -start=0s end=8s class=elastic stream=t1/s1 adjust=-1MiB/s rate=100/s +t=[0s,16s) class=regular stream=t1/s1 adjust=-1MiB/s rate=100/s +t=[0s,8s) class=elastic stream=t1/s1 adjust=-1MiB/s rate=100/s ---- -# start=16s end=32s class=regular stream=t1/s1 adjust=+2MiB/s rate=100/s -# start=32s end=36s class=elastic stream=t1/s1 adjust=+2MiB/s rate=100/s - simulate ---- @@ -119,4 +116,4 @@ kvadmission.flow_controller.elastic_tokens_deducted unit=MiB rate=true 0.0 ┼──╯ rate(elastic_tokens_deducted) (MiB) -# vim:ft=sh +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_overview b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_overview new file mode 100644 index 000000000000..5f752145858a --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_overview @@ -0,0 +1,275 @@ +# Walk through the basics of the datadriven syntax for handle tests. + +# Initialize testing components, including a named handle h. +init +handle=h +---- + +# Connect the handle to three replication streams (like the leader+leaseholder +# for a triply replicated range would). +# - Start writing to the range at 2MiB/s across 10 reqs/s. +# - Grab s1 snapshots at t=2 and t=3, to compare how things evolve. +# - Partway through, at t=4s, disconnect s2. +# - From t=[10s,18s), start returning tokens from s1 and s3. +# - Grab s1 snapshots at t=16s, t=17s and t=19s to compare how things evolve. +# - Start writing to the range again at t=[20s,21s) at 2MiB/s across 10 reqs/s. +# - At t=22s, close the handle. +timeline +t=0s handle=h op=connect stream=t1/s1 log-position=4/0 +t=0s handle=h op=connect stream=t1/s2 log-position=4/0 +t=0s handle=h op=connect stream=t1/s3 log-position=4/0 +t=2s handle=h op=snapshot stream=t1/s1 +t=3s handle=h op=snapshot stream=t1/s1 +t=4s handle=h op=disconnect stream=t1/s2 +t=[0s,8s) handle=h class=regular adjust=-2MiB/s rate=10/s +t=[10s,18s) handle=h class=regular adjust=+2MiB/s rate=10/s stream=t1/s1 +t=[10s,18s) handle=h class=regular adjust=+2MiB/s rate=10/s stream=t1/s3 +t=16s handle=h op=snapshot stream=t1/s1 +t=17s handle=h op=snapshot stream=t1/s1 +t=19s handle=h op=snapshot stream=t1/s1 +t=[20s,21s) handle=h class=regular adjust=-2MiB/s rate=10/s +t=21s handle=h op=snapshot stream=t1/s1 +t=22s handle=h op=close +---- + + +simulate t=[0s,24s) +---- + +# We should observe a few things. +# - We've lazily instantiated three streams for regular traffic. Even when the +# handle disconnects the stream from itself, it's still tracked by the +# controller. +# - Given there are three streams, we start off with 3*16MiB = 48MiB of +# regular tokens. +# - When three streams are connected, quorum writes at 2MiB/s translates to +# token deductions at 3*2MiB/s = 6MiB/s. +# - When s2 is disconnected from h: +# - All s2-specific flow tokens deducted by h are returned. By t=4s, this +# translates to 4s * 2MiB/s = 8MiB. We see this in both +# regular_tokens_available and the 8MiB/s spike in +# rate(regular_tokens_returned). +# - The rate of token deductions decreases to 2*2MiB/s = 4MiB/s. +# - By the time quorum writes are blocked, the available regular tokens is +# 16MiB, corresponding to s2's fully available regular tokens. +plot t=[0s,10s) +kvadmission.flow_controller.regular_stream_count unit=streams +kvadmission.flow_controller.regular_tokens_available unit=MiB +kvadmission.flow_controller.regular_tokens_{deducted,returned} unit=MiB/s rate=true +---- +---- + 3.0 ┼─────────────────────────────────────── + regular_stream_count (streams) + + + 47.4 ┼╮ + 45.3 ┤╰╮ + 43.2 ┤ ╰╮ + 41.1 ┤ ╰─╮ + 39.0 ┤ ╰╮ + 36.9 ┤ ╰─╮ + 34.8 ┤ ╰╮ + 32.7 ┤ ╰╮ + 30.7 ┤ ╰─╮ ╭╮ + 28.6 ┤ ╰╮ │╰──╮ + 26.5 ┤ ╰╮ │ ╰─╮ + 24.4 ┤ ╰─╯ ╰─╮ + 22.3 ┤ ╰─╮ + 20.2 ┤ ╰─╮ + 18.1 ┤ ╰─╮ + 16.0 ┤ ╰───────── + regular_tokens_available (MiB) + + + 8.0 ┤ ╭───╮ + 7.5 ┤ │ │ + 6.9 ┤ │ │ + 6.4 ┤ │ │ + 5.9 ┤ ╭───────────│╮ │ + 5.3 ┤ │ │╰╮ │ + 4.8 ┤ │ │ ╰╮│ + 4.3 ┤ │ │ ╰│───────────╮ + 3.7 ┤ │ │ │ │ + 3.2 ┤ │ │ │ ╰╮ + 2.7 ┤ │ │ │ │ + 2.1 ┤ │ │ │ ╰╮ + 1.6 ┤ │ │ │ │ + 1.1 ┤ │ │ │ ╰╮ + 0.5 ┤ │ │ │ │ + 0.0 ┼───────────────╯ ╰─────────────────── + rate(regular_tokens_{deducted,returned}) (MiB/s) +---- +---- + + +# This period corresponds to: +# t=[20s,21s) handle=h class=regular adjust=-2MiB/s rate=10/s +# t=22s handle=h op=close +# Where the handle is connected to t1/s1 and t1/s3. Note how when the handle is +# closed, all 2MiB*2 = 4MiB tokens for s1 and s3 are returned. +plot t=[19s,24s) +kvadmission.flow_controller.regular_tokens_available unit=MiB +kvadmission.flow_controller.regular_tokens_{deducted,returned} unit=MiB/s rate=true +---- +---- + 48.0 ┼───────╮ ╭─────────────── + 47.7 ┤ │ │ + 47.5 ┤ ╰╮ │ + 47.2 ┤ ╰╮ │ + 46.9 ┤ │ │ + 46.7 ┤ ╰╮ │ + 46.4 ┤ │ │ + 46.1 ┤ ╰╮ │ + 45.9 ┤ │ │ + 45.6 ┤ ╰╮ │ + 45.3 ┤ │ │ + 45.1 ┤ ╰╮ │ + 44.8 ┤ │ │ + 44.5 ┤ ╰╮ │ + 44.3 ┤ │ │ + 44.0 ┤ ╰────────╯ + regular_tokens_available (MiB) + + + 4.0 ┤ ╭╮ ╭───────╮ + 3.7 ┤ ││ │ │ + 3.5 ┤ ╭╯╰╮ │ │ + 3.2 ┤ │ │ │ │ + 2.9 ┤ ╭╯ ╰╮ │ │ + 2.7 ┤ │ ╰╮ │ │ + 2.4 ┤ ╭╯ │ │ │ + 2.1 ┤ │ ╰╮ │ │ + 1.9 ┤ ╭╯ │ │ │ + 1.6 ┤ │ ╰╮ │ │ + 1.3 ┤ ╭╯ │ │ │ + 1.1 ┤ │ ╰╮ │ │ + 0.8 ┤ ╭╯ │ │ │ + 0.5 ┤ ╭╯ ╰╮│ │ + 0.3 ┤ │ ││ │ + 0.0 ┼───────────────────────╯───────╰─────── + rate(regular_tokens_{deducted,returned}) (MiB/s) +---- +---- + +# Observe captured snapshots. +# - At 10 reqs/s, by t=2s we have 10*2=20 tracked tokens from log positions +# 4/1 to 4/20. We add 10 more by t=3s, going up to 4/30. +# - At t=16s the we're tracking tokens from log positions 4/62 to 4/80. We +# return tokens in 10 increments of 205KiB over the next second, so at t=17s +# we've reduced the remaining tracked tokens by 10, now starting at 4/62. +# - At t=19s we have no outstanding tokens being tracked -- we've returned +# everything. +# - At t=21s we've tracked a few more tokens; tokens we haven't returned yet. +# +# TODO(irfansharif): Support filtering this output by stream and time range. +snapshots handle=h +---- +---- +t=2s stream=t1/s1 + pri=normal-pri + tokens=205KiB log-position=4/1 + tokens=205KiB log-position=4/2 + tokens=205KiB log-position=4/3 + tokens=205KiB log-position=4/4 + tokens=205KiB log-position=4/5 + tokens=205KiB log-position=4/6 + tokens=205KiB log-position=4/7 + tokens=205KiB log-position=4/8 + tokens=205KiB log-position=4/9 + tokens=205KiB log-position=4/10 + tokens=205KiB log-position=4/11 + tokens=205KiB log-position=4/12 + tokens=205KiB log-position=4/13 + tokens=205KiB log-position=4/14 + tokens=205KiB log-position=4/15 + tokens=205KiB log-position=4/16 + tokens=205KiB log-position=4/17 + tokens=205KiB log-position=4/18 + tokens=205KiB log-position=4/19 + tokens=205KiB log-position=4/20 + +t=3s stream=t1/s1 + pri=normal-pri + tokens=205KiB log-position=4/1 + tokens=205KiB log-position=4/2 + tokens=205KiB log-position=4/3 + tokens=205KiB log-position=4/4 + tokens=205KiB log-position=4/5 + tokens=205KiB log-position=4/6 + tokens=205KiB log-position=4/7 + tokens=205KiB log-position=4/8 + tokens=205KiB log-position=4/9 + tokens=205KiB log-position=4/10 + tokens=205KiB log-position=4/11 + tokens=205KiB log-position=4/12 + tokens=205KiB log-position=4/13 + tokens=205KiB log-position=4/14 + tokens=205KiB log-position=4/15 + tokens=205KiB log-position=4/16 + tokens=205KiB log-position=4/17 + tokens=205KiB log-position=4/18 + tokens=205KiB log-position=4/19 + tokens=205KiB log-position=4/20 + tokens=205KiB log-position=4/21 + tokens=205KiB log-position=4/22 + tokens=205KiB log-position=4/23 + tokens=205KiB log-position=4/24 + tokens=205KiB log-position=4/25 + tokens=205KiB log-position=4/26 + tokens=205KiB log-position=4/27 + tokens=205KiB log-position=4/28 + tokens=205KiB log-position=4/29 + tokens=205KiB log-position=4/30 + +t=16s stream=t1/s1 + pri=normal-pri + tokens=205KiB log-position=4/62 + tokens=205KiB log-position=4/63 + tokens=205KiB log-position=4/64 + tokens=205KiB log-position=4/65 + tokens=205KiB log-position=4/66 + tokens=205KiB log-position=4/67 + tokens=205KiB log-position=4/68 + tokens=205KiB log-position=4/69 + tokens=205KiB log-position=4/70 + tokens=205KiB log-position=4/71 + tokens=205KiB log-position=4/72 + tokens=205KiB log-position=4/73 + tokens=205KiB log-position=4/74 + tokens=205KiB log-position=4/75 + tokens=205KiB log-position=4/76 + tokens=205KiB log-position=4/77 + tokens=205KiB log-position=4/78 + tokens=205KiB log-position=4/79 + tokens=205KiB log-position=4/80 + +t=17s stream=t1/s1 + pri=normal-pri + tokens=205KiB log-position=4/72 + tokens=205KiB log-position=4/73 + tokens=205KiB log-position=4/74 + tokens=205KiB log-position=4/75 + tokens=205KiB log-position=4/76 + tokens=205KiB log-position=4/77 + tokens=205KiB log-position=4/78 + tokens=205KiB log-position=4/79 + tokens=205KiB log-position=4/80 + +t=19s stream=t1/s1 + +t=21s stream=t1/s1 + pri=normal-pri + tokens=205KiB log-position=4/81 + tokens=205KiB log-position=4/82 + tokens=205KiB log-position=4/83 + tokens=205KiB log-position=4/84 + tokens=205KiB log-position=4/85 + tokens=205KiB log-position=4/86 + tokens=205KiB log-position=4/87 + tokens=205KiB log-position=4/88 + tokens=205KiB log-position=4/89 + tokens=205KiB log-position=4/90 +---- +---- + +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_shared_stream b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_shared_stream new file mode 100644 index 000000000000..159f8e6551aa --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_shared_stream @@ -0,0 +1,110 @@ +# Demonstrate the behavior when a single stream is share across two handles h1 +# and h2. +init +handle=h1 +handle=h2 +---- + +# Set up two handles that connect only to s1, and issue writes at 2MiB/s each. +# Set up token returns for the stream at 1MiB/s. +timeline +t=0s handle=h1 op=connect stream=t1/s1 log-position=4/0 +t=0s handle=h2 op=connect stream=t1/s1 log-position=4/0 +t=[0s,10s) handle=h1 class=regular adjust=-2MiB/s rate=10/s +t=[0s,10s) handle=h2 class=regular adjust=-2MiB/s rate=10/s +t=[0s,10s) stream=t1/s1 class=regular adjust=+1MiB/s rate=200/s +---- + +simulate t=[0,10s) +---- + +# We expect the tokens to get depleted and then eventually observe that the +# aggregate bandwidth of token deductions across the two handles match the +# token return rate. The admission rate goes from 2*10 = 20reqs/s (unthrottled) +# to 5 reqs/s, which is 1/4th of 20. The 1/4th comes from 1MiB/s return rate +# being 1/4th of -4MiB/s demand. +plot +kvadmission.flow_controller.regular_{,blocked_}stream_count unit=streams +kvadmission.flow_controller.regular_tokens_available unit=MiB +kvadmission.flow_controller.regular_tokens_{deducted,returned} unit=MiB/s rate=true +kvadmission.flow_controller.regular_requests_{admitted,waiting} unit=reqs/s rate=true +---- +---- + 1.0 ┼──────────────────────────────────────╭ + 0.9 ┤ ╭╮ ╭╮ │ + 0.9 ┤ ││ ││ ╭╮ │ + 0.8 ┤ ││ ││ ╭╮ ││ │ + 0.7 ┤ ││ ││ ││ ││ │ + 0.7 ┤ ╭╮ ││ ││ ╭╯│ ││ │ + 0.6 ┤ ││ ││ ││ │ │ │╰╮ │ + 0.5 ┤ ││ │╰╮╭╯│ │ │ │ │ │ + 0.5 ┤ ││ │ ││ │ │ │ │ │╭╯ + 0.4 ┤ ││╭╯ ││ ╰╮│ │ │ ││ + 0.3 ┤ │││ ││ ││ │╭╯ ││ + 0.3 ┤ │││ ││ ││ ││ ││ + 0.2 ┤ │││ ││ ││ ╰╯ ││ + 0.1 ┤ │╰╯ ││ ╰╯ ││ + 0.1 ┤ │ ││ ╰╯ + 0.0 ┼────────────────────╯ ╰╯ + regular_{,blocked_}stream_count (streams) + + + 15.6 ┼╮ + 14.6 ┤╰╮ + 13.5 ┤ ╰─╮ + 12.5 ┤ ╰╮ + 11.4 ┤ ╰─╮ + 10.4 ┤ ╰╮ + 9.3 ┤ ╰╮ + 8.3 ┤ ╰─╮ + 7.2 ┤ ╰╮ + 6.2 ┤ ╰╮ + 5.1 ┤ ╰─╮ + 4.1 ┤ ╰╮ + 3.0 ┤ ╰─╮ + 2.0 ┤ ╰╮ + 1.0 ┤ ╰╮ + -0.1 ┤ ╰─────────────────── + regular_tokens_available (MiB) + + + 4.0 ┤ ╭────────────────╮ + 3.7 ┤ │ ╰╮ + 3.5 ┤ │ │ + 3.2 ┤ │ │ + 2.9 ┤ │ ╰╮ + 2.7 ┤ │ │ + 2.4 ┤ │ │ + 2.1 ┤ │ ╰╮ + 1.9 ┤ │ │ + 1.6 ┤ │ │ + 1.3 ┤ │ ╰╮ + 1.1 ┤ ╭─────────────────────────────────── + 0.8 ┤ │ + 0.5 ┤ │ + 0.3 ┤ │ + 0.0 ┼───╯ + rate(regular_tokens_{deducted,returned}) (MiB/s) + + + 20.0 ┤ ╭────────────────╮ + 18.7 ┤ │ ╰╮ + 17.3 ┤ │ │ + 16.0 ┤ │ │ + 14.7 ┤ │ ╰╮ ╭────────────── + 13.3 ┤ │ │╭╯ + 12.0 ┤ │ ││ + 10.7 ┤ │ ╰│ + 9.3 ┤ │ ╭╯ + 8.0 ┤ │ ││ + 6.7 ┤ │ │╰╮ + 5.3 ┤ │ ╭╯ ╰────────────── + 4.0 ┤ │ │ + 2.7 ┤ │ │ + 1.3 ┤ │ ╭╯ + 0.0 ┼────────────────────╯ + rate(regular_requests_{admitted,waiting}) (reqs/s) +---- +---- + +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_single_slow_stream b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_single_slow_stream new file mode 100644 index 000000000000..612c13f56b4e --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_single_slow_stream @@ -0,0 +1,125 @@ +# Demonstrate that a single slow stream (with lower admission/token return +# rate) will end up pacing quorum writes (through the handle) to just the +# slowest rate. +init +handle=h +---- + +# Set up a triply connected handle (to s1, s2, s3) and start issuing writes at +# 1MiB/s. For two of the streams, return tokens at exactly the rate its being +# deducted (1MiB/s). For the third stream (s3), we return flow tokens at only +# 0.5MiB/s. +timeline +t=0s handle=h op=connect stream=t1/s1 log-position=1/0 +t=0s handle=h op=connect stream=t1/s2 log-position=1/0 +t=0s handle=h op=connect stream=t1/s3 log-position=1/0 +t=[0s,50s) handle=h class=regular adjust=-1MiB/s rate=10/s +t=[0.2s,50s) handle=h class=regular adjust=+1MiB/s rate=10/s stream=t1/s1 +t=[0.2s,50s) handle=h class=regular adjust=+1MiB/s rate=10/s stream=t1/s2 +t=[0.2s,50s) handle=h class=regular adjust=+0.5MiB/s rate=10/s stream=t1/s3 +---- + + +simulate +---- + +# Observe: +# - Total available tokens flatlines at 32MiB since flow tokens for s3 +# eventually depletes and later bounces off of 0MiB. We initially have +# 3*16MiB = 48MiB worth of flow tokens, and end up at 48MiB-16MiB = 32MiB. +# - Initially the rate of token deductions (3*1MiB/s = 3MiB/s) is higher than +# the token returns (1MiB/s+1MiB/s+0.5MiB/s = 2.5MiB/s), but after we start +# shaping it to the slowest stream, they end up matching at (0.5MiB/s*3 = +# 1.5MiB/s). +# - The blocked stream count bounces between 0 and 1 as the s3 stream gets +# blocked/unblocked as tokens are deducted/returned. The demand for tokens +# (1MiB/s) is higher than the replenishment rate (0.5MiB/s). +# - The overall admission rate is reduced from 30 reqs/s to 25 reqs/s, +# mapping to token deduction rate of 3MiB/s to 2.5MiB/s (1MiB/s divvied +# across 10 reqs). The difference between 30 reqs/s and 25 reqs/s is found +# in the +5 reqs/s accumulating in the wait queue. +plot +kvadmission.flow_controller.regular_tokens_available unit=MiB +kvadmission.flow_controller.regular_tokens_{deducted,returned} unit=MiB/s rate=true +kvadmission.flow_controller.regular_blocked_stream_count unit=streams +kvadmission.flow_controller.regular_requests_{admitted,waiting} unit=reqs/s rate=true +---- +---- + 47.7 ┼╮ + 46.6 ┤╰─╮ + 45.6 ┤ ╰─╮ + 44.5 ┤ ╰╮ + 43.5 ┤ ╰─╮ + 42.4 ┤ ╰╮ + 41.4 ┤ ╰─╮ + 40.3 ┤ ╰─╮ + 39.3 ┤ ╰╮ + 38.2 ┤ ╰─╮ + 37.2 ┤ ╰─╮ + 36.1 ┤ ╰╮ + 35.1 ┤ ╰─╮ + 34.0 ┤ ╰─╮ + 33.0 ┤ ╰╮ + 31.9 ┤ ╰─────────────── + regular_tokens_available (MiB) + + + 3.0 ┤╭───────────────────────╮ + 2.8 ┤│ │ + 2.6 ┤╭────────────────────────╮ + 2.4 ┤│ ╰│ + 2.2 ┤│ │ + 2.0 ┤│ │ + 1.8 ┤│ │ + 1.6 ┤│ ╰───────────── + 1.4 ┤│ + 1.2 ┤│ + 1.0 ┤│ + 0.8 ┤│ + 0.6 ┤│ + 0.4 ┤│ + 0.2 ┤│ + 0.0 ┼╯ + rate(regular_tokens_{deducted,returned}) (MiB/s) + + + 1.0 ┤ ╭╮ ╭ + 0.9 ┤ ╭╮ ││ │ + 0.9 ┤ ││ ││ │ + 0.8 ┤ ││ ││ │ + 0.7 ┤ ││ ││ │ + 0.7 ┤ ╭╮ ││╭╮ ││ │ + 0.6 ┤ ││ ││││ ││╭─╮│ + 0.5 ┤ │╰╮│││╰╮│││ ││ + 0.5 ┤ │ ││││ ││││ ││ + 0.4 ┤ │ ││││ ││││ ││ + 0.3 ┤ │ ││││ ││││ ││ + 0.3 ┤ │ ╰╯││ ││││ ││ + 0.2 ┤ │ ││ ╰╯╰╯ ╰╯ + 0.1 ┤ ╭╯ ╰╯ + 0.1 ┤ │ + 0.0 ┼────────────────────────╯ + regular_blocked_stream_count (streams) + + + 30.0 ┤╭───────────────────────╮ + 28.0 ┤│ ╰╮ + 26.0 ┤│ ╰───────────── + 24.0 ┤│ + 22.0 ┤│ + 20.0 ┤│ + 18.0 ┤│ + 16.0 ┤│ + 14.0 ┤│ + 12.0 ┤│ + 10.0 ┤│ + 8.0 ┤│ + 6.0 ┤│ ╭───────────── + 4.0 ┤│ │ + 2.0 ┤│ ╭╯ + 0.0 ┼────────────────────────╯ + rate(regular_requests_{admitted,waiting}) (reqs/s) +---- +---- + +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_connection b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_connection new file mode 100644 index 000000000000..03a94ca59f0b --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_connection @@ -0,0 +1,92 @@ +# Demonstrate the effects of a stream being added part way through the process. +init +handle=h +---- + +# Set up a doubly connected handle (to s1, s2) that later connects to s3 at +# t=5s. We issue writes at 1MiB/s and return tokens for each stream at the rate +# of deduction, only 200ms later. +timeline +t=0s handle=h op=connect stream=t1/s1 log-position=4/0 +t=0s handle=h op=connect stream=t1/s2 log-position=4/0 +t=5s handle=h op=connect stream=t1/s3 log-position=4/50 +t=[0s,10s) handle=h class=regular adjust=-1MiB/s rate=10/s +t=[0.2s,10.2s) handle=h class=regular adjust=+1MiB/s rate=10/s stream=t1/s1 +t=[0.2s,10.2s) handle=h class=regular adjust=+1MiB/s rate=10/s stream=t1/s2 +t=[5.2s,10.2s) handle=h class=regular adjust=+1MiB/s rate=10/s stream=t1/s3 +---- + +simulate t=[0s,11s) +---- + +# We observe: +# - Increase in regular stream count once s3 is added. +# - Regular available tokens increasing by 16MiB, corresponding to the +# newly created stream +# - Token deductions increasing from 2MiB/s (for two connected connected +# streams) to 3MiB/s. +plot +kvadmission.flow_controller.regular_stream_count unit=streams +kvadmission.flow_controller.regular_tokens_available unit=MiB +kvadmission.flow_controller.regular_tokens_{deducted,returned} unit=MiB/s rate=true +---- +---- + 3.0 ┤ ╭───────────────────── + 2.9 ┤ │ + 2.9 ┤ │ + 2.8 ┤ │ + 2.7 ┤ │ + 2.7 ┤ │ + 2.6 ┤ │ + 2.5 ┤ │ + 2.5 ┤ │ + 2.4 ┤ │ + 2.3 ┤ │ + 2.3 ┤ │ + 2.2 ┤ │ + 2.1 ┤ │ + 2.1 ┤ │ + 2.0 ┼─────────────────╯ + regular_stream_count (streams) + + + 48.0 ┤ ╭─── + 46.9 ┤ ╭─────────────────╯ + 45.8 ┤ │ + 44.7 ┤ │ + 43.6 ┤ │ + 42.5 ┤ │ + 41.4 ┤ │ + 40.3 ┤ │ + 39.3 ┤ │ + 38.2 ┤ │ + 37.1 ┤ │ + 36.0 ┤ │ + 34.9 ┤ │ + 33.8 ┤ │ + 32.7 ┤ │ + 31.6 ┼─────────────────╯ + regular_tokens_available (MiB) + + + 3.0 ┤ ╭╭──────────────╮ + 2.8 ┤ ╭╯ ││ + 2.6 ┤ ╭│ ╰│ + 2.4 ┤ ╭╭╯ │ + 2.2 ┤ ╭╭╯ ╰╮ + 2.0 ┤ ╭──────────────╯ ││ + 1.8 ┤ │ ││ + 1.6 ┤ │ ╰│ + 1.4 ┤ │ ╰╮ + 1.2 ┤ │ ││ + 1.0 ┤ │ ││ + 0.8 ┤ │ ╰│ + 0.6 ┤ │ ╰ + 0.4 ┤ │ │ + 0.2 ┤ │ │ + 0.0 ┼───╯ ╰ + rate(regular_tokens_{deducted,returned}) (MiB/s) +---- +---- + +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_disconnection b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_disconnection new file mode 100644 index 000000000000..3214d504bed7 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_disconnection @@ -0,0 +1,100 @@ +# Demonstrate the effects of a slow stream (that's causing a build up of +# waiting requests) being disconnected. This test is similar to the one in +# handle_single_slow_stream, except the slow stream is eventually disconnected. + +init +handle=h +---- + +# Set up a triply connected handle (to s1, s2, s3) and start issuing writes at +# 1MiB/s. For two of the streams, return tokens at exactly the rate its being +# deducted (1MiB/s). For the third stream (s3), we return flow tokens at only +# 0.5MiB/s. At t=40s, disconnect the slow stream. +timeline +t=0s handle=h op=connect stream=t1/s1 log-position=4/0 +t=0s handle=h op=connect stream=t1/s2 log-position=4/0 +t=0s handle=h op=connect stream=t1/s3 log-position=4/0 +t=[0s,50s) handle=h class=regular adjust=-1MiB/s rate=10/s +t=[0.2s,50s) handle=h class=regular adjust=+1MiB/s rate=10/s stream=t1/s1 +t=[0.2s,50s) handle=h class=regular adjust=+1MiB/s rate=10/s stream=t1/s2 +t=[0.2s,40s) handle=h class=regular adjust=+0.5MiB/s rate=10/s stream=t1/s3 +t=40s handle=h op=disconnect stream=t1/s3 +---- + +simulate +---- + +# Zoom in near the point where writes are being shaped by the slowest stream +# (s3, at 0.5MiB/s). We see the blocked stream count bouncing between 0 and 1 +# as tokens get depleted and replenished (demand here is higher than the +# replenishment rate). +# +# As soon as s3 is disconnected, we see a release of 16MiB of held tokens back +# into the node-level controller (32MiB -> 48MiB). We see a burst in the number +# of stream-specific requests bypassing Admit() due to the stream +# disconnecting. +plot t=[30s,50s) +kvadmission.flow_controller.regular_blocked_stream_count unit=streams +kvadmission.flow_controller.regular_tokens_available unit=MiB +kvadmission.flow_controller.regular_requests_bypassed unit=reqs/s rate=true +---- +---- + 1.0 ┤ ╭╮ ╭╮ + 0.9 ┤ ││ ││ + 0.8 ┤ ╭╮││ ││ + 0.8 ┤ ││││╭╮ ││ + 0.7 ┤ ││││││ ╭╮││ + 0.6 ┤ ││││││ ││││ + 0.6 ┤ ╭╮││││││╭╮ ││││ + 0.5 ┤ ╭╯││││││││╰╮││││ + 0.5 ┤ │ ││││││││ │││││ + 0.4 ┤ ╭╯ ││││││││ ╰╯│││ + 0.3 ┤ │ ││││││╰╯ │││ + 0.3 ┤ │ ╰╯││││ │││ + 0.2 ┤ │ ││││ │││ + 0.1 ┤ │ ││╰╯ ╰╯│ + 0.1 ┤ │ ╰╯ │ + 0.0 ┼───╯ ╰─────────────────── + regular_blocked_stream_count (streams) + + + 47.6 ┤ ╭─────────────────── + 46.5 ┤ │ + 45.5 ┤ │ + 44.4 ┤ │ + 43.3 ┤ │ + 42.3 ┤ │ + 41.2 ┤ │ + 40.1 ┤ │ + 39.1 ┤ │ + 38.0 ┤ │ + 36.9 ┤ │ + 35.9 ┤ │ + 34.8 ┤ │ + 33.7 ┤ │ + 32.7 ┤ │ + 31.6 ┼───────────────────╯ + regular_tokens_available (MiB) + + + 42.0 ┤ ╭─╮ + 39.2 ┤ │ │ + 36.4 ┤ │ │ + 33.6 ┤ │ │ + 30.8 ┤ │ │ + 28.0 ┤ │ │ + 25.2 ┤ │ │ + 22.4 ┤ │ │ + 19.6 ┤ │ │ + 16.8 ┤ │ │ + 14.0 ┤ │ │ + 11.2 ┤ │ │ + 8.4 ┤ │ │ + 5.6 ┤ │ │ + 2.8 ┤ │ │ + 0.0 ┼───────────────────╯ ╰───────────────── + rate(regular_requests_bypassed) (reqs/s) +---- +---- + +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel new file mode 100644 index 000000000000..bd3536c6fec0 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel @@ -0,0 +1,36 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "kvflowtokentracker", + srcs = ["tracker.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/util/admission/admissionpb", + "//pkg/util/log", + ], +) + +go_test( + name = "kvflowtokentracker_test", + srcs = ["tracker_test.go"], + args = ["-test.timeout=295s"], + data = glob(["testdata/**"]), + embed = [":kvflowtokentracker"], + deps = [ + "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/testutils/datapathutils", + "//pkg/util/admission/admissionpb", + "//pkg/util/leaktest", + "//pkg/util/log", + "@com_github_cockroachdb_datadriven//:datadriven", + "@com_github_dustin_go_humanize//:go-humanize", + "@com_github_stretchr_testify//require", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/testdata/overview b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/testdata/overview new file mode 100644 index 000000000000..b7bc87f839e3 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/testdata/overview @@ -0,0 +1,89 @@ +init +---- + +# Track a set of tokens with incrementing log positions and varying priorities. +track +pri=normal-pri tokens=1B log-position=4/20 +pri=normal-pri tokens=1B log-position=4/21 +pri=normal-pri tokens=1B log-position=4/22 +pri=normal-pri tokens=1B log-position=4/23 +pri=low-pri tokens=1B log-position=4/24 +pri=normal-pri tokens=1B log-position=4/25 +pri=low-pri tokens=1B log-position=4/26 +pri=normal-pri tokens=1B log-position=4/27 +pri=low-pri tokens=1B log-position=4/28 +pri=normal-pri tokens=1B log-position=4/29 +---- + +# Iterate through them. We should find three low-pri tokens being tracked, and +# six normal-pri ones. +iter +---- +pri=low-pri + tokens=1B log-position=4/24 + tokens=1B log-position=4/26 + tokens=1B log-position=4/28 +pri=normal-pri + tokens=1B log-position=4/20 + tokens=1B log-position=4/21 + tokens=1B log-position=4/22 + tokens=1B log-position=4/23 + tokens=1B log-position=4/25 + tokens=1B log-position=4/27 + tokens=1B log-position=4/29 + +# Untrack a subset of normal-pri tokens, up to 4/23. This should get rid of four +# tracked tokens. +untrack pri=normal-pri up-to-log-position=4/23 +---- +pri=normal-pri + tokens=1B log-position=4/20 + tokens=1B log-position=4/21 + tokens=1B log-position=4/22 + tokens=1B log-position=4/23 +total=4B + +iter +---- +pri=low-pri + tokens=1B log-position=4/24 + tokens=1B log-position=4/26 + tokens=1B log-position=4/28 +pri=normal-pri + tokens=1B log-position=4/25 + tokens=1B log-position=4/27 + tokens=1B log-position=4/29 + +# Untracking them again is a no-op. +untrack pri=normal-pri up-to-log-position=4/23 +---- + +iter +---- +pri=low-pri + tokens=1B log-position=4/24 + tokens=1B log-position=4/26 + tokens=1B log-position=4/28 +pri=normal-pri + tokens=1B log-position=4/25 + tokens=1B log-position=4/27 + tokens=1B log-position=4/29 + +# Do the same for the low-pri tokens. Untracking at higher log positions (used +# for normal-pri) requests don't affect tokens other than low-pri ones. +untrack pri=low-pri up-to-log-position=4/29 +---- +pri=low-pri + tokens=1B log-position=4/24 + tokens=1B log-position=4/26 + tokens=1B log-position=4/28 +total=3B + +iter +---- +pri=normal-pri + tokens=1B log-position=4/25 + tokens=1B log-position=4/27 + tokens=1B log-position=4/29 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go new file mode 100644 index 000000000000..5afc6aff7068 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go @@ -0,0 +1,215 @@ +// Copyright 2023 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 kvflowtokentracker + +import ( + "context" + "fmt" + "strings" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// Tracker tracks flow token deductions. Tokens are deducted for proposed +// commands (identified by raft log position), with a given +// admissionpb.WorkPriority, for replication along an individual +// kvflowcontrol.Stream. +type Tracker struct { + trackedM map[admissionpb.WorkPriority][]tracked + + // lowerBound tracks on a per-stream basis the log position below which + // we ignore token deductions. + lowerBound kvflowcontrolpb.RaftLogPosition + + knobs *kvflowcontrol.TestingKnobs +} + +// tracked represents tracked flow tokens; they're tracked with respect to a +// raft log position (typically where the proposed command is expected to end +// up). +type tracked struct { + tokens kvflowcontrol.Tokens + position kvflowcontrolpb.RaftLogPosition +} + +// New constructs a new Tracker with the given lower bound raft log position +// (below which we're not allowed to deduct tokens). +func New(lb kvflowcontrolpb.RaftLogPosition, knobs *kvflowcontrol.TestingKnobs) *Tracker { + if knobs == nil { + knobs = &kvflowcontrol.TestingKnobs{} + } + return &Tracker{ + trackedM: make(map[admissionpb.WorkPriority][]tracked), + lowerBound: lb, + knobs: knobs, + } +} + +// Track token deductions of the given priority with the given raft log +// position. +func (dt *Tracker) Track( + ctx context.Context, + pri admissionpb.WorkPriority, + tokens kvflowcontrol.Tokens, + pos kvflowcontrolpb.RaftLogPosition, +) { + if !(dt.lowerBound.Less(pos)) { + // We're trying to track a token deduction at a position less than the + // stream's lower-bound. Shout loudly but ultimately no-op. This + // regression indicates buggy usage since: + // - token deductions are done so with monotonically increasing log + // positions (see Handle.DeductTokensFor); + // - the monotonically increasing log positions for token deductions + // also extends to positions at which streams are connected, + // which typically happen when (a) members are added to the raft + // group, (b) previously crashed follower nodes restart, (c) + // replicas are unpaused, or (d) caught back up via snapshots (see + // Handle.ConnectStream). + // - token returns upto some log position don't precede deductions at + // lower log positions (see Handle.ReturnTokensUpto); + log.Errorf(ctx, "observed raft log position less than per-stream lower bound (%s <= %s)", + pos, dt.lowerBound) + return + } + dt.lowerBound = pos + + if len(dt.trackedM[pri]) >= 1 { + last := dt.trackedM[pri][len(dt.trackedM[pri])-1] + if !last.position.Less(pos) { + log.Fatalf(ctx, "expected in order tracked log positions (%s < %s)", + last.position, pos) + } + } + dt.trackedM[pri] = append(dt.trackedM[pri], tracked{ + tokens: tokens, + position: pos, + }) +} + +// Untrack all token deductions of the given priority that have log positions +// less than or equal to the one provided. +func (dt *Tracker) Untrack( + ctx context.Context, pri admissionpb.WorkPriority, upto kvflowcontrolpb.RaftLogPosition, +) kvflowcontrol.Tokens { + if dt == nil { + return 0 + } + if _, ok := dt.trackedM[pri]; !ok { + return 0 + } + + var untracked int + var tokens kvflowcontrol.Tokens + for { + if untracked == len(dt.trackedM[pri]) { + break + } + + deduction := dt.trackedM[pri][untracked] + if !deduction.position.LessEq(upto) { + break + } + + if fn := dt.knobs.UntrackTokensInterceptor; fn != nil { + fn(deduction.tokens, deduction.position) + } + + untracked += 1 + tokens += deduction.tokens + } + + trackedBefore := len(dt.trackedM[pri]) + dt.trackedM[pri] = dt.trackedM[pri][untracked:] + if log.ExpensiveLogEnabled(ctx, 1) { + remaining := "" + if len(dt.trackedM[pri]) > 0 { + remaining = fmt.Sprintf(" (%s, ...)", dt.trackedM[pri][0].tokens) + } + log.VInfof(ctx, 1, "released flow control tokens for %d/%d pri=%s tracked deductions, upto %s; %d tracked deduction(s) remain%s", + untracked, trackedBefore, pri, upto, len(dt.trackedM[pri]), remaining) + } + if len(dt.trackedM[pri]) == 0 { + delete(dt.trackedM, pri) + } + + if dt.lowerBound.Less(upto) { + dt.lowerBound = upto + } + return tokens +} + +// Iter iterates through all tracked token deductions, invoking the provided +// callback with the sum of all tokens at a per-priority level. +func (dt *Tracker) Iter(_ context.Context, f func(admissionpb.WorkPriority, kvflowcontrol.Tokens)) { + for pri, deductions := range dt.trackedM { + var tokens kvflowcontrol.Tokens + for _, deduction := range deductions { + tokens += deduction.tokens + } + f(pri, tokens) + } +} + +// TestingIter is a testing-only re-implementation of Iter. It iterates through +// all tracked token deductions, invoking the provided callback with tracked +// pri<->token<->position triples. +func (dt *Tracker) TestingIter( + f func(admissionpb.WorkPriority, kvflowcontrol.Tokens, kvflowcontrolpb.RaftLogPosition) bool, +) { + for pri, deductions := range dt.trackedM { + for _, deduction := range deductions { + if !f(pri, deduction.tokens, deduction.position) { + return + } + } + } +} + +// TestingPrintIter iterates through all tracked tokens and returns a printable +// string, for use in tests. +func (dt *Tracker) TestingPrintIter() string { + type tracked struct { + tokens kvflowcontrol.Tokens + raftLogPosition kvflowcontrolpb.RaftLogPosition + } + const numPriorities = int(admissionpb.HighPri) - int(admissionpb.LowPri) + deductions := [numPriorities][]tracked{} + dt.TestingIter( + func(pri admissionpb.WorkPriority, tokens kvflowcontrol.Tokens, pos kvflowcontrolpb.RaftLogPosition) bool { + i := int(pri) - int(admissionpb.LowPri) + deductions[i] = append(deductions[i], tracked{ + tokens: tokens, + raftLogPosition: pos, + }) + return true + }, + ) + var buf strings.Builder + for i, ds := range deductions { + pri := i + int(admissionpb.LowPri) + if len(ds) == 0 { + continue + } + buf.WriteString(fmt.Sprintf("pri=%s\n", admissionpb.WorkPriority(pri))) + for _, deduction := range ds { + buf.WriteString(fmt.Sprintf(" tokens=%s %s\n", + testingPrintTrimmedTokens(deduction.tokens), deduction.raftLogPosition)) + } + } + return buf.String() +} + +func testingPrintTrimmedTokens(t kvflowcontrol.Tokens) string { + return strings.TrimPrefix(strings.ReplaceAll(t.String(), " ", ""), "+") +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go new file mode 100644 index 000000000000..858bda480797 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go @@ -0,0 +1,144 @@ +// Copyright 2023 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 kvflowtokentracker + +import ( + "context" + "fmt" + "strconv" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "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/datadriven" + "github.com/dustin/go-humanize" + "github.com/stretchr/testify/require" +) + +func TestTracker(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + reverseWorkPriorityDict := make(map[string]admissionpb.WorkPriority) + for k, v := range admissionpb.WorkPriorityDict { + reverseWorkPriorityDict[v] = k + } + + ctx := context.Background() + datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) { + var tracker *Tracker + knobs := &kvflowcontrol.TestingKnobs{ + UntrackTokensInterceptor: func(tokens kvflowcontrol.Tokens, pos kvflowcontrolpb.RaftLogPosition) { + + }, + } + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + tracker = New(kvflowcontrolpb.RaftLogPosition{Term: 1, Index: 0}, knobs) + return "" + + case "track": + require.NotNilf(t, tracker, "uninitialized tracker (did you use 'init'?)") + + for _, line := range strings.Split(d.Input, "\n") { + parts := strings.Fields(line) + require.Len(t, parts, 3, "expected form 'pri= tokens= log-position=/'") + + var ( + tokens kvflowcontrol.Tokens + pri admissionpb.WorkPriority + logPosition kvflowcontrolpb.RaftLogPosition + ) + + for i := range parts { + parts[i] = strings.TrimSpace(parts[i]) + inner := strings.Split(parts[i], "=") + require.Len(t, inner, 2) + arg := strings.TrimSpace(inner[1]) + + switch { + case strings.HasPrefix(parts[i], "pri="): + var found bool + pri, found = reverseWorkPriorityDict[arg] + require.True(t, found) + + case strings.HasPrefix(parts[i], "tokens="): + // Parse tokens=. + bytes, err := humanize.ParseBytes(arg) + require.NoError(t, err) + tokens = kvflowcontrol.Tokens(int64(bytes)) + + case strings.HasPrefix(parts[i], "log-position="): + // Parse log-position=/. + logPosition = parseLogPosition(t, arg) + + default: + t.Fatalf("unrecognized prefix: %s", parts[i]) + } + } + tracker.Track(ctx, pri, tokens, logPosition) + } + return "" + + case "iter": + require.NotNilf(t, tracker, "uninitialized tracker (did you use 'init'?)") + return tracker.TestingPrintIter() + + case "untrack": + require.NotNilf(t, tracker, "uninitialized tracker (did you use 'init'?)") + var priStr, logPositionStr string + d.ScanArgs(t, "pri", &priStr) + d.ScanArgs(t, "up-to-log-position", &logPositionStr) + pri, found := reverseWorkPriorityDict[priStr] + require.True(t, found) + logPosition := parseLogPosition(t, logPositionStr) + + count := 0 + var buf strings.Builder + buf.WriteString(fmt.Sprintf("pri=%s\n", pri)) + knobs.UntrackTokensInterceptor = func(tokens kvflowcontrol.Tokens, position kvflowcontrolpb.RaftLogPosition) { + count += 1 + buf.WriteString(fmt.Sprintf(" tokens=%s %s\n", + testingPrintTrimmedTokens(tokens), position)) + } + tokens := tracker.Untrack(ctx, pri, logPosition) + if count == 0 { + return "" + } + buf.WriteString(fmt.Sprintf("total=%s\n", + testingPrintTrimmedTokens(tokens))) + return buf.String() + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) + }) +} + +func parseLogPosition(t *testing.T, input string) kvflowcontrolpb.RaftLogPosition { + inner := strings.Split(input, "/") + require.Len(t, inner, 2) + term, err := strconv.Atoi(inner[0]) + require.NoError(t, err) + index, err := strconv.Atoi(inner[1]) + require.NoError(t, err) + return kvflowcontrolpb.RaftLogPosition{ + Term: uint64(term), + Index: uint64(index), + } +} diff --git a/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go b/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go new file mode 100644 index 000000000000..bfdf3a324126 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go @@ -0,0 +1,29 @@ +// Copyright 2023 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 kvflowcontrol + +import ( + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" +) + +// TestingKnobs provide fine-grained control over the various kvflowcontrol +// components for testing. +type TestingKnobs struct { + // UntrackTokensInterceptor is invoked whenever tokens are untracked, along + // with their corresponding log positions. + UntrackTokensInterceptor func(Tokens, kvflowcontrolpb.RaftLogPosition) +} + +// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. +func (t *TestingKnobs) ModuleTestingKnobs() {} + +var _ base.ModuleTestingKnobs = (*TestingKnobs)(nil)