From d60dcdfc173b2804c7cf384624a745cb4362fc05 Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Thu, 19 Sep 2024 13:58:36 -0400 Subject: [PATCH 1/8] kvserver: pass tokens into token deduction override test knob Pass the original token deduction into the `OverrideTokenDeduction` testing knob function. This change enables rounding or modifying the original value, which is useful for flow control integration tests which wish to issue variable sized writes, without the associated flakiness. Resolves: #130187 Release note: None --- pkg/kv/kvserver/flow_control_integration_test.go | 10 +++++----- .../kvflowcontrol/kvflowhandle/kvflowhandle.go | 2 +- pkg/kv/kvserver/kvflowcontrol/rac2/range_controller.go | 2 +- pkg/kv/kvserver/kvflowcontrol/testing_knobs.go | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/kv/kvserver/flow_control_integration_test.go b/pkg/kv/kvserver/flow_control_integration_test.go index 98d1d36eefce..62b0564ee155 100644 --- a/pkg/kv/kvserver/flow_control_integration_test.go +++ b/pkg/kv/kvserver/flow_control_integration_test.go @@ -800,7 +800,7 @@ func TestFlowControlRaftSnapshot(t *testing.T) { Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ UseOnlyForScratchRanges: true, - OverrideTokenDeduction: func() kvflowcontrol.Tokens { + OverrideTokenDeduction: func(_ kvflowcontrol.Tokens) kvflowcontrol.Tokens { // This test makes use of (small) increment // requests, but wants to see large token // deductions/returns. @@ -1210,7 +1210,7 @@ func TestFlowControlRaftTransportCulled(t *testing.T) { Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ UseOnlyForScratchRanges: true, - OverrideTokenDeduction: func() kvflowcontrol.Tokens { + OverrideTokenDeduction: func(_ kvflowcontrol.Tokens) kvflowcontrol.Tokens { // This test asserts on the exact values of tracked // tokens. In non-test code, the tokens deducted are // a few bytes off (give or take) from the size of @@ -1723,7 +1723,7 @@ func TestFlowControlQuiescedRange(t *testing.T) { Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ - OverrideTokenDeduction: func() kvflowcontrol.Tokens { + OverrideTokenDeduction: func(_ kvflowcontrol.Tokens) kvflowcontrol.Tokens { // This test asserts on the exact values of tracked // tokens. In non-test code, the tokens deducted are // a few bytes off (give or take) from the size of @@ -1861,7 +1861,7 @@ func TestFlowControlUnquiescedRange(t *testing.T) { Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ - OverrideTokenDeduction: func() kvflowcontrol.Tokens { + OverrideTokenDeduction: func(_ kvflowcontrol.Tokens) kvflowcontrol.Tokens { // This test asserts on the exact values of tracked // tokens. In non-test code, the tokens deducted are // a few bytes off (give or take) from the size of @@ -2236,7 +2236,7 @@ func TestFlowControlGranterAdmitOneByOne(t *testing.T) { Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ UseOnlyForScratchRanges: true, - OverrideTokenDeduction: func() kvflowcontrol.Tokens { + OverrideTokenDeduction: func(_ kvflowcontrol.Tokens) kvflowcontrol.Tokens { // This test asserts on the exact values of tracked // tokens. In non-test code, the tokens deducted are // a few bytes off (give or take) from the size of diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go index 4cddf1ebbf1c..82062567966f 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go @@ -163,7 +163,7 @@ func (h *Handle) deductTokensForInner( } if fn := h.knobs.OverrideTokenDeduction; fn != nil { - tokens = fn() + tokens = fn(tokens) } for _, c := range h.mu.connections { diff --git a/pkg/kv/kvserver/kvflowcontrol/rac2/range_controller.go b/pkg/kv/kvserver/kvflowcontrol/rac2/range_controller.go index 12e93d788bd5..46c5dab3606a 100644 --- a/pkg/kv/kvserver/kvflowcontrol/rac2/range_controller.go +++ b/pkg/kv/kvserver/kvflowcontrol/rac2/range_controller.go @@ -802,7 +802,7 @@ func (rs *replicaState) handleReadyEntries(ctx context.Context, entries []entryF } tokens := entry.tokens if fn := rs.parent.opts.Knobs.OverrideTokenDeduction; fn != nil { - tokens = fn() + tokens = fn(tokens) } rs.sendStream.mu.tracker.Track(ctx, entry.term, entry.index, entry.pri, tokens) rs.evalTokenCounter.Deduct( diff --git a/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go b/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go index 1dc470b44c07..6ad28b520204 100644 --- a/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go +++ b/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go @@ -22,7 +22,7 @@ type TestingKnobs struct { UseOnlyForScratchRanges bool // OverrideTokenDeduction is used to override how many tokens are deducted // post-evaluation. - OverrideTokenDeduction func() Tokens + OverrideTokenDeduction func(tokens Tokens) Tokens // OverrideV2EnabledWhenLeaderLevel is used to override the level at which // RACv2 is enabled when a replica is the leader. OverrideV2EnabledWhenLeaderLevel func() V2EnabledWhenLeaderLevel From 8017c61fcd09ad6b10d601b88fc694faf3bcf2a6 Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Fri, 20 Sep 2024 11:23:20 -0400 Subject: [PATCH 2/8] kvserver: add replica unreachable bypass test knob Introduce a new field on `StoreTestingKnobs`: `RaftReportUnreachableBypass`. `RaftReportUnreachableBypass` is called when a replica reports that another replica is unreachable. If the bypass function is non-nil and returns true, the report is ignored and `ReportUnreachable` is not called on the raft group for that replica. Resolves: #130187 Release note: None --- pkg/kv/kvserver/replica_raft.go | 4 ++++ pkg/kv/kvserver/replica_raft_overload.go | 4 ++++ pkg/kv/kvserver/testing_knobs.go | 6 ++++++ 3 files changed, 14 insertions(+) diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 28ef6e547143..22f9116cd623 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -1369,7 +1369,11 @@ func (r *Replica) tick( remotes := r.unreachablesMu.remotes r.unreachablesMu.remotes = nil r.unreachablesMu.Unlock() + bypassFn := r.store.TestingKnobs().RaftReportUnreachableBypass for remoteReplica := range remotes { + if bypassFn != nil && bypassFn(remoteReplica) { + continue + } r.mu.internalRaftGroup.ReportUnreachable(raftpb.PeerID(remoteReplica)) } diff --git a/pkg/kv/kvserver/replica_raft_overload.go b/pkg/kv/kvserver/replica_raft_overload.go index ebc20eea4dae..9268aa14efbc 100644 --- a/pkg/kv/kvserver/replica_raft_overload.go +++ b/pkg/kv/kvserver/replica_raft_overload.go @@ -357,7 +357,11 @@ func (r *Replica) updatePausedFollowersLocked(ctx context.Context, ioThresholdMa seed: seed, } r.mu.pausedFollowers, _ = computeExpendableOverloadedFollowers(ctx, d) + bypassFn := r.store.TestingKnobs().RaftReportUnreachableBypass for replicaID := range r.mu.pausedFollowers { + if bypassFn != nil && bypassFn(replicaID) { + continue + } // We're dropping messages to those followers (see handleRaftReady) but // it's a good idea to tell raft not to even bother sending in the first // place. Raft will react to this by moving the follower to probing state diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index b50281cbed76..a0024fac9e3e 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -540,6 +540,12 @@ type StoreTestingKnobs struct { // FlowControlTestingKnobs provide fine-grained control over the various // kvflowcontrol components for testing. FlowControlTestingKnobs *kvflowcontrol.TestingKnobs + + // RaftReportUnreachableBypass is called when a replica reports that another + // replica is unreachable. If the bypass function is non-nil and returns + // true, the report is ignored and ReportUnreachable is not called on the + // raft group for that replica. + RaftReportUnreachableBypass func(roachpb.ReplicaID) bool } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. From 6faefcf4ce2f33f59cb9f768a9a0bd4309b46e05 Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Tue, 24 Sep 2024 17:25:51 -0400 Subject: [PATCH 3/8] replica_rac2: fix is leader v2 protocol when leader The `Processor` calls `isLeaderUsingV2ProcLocked` to determine which store work queue admit method to call, opting for the v1 method if `isLeaderUsingV2ProcLocked` returns false. Update `isLeaderUsingV2ProcLocked` to correctly return false when the local replica is the leader and running v1, having previously seen a leader running v2. Part of: #130431 Release note: None --- pkg/kv/kvserver/kvflowcontrol/replica_rac2/processor.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/kvflowcontrol/replica_rac2/processor.go b/pkg/kv/kvserver/kvflowcontrol/replica_rac2/processor.go index e6698b706365..8d65c6a42164 100644 --- a/pkg/kv/kvserver/kvflowcontrol/replica_rac2/processor.go +++ b/pkg/kv/kvserver/kvflowcontrol/replica_rac2/processor.go @@ -535,7 +535,7 @@ func NewProcessor(opts ProcessorOptions) Processor { func (p *processorImpl) isLeaderUsingV2ProcLocked() bool { // We are the leader using V2, or a follower who learned that the leader is // using the V2 protocol. - return p.leader.rc != nil || p.follower.isLeaderUsingV2Protocol + return p.leader.rc != nil || (p.opts.ReplicaID != p.leaderID && p.follower.isLeaderUsingV2Protocol) } // InitRaftLocked implements Processor. @@ -933,7 +933,7 @@ func (p *processorImpl) AdmitRaftEntriesRaftMuLocked(ctx context.Context, e rac2 log.Infof(ctx, "decoded v2 raft admission meta below-raft: pri=%v create-time=%d "+ "proposer=n%v receiver=[n%d,s%v] tenant=t%d tokens≈%d "+ - "sideloaded=%t raft-entry=%d/%d", + "sideloaded=%t raft-entry=%d/%d lead-v2=%v", raftpb.Priority(meta.AdmissionPriority), meta.AdmissionCreateTime, meta.AdmissionOriginNode, @@ -944,12 +944,13 @@ func (p *processorImpl) AdmitRaftEntriesRaftMuLocked(ctx context.Context, e rac2 typ.IsSideloaded(), entry.Term, entry.Index, + p.isLeaderUsingV2ProcLocked(), ) } else { log.Infof(ctx, "decoded v1 raft admission meta below-raft: pri=%v create-time=%d "+ "proposer=n%v receiver=[n%d,s%v] tenant=t%d tokens≈%d "+ - "sideloaded=%t raft-entry=%d/%d", + "sideloaded=%t raft-entry=%d/%d lead-v2=%v", admissionpb.WorkPriority(meta.AdmissionPriority), meta.AdmissionCreateTime, meta.AdmissionOriginNode, @@ -960,6 +961,7 @@ func (p *processorImpl) AdmitRaftEntriesRaftMuLocked(ctx context.Context, e rac2 typ.IsSideloaded(), entry.Term, entry.Index, + p.isLeaderUsingV2ProcLocked(), ) } } From b0564374b2115baa19a0b292586469eabda6420d Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Thu, 19 Sep 2024 20:05:48 -0400 Subject: [PATCH 4/8] kvserver: refactor v1 flow control integration tests for v2 The existing flow control integration tests, in `flow_control_integration_test.go`, provide a substantial suite of tests to verify flow control integration behavior. Refactor the `flowControlTestHelper` and associated use in tests, to enable sharing the helper between v1 (existing) and not-yet implemented v2 flow control integration tests. Part of: #130187 Release note: None --- .../kvserver/flow_control_integration_test.go | 617 +++++++++--------- 1 file changed, 298 insertions(+), 319 deletions(-) diff --git a/pkg/kv/kvserver/flow_control_integration_test.go b/pkg/kv/kvserver/flow_control_integration_test.go index 62b0564ee155..1c21b116cd01 100644 --- a/pkg/kv/kvserver/flow_control_integration_test.go +++ b/pkg/kv/kvserver/flow_control_integration_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -98,47 +99,24 @@ func TestFlowControlBasic(t *testing.T) { desc, err := tc.LookupRange(k) require.NoError(t, err) - for i := 0; i < numNodes; i++ { - si, err := tc.Server(i).GetStores().(*kvserver.Stores).GetStore(tc.Server(i).GetFirstStoreID()) - require.NoError(t, err) - tc.Servers[i].RaftTransport().(*kvserver.RaftTransport).ListenIncomingRaftMessages(si.StoreID(), - &unreliableRaftHandler{ - rangeID: desc.RangeID, - IncomingRaftMessageHandler: si, - unreliableRaftHandlerFuncs: unreliableRaftHandlerFuncs{ - dropReq: func(req *kvserverpb.RaftMessageRequest) bool { - // Install a raft handler to get verbose raft logging. - // - // TODO(irfansharif): Make this a more ergonomic - // testing knob instead. - return false - }, - }, - }) - } - n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("basic") // this test behaves identically with or without the fast path + h.enableVerboseRaftMsgLoggingForRange(desc.RangeID) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- Flow token metrics, before issuing the regular 1MiB replicated write.`) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- (Issuing + admitting a regular 1MiB, triply replicated write...)`) h.log("sending put request") h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) h.log("sent put request") - h.waitForAllTokensReturned(ctx, 3) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) h.comment(` -- Stream counts as seen by n1 post-write. We should see three {regular,elastic} -- streams given there are three nodes and we're using a replication factor of @@ -165,12 +143,7 @@ ORDER BY streams DESC; -- {regular,elastic} tokens deducted and returned, and {8*3=24MiB,16*3=48MiB} of -- {regular,elastic} tokens available. Everything should be accounted for. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) // When run using -v the vmodule described at the top of this file, this // test demonstrates end-to-end flow control machinery in the happy @@ -270,34 +243,29 @@ func TestFlowControlRangeSplitMerge(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("split_merge") desc, err := tc.LookupRange(k) require.NoError(t, err) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.log("sending put request to pre-split range") h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) h.log("sent put request to pre-split range") - h.waitForAllTokensReturned(ctx, 3) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) h.comment(` -- Flow token metrics from n1 after issuing + admitting the regular 1MiB 3x -- replicated write to the pre-split range. There should be 3MiB of -- {regular,elastic} tokens {deducted,returned}. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- (Splitting range.)`) left, right := tc.SplitRangeOrFatal(t, k.Next()) - h.waitForConnectedStreams(ctx, right.RangeID, 3) + h.waitForConnectedStreams(ctx, right.RangeID, 3, 0 /* serverIdx */) // [T1,n1,s1,r63/1:/{Table/62-Max},*kvpb.AdminSplitRequest] initiating a split of this range at key /Table/Max [r64] (manual) // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] connected to stream: t1/s1 // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] connected to stream: t1/s2 @@ -311,19 +279,14 @@ ORDER BY name ASC; h.put(ctx, roachpb.Key(right.StartKey), 3<<20 /* 3MiB */, admissionpb.NormalPri) h.log("sent 3MiB put request to post-split RHS") - h.waitForAllTokensReturned(ctx, 3) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) h.comment(` -- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to -- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens -- {deducted,returned}, which comes from (2MiB+3MiB)*3=15MiB. So we stand at -- 3MiB+15MiB=18MiB now. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- Observe the newly split off replica, with its own three streams.`) h.query(n1, ` @@ -348,18 +311,13 @@ ORDER BY streams DESC; h.put(ctx, roachpb.Key(merged.StartKey), 4<<20 /* 4MiB */, admissionpb.NormalPri) h.log("sent 4MiB put request to post-merged range") - h.waitForAllTokensReturned(ctx, 3) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) h.comment(` -- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to -- the post-merged range. We should see 12MiB extra tokens {deducted,returned}, -- which comes from 4MiB*3=12MiB. So we stand at 18MiB+12MiB=30MiB now. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- Observe only the merged replica with its own three streams.`) h.query(n1, ` @@ -411,13 +369,13 @@ func TestFlowControlBlockedAdmission(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) n2 := sqlutils.MakeSQLRunner(tc.ServerConn(1)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("blocked_admission") desc, err := tc.LookupRange(k) require.NoError(t, err) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing regular 1MiB, 3x replicated write that's not admitted.)`) h.log("sending put requests") @@ -431,12 +389,7 @@ func TestFlowControlBlockedAdmission(t *testing.T) { -- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of -- {regular,elastic} tokens with no corresponding returns. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- Observe the total tracked tokens per-stream on n1.`) h.query(n1, ` @@ -452,7 +405,7 @@ ORDER BY name ASC; h.comment(`-- (Allow below-raft admission to proceed.)`) disableWorkQueueGranting.Store(false) - h.waitForAllTokensReturned(ctx, 3) // wait for admission + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) // wait for admission h.comment(`-- Observe flow token dispatch metrics from n1.`) h.query(n1, ` @@ -475,12 +428,7 @@ ORDER BY name ASC; -- {regular,elastic} tokens, and the available capacities going back to what -- they were. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) } // TestFlowControlAdmissionPostSplitMerge walks through what happens with flow @@ -529,14 +477,14 @@ func TestFlowControlAdmissionPostSplitMerge(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("admission_post_split_merge") desc, err := tc.LookupRange(k) require.NoError(t, err) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.log("sending put request to pre-split range") h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) @@ -549,16 +497,11 @@ func TestFlowControlAdmissionPostSplitMerge(t *testing.T) { -- {regular,elastic} tokens with no corresponding returns. The 2*1MiB writes -- happened on what is soon going to be the LHS and RHS of a range being split. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- (Splitting range.)`) left, right := tc.SplitRangeOrFatal(t, k.Next()) - h.waitForConnectedStreams(ctx, right.RangeID, 3) + h.waitForConnectedStreams(ctx, right.RangeID, 3, 0 /* serverIdx */) h.log("sending 2MiB put request to post-split LHS") h.put(ctx, k, 2<<20 /* 2MiB */, admissionpb.NormalPri) @@ -574,12 +517,7 @@ ORDER BY name ASC; -- deducted which comes from (2MiB+3MiB)*3=15MiB. So we stand at -- 6MiB+15MiB=21MiB now. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- Observe the newly split off replica, with its own three streams.`) h.query(n1, ` @@ -620,7 +558,7 @@ ORDER BY streams DESC; h.comment(`-- (Allow below-raft admission to proceed.)`) disableWorkQueueGranting.Store(false) - h.waitForAllTokensReturned(ctx, 3) // wait for admission + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) // wait for admission h.comment(` -- Flow token metrics from n1 after work gets admitted. We see all outstanding @@ -628,12 +566,7 @@ ORDER BY streams DESC; -- - the LHS before the merge, and -- - the LHS and RHS before the original split. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) } // TestFlowControlCrashedNode tests flow token behavior in the presence of @@ -694,14 +627,14 @@ func TestFlowControlCrashedNode(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("crashed_node") desc, err := tc.LookupRange(k) require.NoError(t, err) tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) - h.waitForConnectedStreams(ctx, desc.RangeID, 2) + h.waitForConnectedStreams(ctx, desc.RangeID, 2, 0 /* serverIdx */) h.comment(`-- (Issuing regular 5x1MiB, 2x replicated writes that are not admitted.)`) h.log("sending put requests") @@ -715,12 +648,7 @@ func TestFlowControlCrashedNode(t *testing.T) { -- that are yet to get admitted. We see 5*1MiB*2=10MiB deductions of -- {regular,elastic} tokens with no corresponding returns. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- Observe the per-stream tracked tokens on n1, before n2 is crashed.`) h.query(n1, ` SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) @@ -737,7 +665,7 @@ ORDER BY name ASC; h.comment(`-- (Crashing n2 but disabling the raft-transport-break token return mechanism.)`) tc.StopServer(1) - h.waitForConnectedStreams(ctx, desc.RangeID, 1) + h.waitForConnectedStreams(ctx, desc.RangeID, 1, 0 /* serverIdx */) h.comment(` -- Observe the per-stream tracked tokens on n1, after n2 crashed. We're no @@ -752,12 +680,7 @@ ORDER BY name ASC; -- Flow token metrics from n1 after n2 crashed. Observe that we've returned the -- 5MiB previously held by n2. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) } // TestFlowControlRaftSnapshot tests flow token behavior when one replica needs @@ -847,7 +770,7 @@ func TestFlowControlRaftSnapshot(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) n4 := sqlutils.MakeSQLRunner(tc.ServerConn(3)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("raft_snapshot") @@ -863,7 +786,7 @@ func TestFlowControlRaftSnapshot(t *testing.T) { tc.AddVotersOrFatal(t, k, tc.Targets(3, 4)...) repl := store.LookupReplica(roachpb.RKey(k)) require.NotNil(t, repl) - h.waitForConnectedStreams(ctx, repl.RangeID, 5) + h.waitForConnectedStreams(ctx, repl.RangeID, 5, 0 /* serverIdx */) // Set up a key to replicate across the cluster. We're going to modify this // key and truncate the raft logs from that command after killing one of the @@ -881,12 +804,7 @@ func TestFlowControlRaftSnapshot(t *testing.T) { -- that's not admitted. Since this test is ignoring crashed nodes for token -- deduction purposes, we see a deduction of 5MiB {regular,elastic} tokens. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; - `) + h.query(n1, v1FlowTokensQueryStr) h.comment(` -- Observe the total tracked tokens per-stream on n1. 1MiB is tracked for n1-n5. `) @@ -951,12 +869,7 @@ ORDER BY name ASC; -- RaftTransport streams). But this test is intentionally suppressing that -- behavior to observe token returns when sending raft snapshots. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; - `) + h.query(n1, v1FlowTokensQueryStr) h.comment(` -- Observe the total tracked tokens per-stream on n1. 2MiB is tracked for n1-n5; -- see last comment for an explanation why we're still deducting for n2, n3. @@ -1007,12 +920,7 @@ ORDER BY name ASC; -- progress state, noting that since we've truncated our log, we need to catch -- it up via snapshot. So we release all held tokens. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(` -- Observe the total tracked tokens per-stream on n1. There's nothing tracked @@ -1024,14 +932,14 @@ ORDER BY name ASC; WHERE total_tracked_tokens > 0 `, "range_id", "store_id", "total_tracked_tokens") - h.waitForConnectedStreams(ctx, repl.RangeID, 5) + h.waitForConnectedStreams(ctx, repl.RangeID, 5, 0 /* serverIdx */) // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 651 connected to stream: t1/s2 // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 710 connected to stream: t1/s3 h.comment(`-- (Allow below-raft admission to proceed.)`) disableWorkQueueGranting.Store(false) - h.waitForAllTokensReturned(ctx, 5) + h.waitForAllTokensReturned(ctx, 5, 0 /* serverIdx */) h.comment(`-- Observe flow token dispatch metrics from n4.`) h.query(n4, ` @@ -1045,12 +953,7 @@ ORDER BY name ASC; -- Flow token metrics from n1 after work gets admitted. We see the remaining -- 6MiB of {regular,elastic} tokens returned. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(` -- Observe the total tracked tokens per-stream on n1; there should be nothing. @@ -1119,14 +1022,14 @@ func TestFlowControlRaftTransportBreak(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("raft_transport_break") desc, err := tc.LookupRange(k) require.NoError(t, err) tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing regular 5x1MiB, 3x replicated writes that are not admitted.)`) h.log("sending put requests") @@ -1140,12 +1043,7 @@ func TestFlowControlRaftTransportBreak(t *testing.T) { -- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of -- {regular,elastic} tokens with no corresponding returns. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(` -- Observe the per-stream tracked tokens on n1, before n2 is crashed. `) @@ -1165,7 +1063,7 @@ ORDER BY name ASC; h.comment(`-- (Crashing n2 but disabling the last-updated token return mechanism.)`) tc.StopServer(1) - h.waitForConnectedStreams(ctx, desc.RangeID, 2) + h.waitForConnectedStreams(ctx, desc.RangeID, 2, 0 /* serverIdx */) h.comment(` -- Observe the per-stream tracked tokens on n1, after n2 crashed. We're no @@ -1181,12 +1079,7 @@ ORDER BY name ASC; -- Flow token metrics from n1 after n2 crashed. Observe that we've returned the -- 5MiB previously held by n2. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) } // TestFlowControlRaftTransportCulled tests flow token behavior when the raft @@ -1255,14 +1148,14 @@ func TestFlowControlRaftTransportCulled(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("raft_transport_culled") desc, err := tc.LookupRange(k) require.NoError(t, err) tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing regular 5x1MiB, 3x replicated writes that are not admitted.)`) h.log("sending put requests") @@ -1276,12 +1169,7 @@ func TestFlowControlRaftTransportCulled(t *testing.T) { -- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of -- {regular,elastic} tokens with no corresponding returns. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(` -- Observe the per-stream tracked tokens on n1, before we cull the n1<->n2 raft -- transport stream out of idleness. @@ -1305,7 +1193,7 @@ ORDER BY name ASC; t.Fatalf("timed out") } - h.waitForTotalTrackedTokens(ctx, desc.RangeID, 10<<20 /* 5*1MiB*2=10MiB */) + h.waitForTotalTrackedTokens(ctx, desc.RangeID, 10<<20 /* 5*1MiB*2=10MiB */, 0 /* serverIdx */) h.comment(` -- Observe the per-stream tracked tokens on n1 after n2->n1 raft transport @@ -1322,12 +1210,7 @@ ORDER BY name ASC; -- Flow token metrics from n1 after n2->n1 raft transport stream is culled. -- Observe that we've returned the 5MiB previously held by n2. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) disableWorkerTeardown.Store(true) } @@ -1371,13 +1254,13 @@ func TestFlowControlRaftMembership(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("raft_membership") desc, err := tc.LookupRange(k) require.NoError(t, err) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) @@ -1396,7 +1279,7 @@ ORDER BY name ASC; h.comment(`-- (Adding a voting replica on n4.)`) tc.AddVotersOrFatal(t, k, tc.Target(3)) - h.waitForConnectedStreams(ctx, desc.RangeID, 4) + h.waitForConnectedStreams(ctx, desc.RangeID, 4, 0 /* serverIdx */) h.comment(` -- Observe the total tracked tokens per-stream on n1. s1-s3 should have 1MiB @@ -1421,11 +1304,11 @@ ORDER BY name ASC; h.comment(`-- (Removing voting replica from n3.)`) tc.RemoveVotersOrFatal(t, k, tc.Target(2)) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Adding non-voting replica to n5.)`) tc.AddNonVotersOrFatal(t, k, tc.Target(4)) - h.waitForConnectedStreams(ctx, desc.RangeID, 4) + h.waitForConnectedStreams(ctx, desc.RangeID, 4, 0 /* serverIdx */) h.comment(`-- (Issuing 1x1MiB, 4x replicated write (w/ one non-voter) that's not admitted.`) h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) @@ -1443,7 +1326,7 @@ ORDER BY name ASC; h.comment(`-- (Allow below-raft admission to proceed.)`) disableWorkQueueGranting.Store(false) - h.waitForAllTokensReturned(ctx, 5) + h.waitForAllTokensReturned(ctx, 5, 0 /* serverIdx */) h.comment(`-- Observe that there no tracked tokens across s1,s2,s4,s5.`) h.query(n1, ` @@ -1456,12 +1339,7 @@ ORDER BY name ASC; -- tokens deducted are returned, including from when s3 was removed as a raft -- member. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) } // TestFlowControlRaftMembershipRemoveSelf tests flow token behavior when the @@ -1524,7 +1402,7 @@ func TestFlowControlRaftMembershipRemoveSelf(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("raft_membership_remove_self") // this test behaves identically independent of we transfer the lease first @@ -1533,7 +1411,7 @@ func TestFlowControlRaftMembershipRemoveSelf(t *testing.T) { // Make sure the lease is on n1 and that we're triply connected. tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) @@ -1568,33 +1446,23 @@ ORDER BY name ASC; } return nil }) - h.waitForAllTokensReturned(ctx, 3) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) h.comment(` -- Flow token metrics from n1 after raft leader removed itself from raft group. -- All {regular,elastic} tokens deducted are returned. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- (Allow below-raft admission to proceed.)`) disableWorkQueueGranting.Store(false) - h.waitForAllTokensReturned(ctx, 3) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) h.comment(` -- Flow token metrics from n1 after work gets admitted. Tokens were already -- returned earlier, so there's no change. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) }) } @@ -1639,13 +1507,13 @@ func TestFlowControlClassPrioritization(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("class_prioritization") desc, err := tc.LookupRange(k) require.NoError(t, err) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.)`) h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.BulkNormalPri) @@ -1655,12 +1523,7 @@ func TestFlowControlClassPrioritization(t *testing.T) { -- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with -- no corresponding returns. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- (Issuing 1x1MiB, 3x replicated regular write that's not admitted.)`) h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) @@ -1670,27 +1533,17 @@ ORDER BY name ASC; -- that's not admitted. We see 1*1MiB*3=3MiB deductions of {regular,elastic} -- tokens with no corresponding returns. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- (Allow below-raft admission to proceed.)`) disableWorkQueueGranting.Store(false) - h.waitForAllTokensReturned(ctx, 3) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) h.comment(` -- Flow token metrics from n1 after work gets admitted. All {regular,elastic} -- tokens deducted are returned. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) } // TestFlowControlQuiescedRange tests flow token behavior when ranges are @@ -1758,13 +1611,13 @@ func TestFlowControlQuiescedRange(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("quiesced_range") desc, err := tc.LookupRange(k) require.NoError(t, err) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.)`) h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.BulkNormalPri) @@ -1805,7 +1658,7 @@ ORDER BY name ASC; -- dispatch mechanism is disabled. Deducted elastic tokens from remote stores -- are yet to be returned. Tokens for the local store are. `) - h.waitForTotalTrackedTokens(ctx, desc.RangeID, 2<<20 /* 2*1MiB=2MiB */) + h.waitForTotalTrackedTokens(ctx, desc.RangeID, 2<<20 /* 2*1MiB=2MiB */, 0 /* serverIdx */) h.query(n1, ` SELECT name, crdb_internal.humanize_bytes(value::INT8) FROM crdb_internal.node_metrics @@ -1815,7 +1668,7 @@ ORDER BY name ASC; h.comment(`-- (Enable the fallback token dispatch mechanism.)`) disableFallbackTokenDispatch.Store(false) - h.waitForAllTokensReturned(ctx, 3) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) h.comment(` -- Flow token metrics from n1 after work gets admitted and all elastic tokens @@ -1904,33 +1757,15 @@ func TestFlowControlUnquiescedRange(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("unquiesced_range") desc, err := tc.LookupRange(k) require.NoError(t, err) + h.enableVerboseRaftMsgLoggingForRange(desc.RangeID) - for i := 0; i < numNodes; i++ { - si, err := tc.Server(i).GetStores().(*kvserver.Stores).GetStore(tc.Server(i).GetFirstStoreID()) - require.NoError(t, err) - tc.Servers[i].RaftTransport().(*kvserver.RaftTransport).ListenIncomingRaftMessages(si.StoreID(), - &unreliableRaftHandler{ - rangeID: desc.RangeID, - IncomingRaftMessageHandler: si, - unreliableRaftHandlerFuncs: unreliableRaftHandlerFuncs{ - dropReq: func(req *kvserverpb.RaftMessageRequest) bool { - // Install a raft handler to get verbose raft logging. - // - // TODO(irfansharif): Make this a more ergonomic - // testing knob instead. - return false - }, - }, - }) - } - - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.)`) h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.BulkNormalPri) @@ -1969,7 +1804,7 @@ ORDER BY name ASC; -- dispatch mechanism is disabled. Deducted elastic tokens from remote stores -- are yet to be returned. Tokens for the local store are. `) - h.waitForTotalTrackedTokens(ctx, desc.RangeID, 2<<20 /* 2*1MiB=2MiB */) + h.waitForTotalTrackedTokens(ctx, desc.RangeID, 2<<20 /* 2*1MiB=2MiB */, 0 /* serverIdx */) h.query(n1, ` SELECT name, crdb_internal.humanize_bytes(value::INT8) FROM crdb_internal.node_metrics @@ -1984,7 +1819,7 @@ ORDER BY name ASC; testutils.SucceedsSoon(t, func() error { _, err := tc.GetRaftLeader(t, roachpb.RKey(k)).MaybeUnquiesceAndPropose() require.NoError(t, err) - return h.checkAllTokensReturned(ctx, 3) + return h.checkAllTokensReturned(ctx, 3, 0 /* serverIdx */) }) h.comment(` @@ -2039,13 +1874,13 @@ func TestFlowControlTransferLease(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("transfer_lease") desc, err := tc.LookupRange(k) require.NoError(t, err) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) @@ -2070,7 +1905,7 @@ ORDER BY name ASC; } return nil }) - h.waitForAllTokensReturned(ctx, 3) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) h.comment(` -- Flow token metrics from n1 having lost the lease and raft leadership. All @@ -2129,13 +1964,13 @@ func TestFlowControlLeaderNotLeaseholder(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) n2 := sqlutils.MakeSQLRunner(tc.ServerConn(1)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("leader_not_leaseholder") desc, err := tc.LookupRange(k) require.NoError(t, err) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) @@ -2197,7 +2032,7 @@ ORDER BY name ASC; h.comment(`-- (Allow below-raft admission to proceed.)`) disableWorkQueueGranting.Store(false) - h.waitForAllTokensReturned(ctx, 3) // wait for admission + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) // wait for admission h.comment(` -- All deducted flow tokens are returned back to where the raft leader is. @@ -2273,13 +2108,13 @@ func TestFlowControlGranterAdmitOneByOne(t *testing.T) { n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - h := newFlowControlTestHelper(t, tc) + h := newFlowControlTestHelperV1(t, tc) h.init() defer h.close("granter_admit_one_by_one") desc, err := tc.LookupRange(k) require.NoError(t, err) - h.waitForConnectedStreams(ctx, desc.RangeID, 3) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) h.comment(`-- (Issuing regular 1024*1KiB, 3x replicated writes that are not admitted.)`) h.log("sending put requests") @@ -2293,12 +2128,7 @@ func TestFlowControlGranterAdmitOneByOne(t *testing.T) { -- that are yet to get admitted. We see 3*1MiB=3MiB deductions of -- {regular,elastic} tokens with no corresponding returns. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) h.comment(`-- Observe the total tracked tokens per-stream on n1.`) h.query(n1, ` @@ -2308,39 +2138,57 @@ ORDER BY name ASC; h.comment(`-- (Allow below-raft admission to proceed.)`) disableWorkQueueGranting.Store(false) - h.waitForAllTokensReturned(ctx, 3) // wait for admission + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) // wait for admission h.comment(` -- Flow token metrics from n1 after work gets admitted. We see 3MiB returns of -- {regular,elastic} tokens, and the available capacities going back to what -- they were. In #105185, by now we would've observed panics. `) - h.query(n1, ` - SELECT name, crdb_internal.humanize_bytes(value::INT8) - FROM crdb_internal.node_metrics - WHERE name LIKE '%kvadmission%tokens%' -ORDER BY name ASC; -`) + h.query(n1, v1FlowTokensQueryStr) } type flowControlTestHelper struct { - t *testing.T - tc *testcluster.TestCluster - buf *strings.Builder - rng *rand.Rand + t *testing.T + tc *testcluster.TestCluster + st *cluster.Settings + buf *strings.Builder + rng *rand.Rand + testdata string + level kvflowcontrol.V2EnabledWhenLeaderLevel + isStaticLevel bool } -func newFlowControlTestHelper(t *testing.T, tc *testcluster.TestCluster) *flowControlTestHelper { +func newFlowControlTestHelper( + t *testing.T, + tc *testcluster.TestCluster, + testdata string, + level kvflowcontrol.V2EnabledWhenLeaderLevel, + isStatic bool, +) *flowControlTestHelper { rng, _ := randutil.NewPseudoRand() buf := &strings.Builder{} return &flowControlTestHelper{ - t: t, - tc: tc, - buf: buf, - rng: rng, + t: t, + tc: tc, + st: tc.Server(0).ClusterSettings(), + buf: buf, + rng: rng, + testdata: testdata, + level: level, + isStaticLevel: isStatic, } } +func newFlowControlTestHelperV1(t *testing.T, tc *testcluster.TestCluster) *flowControlTestHelper { + return newFlowControlTestHelper(t, + tc, + "flow_control_integration", /* testdata */ + kvflowcontrol.V2NotEnabledWhenLeader, + true, /* isStatic */ + ) +} + func (h *flowControlTestHelper) init() { // Reach into each server's cluster setting and override. This causes any // registered change callbacks to run immediately, which is important since @@ -2352,56 +2200,100 @@ func (h *flowControlTestHelper) init() { } } -func (h *flowControlTestHelper) waitForAllTokensReturned(ctx context.Context, expStreamCount int) { +// waitForAllTokensReturned waits for all tokens to be returned across all +// streams. The expected number of streams and protocol level is passed in as +// an argument, in order to allow switching between v1 and v2 flow control. +func (h *flowControlTestHelper) waitForAllTokensReturned( + ctx context.Context, expStreamCount, serverIdx int, lvl ...kvflowcontrol.V2EnabledWhenLeaderLevel, +) { testutils.SucceedsSoon(h.t, func() error { - return h.checkAllTokensReturned(ctx, expStreamCount) + return h.checkAllTokensReturned(ctx, expStreamCount, serverIdx, lvl...) }) } +// checkAllTokensReturned checks that all tokens have been returned across all +// streams. It also checks that the expected number of streams are present. The +// protocol level is passed in as an argument, in order to allow switching +// between v1 and v2 flow control. func (h *flowControlTestHelper) checkAllTokensReturned( - ctx context.Context, expStreamCount int, + ctx context.Context, expStreamCount, serverIdx int, lvl ...kvflowcontrol.V2EnabledWhenLeaderLevel, ) error { - kfc := h.tc.Server(0).KVFlowController().(kvflowcontrol.Controller) - streams := kfc.Inspect(ctx) + var streams []kvflowinspectpb.Stream + level := h.resolveLevelArgs(lvl...) + switch level { + case kvflowcontrol.V2NotEnabledWhenLeader: + streams = h.tc.Server(serverIdx).KVFlowController().(kvflowcontrol.Controller).Inspect(ctx) + case kvflowcontrol.V2EnabledWhenLeaderV1Encoding, kvflowcontrol.V2EnabledWhenLeaderV2Encoding: + streams = h.tc.GetFirstStoreFromServer(h.t, serverIdx).GetStoreConfig().KVFlowStreamTokenProvider.Inspect(ctx) + default: + h.t.Fatalf("unknown level: %v", level) + } + + elasticTokensPerStream := kvflowcontrol.ElasticTokensPerStream.Get(&h.st.SV) + regularTokensPerStream := kvflowcontrol.RegularTokensPerStream.Get(&h.st.SV) if len(streams) != expStreamCount { - return fmt.Errorf("expected %d replication streams, got %d", expStreamCount, len(streams)) + return fmt.Errorf("expected %d replication streams, got %d [%+v]", expStreamCount, len(streams), streams) } - for _, stream := range streams { - if stream.AvailableEvalRegularTokens != 16<<20 { - return fmt.Errorf("expected %s of regular flow tokens for %s, got %s", - humanize.IBytes(16<<20), - kvflowcontrol.Stream{ - TenantID: stream.TenantID, - StoreID: stream.StoreID, - }, - humanize.IBytes(uint64(stream.AvailableEvalRegularTokens)), + + checkTokens := func( + expTokens, actualTokens int64, + stream kvflowcontrol.Stream, + typName string, + ) error { + if actualTokens != expTokens { + return fmt.Errorf("expected %v of %s flow tokens for %v, got %v", + humanize.IBytes(uint64(expTokens)), typName, stream, + humanize.IBytes(uint64(actualTokens)), ) } - if stream.AvailableEvalElasticTokens != 8<<20 { - return fmt.Errorf("expected %s of elastic flow tokens for %s, got %s", - humanize.IBytes(8<<20), - kvflowcontrol.Stream{ - TenantID: stream.TenantID, - StoreID: stream.StoreID, - }, - humanize.IBytes(uint64(stream.AvailableEvalElasticTokens)), - ) + return nil + } + + for _, stream := range streams { + s := kvflowcontrol.Stream{ + TenantID: stream.TenantID, + StoreID: stream.StoreID, + } + if err := checkTokens( + regularTokensPerStream, stream.AvailableEvalRegularTokens, s, "regular eval", + ); err != nil { + return err + } + if err := checkTokens( + elasticTokensPerStream, stream.AvailableEvalElasticTokens, s, "elastic eval", + ); err != nil { + return err + } + if level > kvflowcontrol.V2NotEnabledWhenLeader { + // V2 flow control also has send tokens. + if err := checkTokens( + regularTokensPerStream, stream.AvailableSendRegularTokens, s, "regular send", + ); err != nil { + return err + } + if err := checkTokens( + elasticTokensPerStream, stream.AvailableSendElasticTokens, s, "elastic send", + ); err != nil { + return err + } } } return nil } func (h *flowControlTestHelper) waitForConnectedStreams( - ctx context.Context, rangeID roachpb.RangeID, expConnectedStreams int, + ctx context.Context, + rangeID roachpb.RangeID, + expConnectedStreams, serverIdx int, + lvl ...kvflowcontrol.V2EnabledWhenLeaderLevel, ) { + level := h.resolveLevelArgs(lvl...) testutils.SucceedsSoon(h.t, func() error { - kfh := h.tc.Server(0).KVFlowHandles().(kvflowcontrol.Handles) - handle, found := kfh.Lookup(rangeID) + state, found := h.getInspectHandlesForLevel(serverIdx, level).LookupInspect(rangeID) if !found { return fmt.Errorf("handle for %s not found", rangeID) } require.True(h.t, found) - state := handle.Inspect(ctx) if len(state.ConnectedStreams) != expConnectedStreams { return fmt.Errorf("expected %d connected streams, got %d", expConnectedStreams, len(state.ConnectedStreams)) @@ -2411,16 +2303,19 @@ func (h *flowControlTestHelper) waitForConnectedStreams( } func (h *flowControlTestHelper) waitForTotalTrackedTokens( - ctx context.Context, rangeID roachpb.RangeID, expTotalTrackedTokens int64, + ctx context.Context, + rangeID roachpb.RangeID, + expTotalTrackedTokens int64, + serverIdx int, + lvl ...kvflowcontrol.V2EnabledWhenLeaderLevel, ) { + level := h.resolveLevelArgs(lvl...) testutils.SucceedsSoon(h.t, func() error { - kfh := h.tc.Server(0).KVFlowHandles().(kvflowcontrol.Handles) - handle, found := kfh.Lookup(rangeID) + state, found := h.getInspectHandlesForLevel(serverIdx, level).LookupInspect(rangeID) if !found { return fmt.Errorf("handle for %s not found", rangeID) } require.True(h.t, found) - state := handle.Inspect(ctx) var totalTracked int64 for _, stream := range state.ConnectedStreams { for _, tracked := range stream.TrackedDeductions { @@ -2451,7 +2346,49 @@ func (h *flowControlTestHelper) log(msg string) { } } +// resolveLevelArgs resolves the level to use for the test. If the level is +// static, the level is returned as is. If the level is dynamic, the level is +// resolved via arguments if provided, otherwise the default given at +// construction is used. The function verifies that no more than one level is +// provided. +func (h *flowControlTestHelper) resolveLevelArgs( + level ...kvflowcontrol.V2EnabledWhenLeaderLevel, +) kvflowcontrol.V2EnabledWhenLeaderLevel { + if h.isStaticLevel { + // The level is static and should not change during the test via arguments. + require.Len(h.t, level, 0) + return h.level + } + // The level is dynamic and should be resolved via arguments if provided, + // otherwise the default given at construction is used. Verify that no more + // than one level is provided. + require.Less(h.t, len(level), 2) + if len(level) == 0 { + return h.level + } + return level[0] +} + +// v1FlowTokensQueryStr is the query string to fetch flow tokens metrics from +// the node metrics table. It fetches all flow token metrics available in v1. +const v1FlowTokensQueryStr = ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +` + +// query runs the given SQL query against the given SQLRunner, and appends the +// output to the testdata file buffer. func (h *flowControlTestHelper) query(runner *sqlutils.SQLRunner, sql string, headers ...string) { + // NB: We update metric gauges here to ensure that periodically updated + // metrics (via the node metrics loop) are up-to-date. + for _, server := range h.tc.Servers { + require.NoError(h.t, server.GetStores().(*kvserver.Stores).VisitStores(func(s *kvserver.Store) error { + s.GetStoreConfig().KVFlowStreamTokenProvider.UpdateMetricGauges() + return nil + })) + } sql = strings.TrimSpace(sql) h.log(sql) h.buf.WriteString(fmt.Sprintf("%s\n\n", sql)) @@ -2468,22 +2405,64 @@ func (h *flowControlTestHelper) query(runner *sqlutils.SQLRunner, sql string, he tbl.Render() } +// put issues a put request for the given key at the priority specified, +// against the first server in the cluster. func (h *flowControlTestHelper) put( - ctx context.Context, key roachpb.Key, size int, pri admissionpb.WorkPriority, -) *kvpb.BatchRequest { - value := roachpb.MakeValueFromString(randutil.RandString(h.rng, size, randutil.PrintableKeyAlphabet)) - ba := &kvpb.BatchRequest{} - ba.Add(kvpb.NewPut(key, value)) - ba.AdmissionHeader.Priority = int32(pri) - ba.AdmissionHeader.Source = kvpb.AdmissionHeader_FROM_SQL - if _, pErr := h.tc.Server(0).DB().NonTransactionalSender().Send( - ctx, ba, - ); pErr != nil { - h.t.Fatal(pErr.GoError()) + ctx context.Context, key roachpb.Key, size int, pri admissionpb.WorkPriority, serverIdxs ...int, +) { + if len(serverIdxs) == 0 { + // Default to the first server if none are given. + serverIdxs = []int{0} + } + for _, serverIdx := range serverIdxs { + value := roachpb.MakeValueFromString(randutil.RandString(h.rng, size, randutil.PrintableKeyAlphabet)) + ba := &kvpb.BatchRequest{} + ba.Add(kvpb.NewPut(key, value)) + ba.AdmissionHeader.Priority = int32(pri) + ba.AdmissionHeader.Source = kvpb.AdmissionHeader_FROM_SQL + if _, pErr := h.tc.Server(serverIdx).DB().NonTransactionalSender().Send( + ctx, ba, + ); pErr != nil { + h.t.Fatal(pErr.GoError()) + } } - return ba } +// close writes the buffer to a file in the testdata directory and compares it +// against the expected output. func (h *flowControlTestHelper) close(filename string) { - echotest.Require(h.t, h.buf.String(), datapathutils.TestDataPath(h.t, "flow_control_integration", filename)) + echotest.Require(h.t, h.buf.String(), datapathutils.TestDataPath(h.t, h.testdata, filename)) +} + +func (h *flowControlTestHelper) getInspectHandlesForLevel( + serverIdx int, level kvflowcontrol.V2EnabledWhenLeaderLevel, +) kvflowcontrol.InspectHandles { + switch level { + case kvflowcontrol.V2NotEnabledWhenLeader: + return h.tc.Server(serverIdx).KVFlowHandles().(kvflowcontrol.Handles) + case kvflowcontrol.V2EnabledWhenLeaderV1Encoding, kvflowcontrol.V2EnabledWhenLeaderV2Encoding: + return kvserver.MakeStoresForRACv2(h.tc.Server(serverIdx).GetStores().(*kvserver.Stores)) + default: + h.t.Fatalf("unknown level: %v", level) + } + panic("unreachable") +} + +// enableVerboseRaftMsgLoggingForRange installs a raft handler on each node, +// which in turn enables verbose message logging. +func (h *flowControlTestHelper) enableVerboseRaftMsgLoggingForRange(rangeID roachpb.RangeID) { + for i := 0; i < len(h.tc.Servers); i++ { + si, err := h.tc.Server(i).GetStores().(*kvserver.Stores).GetStore(h.tc.Server(i).GetFirstStoreID()) + require.NoError(h.t, err) + h.tc.Servers[i].RaftTransport().(*kvserver.RaftTransport).ListenIncomingRaftMessages(si.StoreID(), + &unreliableRaftHandler{ + rangeID: rangeID, + IncomingRaftMessageHandler: si, + unreliableRaftHandlerFuncs: unreliableRaftHandlerFuncs{ + dropReq: func(req *kvserverpb.RaftMessageRequest) bool { + return false + }, + }, + }) + } } From b6b6db66177416d9d8c9f22d8bf7a7d247036a9d Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Fri, 20 Sep 2024 15:41:04 -0400 Subject: [PATCH 5/8] roachtest: port decommission/mixed-versions This commit ports the `decommission/mixed-versions` roachtest to use the `mixedversion` framework (instead of the old `newUpgradeTest` API). It also updates `acceptance/decommission-self` since both tests used shared functionality that needed to be updated. Prior to this commit, the acceptance test used the old upgrade test API even though it was not an upgrade test. Fixes: #110531 Fixes: #110530 Release note: None --- pkg/cmd/roachtest/tests/decommission.go | 2 +- pkg/cmd/roachtest/tests/decommission_self.go | 30 +- .../tests/mixed_version_decommission.go | 544 +++++++++--------- pkg/cmd/roachtest/tests/versionupgrade.go | 26 - 4 files changed, 295 insertions(+), 307 deletions(-) diff --git a/pkg/cmd/roachtest/tests/decommission.go b/pkg/cmd/roachtest/tests/decommission.go index e06902faedd3..1200f070e2b2 100644 --- a/pkg/cmd/roachtest/tests/decommission.go +++ b/pkg/cmd/roachtest/tests/decommission.go @@ -116,7 +116,7 @@ func registerDecommission(r registry.Registry) { CompatibleClouds: registry.AllExceptAWS, Suites: registry.Suites(registry.Nightly), Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { - runDecommissionMixedVersions(ctx, t, c, t.BuildVersion()) + runDecommissionMixedVersions(ctx, t, c) }, }) } diff --git a/pkg/cmd/roachtest/tests/decommission_self.go b/pkg/cmd/roachtest/tests/decommission_self.go index 369fb1252efc..fa4f95020f98 100644 --- a/pkg/cmd/roachtest/tests/decommission_self.go +++ b/pkg/cmd/roachtest/tests/decommission_self.go @@ -14,27 +14,27 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" - "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/roachprod/install" ) // runDecommissionSelf decommissions n2 through n2. This is an acceptance test. // // See https://github.com/cockroachdb/cockroach/issues/56718 func runDecommissionSelf(ctx context.Context, t test.Test, c cluster.Cluster) { - allNodes := c.All() - u := newVersionUpgradeTest(c, - uploadCockroachStep(allNodes, clusterupgrade.CurrentVersion()), - startVersion(allNodes, clusterupgrade.CurrentVersion()), - fullyDecommissionStep(2, 2, clusterupgrade.CurrentVersion()), - func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - // Stop n2 and exclude it from post-test consistency checks, - // as this node can't contact cluster any more and operations - // on it will hang. - u.c.Wipe(ctx, c.Node(2)) - }, - checkOneMembership(1, "decommissioned"), - ) + n1, n2 := 1, 2 + c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) - u.run(ctx, t) + if err := fullyDecommission(ctx, c, n2, n2, test.DefaultCockroachPath); err != nil { + t.Fatal(err) + } + + t.L().Printf("n2 decommissioned") + db := c.Conn(ctx, t.L(), n1) + defer db.Close() + + if err := newLivenessInfo(db).membershipEquals("decommissioned").eventuallyOnlyNode(n2); err != nil { + t.Fatal(err) + } } diff --git a/pkg/cmd/roachtest/tests/mixed_version_decommission.go b/pkg/cmd/roachtest/tests/mixed_version_decommission.go index 251ef30f620a..5afdec97fc0b 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_decommission.go +++ b/pkg/cmd/roachtest/tests/mixed_version_decommission.go @@ -12,328 +12,342 @@ package tests import ( "context" + gosql "database/sql" "fmt" - "strconv" + "math/rand" "time" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/mixedversion" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/install" + "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/testutils/release" - "github.com/cockroachdb/cockroach/pkg/util/httputil" - "github.com/cockroachdb/cockroach/pkg/util/retry" - "github.com/cockroachdb/cockroach/pkg/util/version" "github.com/cockroachdb/errors" ) -// runDecommissionMixedVersions runs through randomized -// decommission/recommission processes in mixed-version clusters. -func runDecommissionMixedVersions( - ctx context.Context, t test.Test, c cluster.Cluster, buildVersion *version.Version, -) { - predecessorVersionStr, err := release.LatestPredecessor(buildVersion) - if err != nil { - t.Fatal(err) - } - predecessorVersion := clusterupgrade.MustParseVersion(predecessorVersionStr) - - h := newDecommTestHelper(t, c) - - pinnedUpgrade := h.getRandNode() - t.L().Printf("pinned n%d for upgrade", pinnedUpgrade) - +func runDecommissionMixedVersions(ctx context.Context, t test.Test, c cluster.Cluster) { // NB: The suspect duration must be at least 10s, as versions 23.2 and // beyond will reset to the default of 30s if it fails validation, even if // set by a previous version. const suspectDuration = 10 * time.Second - allNodes := c.All() - u := newVersionUpgradeTest(c, - // We upload both binaries to each node, to be able to vary the binary - // used when issuing `cockroach node` subcommands. - uploadCockroachStep(allNodes, predecessorVersion), - uploadCockroachStep(allNodes, clusterupgrade.CurrentVersion()), - - startVersion(allNodes, predecessorVersion), - waitForUpgradeStep(allNodes), - preventAutoUpgradeStep(h.nodeIDs[0]), - suspectLivenessSettingsStep(h.nodeIDs[0], suspectDuration), - - preloadDataStep(pinnedUpgrade), - - // We upgrade a pinned node and one other random node of the cluster to the current version. - binaryUpgradeStep(c.Node(pinnedUpgrade), clusterupgrade.CurrentVersion()), - binaryUpgradeStep(c.Node(h.getRandNodeOtherThan(pinnedUpgrade)), clusterupgrade.CurrentVersion()), - checkAllMembership(pinnedUpgrade, "active"), - - // After upgrading, which restarts the nodes, ensure that nodes are not - // considered suspect unnecessarily. - sleepStep(2*suspectDuration), - - // Partially decommission a random node from another random node. We - // use the predecessor CLI to do so. - partialDecommissionStep(h.getRandNode(), h.getRandNode(), predecessorVersion), - checkOneDecommissioning(h.getRandNode()), - checkOneMembership(pinnedUpgrade, "decommissioning"), - - // Recommission all nodes, including the partially decommissioned - // one, from a random node. Use the predecessor CLI to do so. - recommissionAllStep(h.getRandNode(), predecessorVersion), - checkNoDecommissioning(h.getRandNode()), - checkAllMembership(pinnedUpgrade, "active"), - - // Roll back, which should to be fine because the cluster upgrade was - // not finalized. - binaryUpgradeStep(allNodes, predecessorVersion), - - // Roll all nodes forward, and finalize upgrade. - binaryUpgradeStep(allNodes, clusterupgrade.CurrentVersion()), - allowAutoUpgradeStep(1), - waitForUpgradeStep(allNodes), - - // Again ensure that nodes are not considered suspect unnecessarily. - sleepStep(2*suspectDuration), - - // Fully decommission a random node. Note that we can no longer use the - // predecessor cli, as the cluster has upgraded and won't allow connections - // from the predecessor version binary. - // - // Note also that this has to remain the last step unless we want this test to - // handle the fact that the decommissioned node will no longer be able - // to communicate with the cluster (i.e. most commands against it will fail). - // This is also why we're making sure to avoid decommissioning the pinned node - // itself, as we use it to check the membership after. - fullyDecommissionStep( - h.getRandNodeOtherThan(pinnedUpgrade), h.getRandNode(), clusterupgrade.CurrentVersion(), - ), - checkOneMembership(pinnedUpgrade, "decommissioned"), + mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All(), + // We test only upgrades from 23.2 in this test because it uses + // the `workload fixtures import` command, which is only supported + // reliably multi-tenant mode starting from that version. + mixedversion.MinimumSupportedVersion("v23.2.0"), ) + n1 := 1 + n2 := 2 + + mvt.OnStartup( + "set suspect duration", + func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { + return h.System.Exec( + rng, + "SET CLUSTER SETTING server.time_after_store_suspect = $1", + suspectDuration.String(), + ) + }) + + mvt.OnStartup( + "preload data", + func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { + node, db := h.RandomDB(rng) + cmd := `./cockroach workload fixtures import tpcc --warehouses=100 {pgurl:1}` + if err := c.RunE(ctx, option.WithNodes(c.Node(node)), cmd); err != nil { + return errors.Wrap(err, "failed to import fixtures") + } + + return errors.Wrapf( + roachtestutil.WaitFor3XReplication(ctx, l, db), + "error waiting for 3x replication", + ) + }) + + mvt.InMixedVersion( + "test decommission", + func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { + n1Version, _ := h.System.NodeVersion(n1) // safe to ignore error as n1 is part of the cluster + n2Version, _ := h.System.NodeVersion(n2) // safe to ignore error as n1 is part of the cluster + db1 := h.System.Connect(n1) + db2 := h.System.Connect(n2) + + l.Printf("checking membership via n%d (%s)", n1, n1Version) + if err := newLivenessInfo(db1).membershipNotEquals("active").eventuallyEmpty(); err != nil { + return err + } + + sleepDur := 2 * suspectDuration + l.Printf("sleeping for %s", sleepDur) + sleepCtx(ctx, sleepDur) + + // Run self-decommission on some runs. + from := n2 + fromVersion := n2Version + if rng.Float64() < 0.5 { + from = n1 + fromVersion = n1Version + } - u.run(ctx, t) + l.Printf("partially decommissioning n1 (%s) from n%d (%s)", n1Version, from, fromVersion) + if err := partialDecommission(ctx, c, n1, from, clusterupgrade.CockroachPathForVersion(t, fromVersion)); err != nil { + return err + } + + l.Printf("verifying n1 is decommissioning via n2 (%s)", n2Version) + err := newLivenessInfo(db2). + membershipEquals("decommissioning"). + isDecommissioning(). + eventuallyOnlyNode(n1) + if err != nil { + return err + } + + l.Printf("recommissioning all nodes via n1 (%s)", n1Version) + if err := recommissionNodes(ctx, c, c.All(), n1, clusterupgrade.CockroachPathForVersion(t, n1Version)); err != nil { + return err + } + + l.Printf("verifying no node is decommissioning") + if err := newLivenessInfo(db1).isDecommissioning().eventuallyEmpty(); err != nil { + return err + } + + l.Printf("verifying all nodes are active") + if err := newLivenessInfo(db1).membershipNotEquals("active").eventuallyEmpty(); err != nil { + return err + } + + return nil + }) + + mvt.AfterUpgradeFinalized( + "fully decommission on last upgrade", + func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { + sleepDur := 2 * suspectDuration + l.Printf("sleeping for %s", sleepDur) + sleepCtx(ctx, sleepDur) + + if h.Context().ToVersion.IsCurrent() { + l.Printf("fully decommissioning n1 via n2") + + n1Version, _ := h.System.NodeVersion(n1) // safe to ignore error as n1 is part of the cluster + return fullyDecommission(ctx, c, n1, n2, clusterupgrade.CockroachPathForVersion(t, n1Version)) + } else { + l.Printf("skipping -- still more upgrades to go through") + return nil + } + }) + + mvt.Run() } -// suspectLivenessSettingsStep sets the duration a node is considered "suspect" -// after it becomes unavailable. -func suspectLivenessSettingsStep(target int, suspectDuration time.Duration) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - db := u.conn(ctx, t, target) - _, err := db.ExecContext(ctx, `SET CLUSTER SETTING server.time_after_store_suspect = $1`, suspectDuration.String()) - if err != nil { - t.Fatal(err) - } +// partialDecommission runs `cockroach node decommission --wait=none` +// from a given node, targeting another. It uses the specified binary +// to run the command. +func partialDecommission( + ctx context.Context, c cluster.Cluster, target, from int, cockroachPath string, +) error { + cmd := roachtestutil.NewCommand("%s node decommission %d", cockroachPath, target) + if target == from { + cmd = roachtestutil.NewCommand("%s node decommission", cockroachPath).Option("self") } + + cmd = cmd. + WithEqualsSyntax(). + Flag("wait", "none"). + Flag("port", fmt.Sprintf("{pgport:%d}", from)). + Flag("certs-dir", install.CockroachNodeCertsDir) + + return c.RunE(ctx, option.WithNodes(c.Node(from)), cmd.String()) } -// preloadDataStep load data into cluster to ensure we have a large enough -// number of replicas to move on decommissioning. -func preloadDataStep(target int) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - // Load data into cluster to ensure we have a large enough number of replicas - // to move on decommissioning. - c := u.c - c.Run(ctx, option.WithNodes(c.Node(target)), - `./cockroach workload fixtures import tpcc --warehouses=100 {pgurl:1}`) - db := c.Conn(ctx, t.L(), target) - defer db.Close() - if err := roachtestutil.WaitFor3XReplication(ctx, t.L(), db); err != nil { - t.Fatal(err) - } - } +// recommissionNodes runs `cockroach node recommission` from a given +// node, targeting the `nodes` in the cluster. It uses the specified +// binary to run the command. +func recommissionNodes( + ctx context.Context, + c cluster.Cluster, + nodes option.NodeListOption, + from int, + cockroachPath string, +) error { + cmd := roachtestutil.NewCommand("%s node recommission %s", cockroachPath, nodes.NodeIDsString()). + Flag("port", fmt.Sprintf("{pgport:%d}", from)). + Flag("certs-dir", install.CockroachNodeCertsDir). + String() + + return c.RunE(ctx, option.WithNodes(c.Node(from)), cmd) } -// partialDecommissionStep runs `cockroach node decommission --wait=none` from a -// given node, targeting another. It uses the specified binary version to run -// the command. -func partialDecommissionStep(target, from int, binaryVersion *clusterupgrade.Version) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - c := u.c - c.Run(ctx, option.WithNodes(c.Node(from)), clusterupgrade.CockroachPathForVersion(t, binaryVersion), "node", "decommission", - "--wait=none", strconv.Itoa(target), "--port", fmt.Sprintf("{pgport:%d}", from), fmt.Sprintf("--certs-dir=%s", install.CockroachNodeCertsDir)) - } +// fullyDecommission is like partialDecommission, except it uses +// `--wait=all`. +func fullyDecommission( + ctx context.Context, c cluster.Cluster, target, from int, cockroachPath string, +) error { + cmd := roachtestutil.NewCommand("%s node decommission %d", cockroachPath, target). + WithEqualsSyntax(). + Flag("wait", "all"). + Flag("port", fmt.Sprintf("{pgport:%d}", from)). + Flag("certs-dir", install.CockroachNodeCertsDir). + String() + + return c.RunE(ctx, option.WithNodes(c.Node(from)), cmd) } -// recommissionAllStep runs `cockroach node recommission` from a given node, -// targeting all nodes in the cluster. It uses the specified binary version to -// run the command. -func recommissionAllStep(from int, binaryVersion *clusterupgrade.Version) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - c := u.c - c.Run(ctx, option.WithNodes(c.Node(from)), clusterupgrade.CockroachPathForVersion(t, binaryVersion), "node", "recommission", - c.All().NodeIDsString(), "--port", fmt.Sprintf("{pgport:%d}", from), fmt.Sprintf("--certs-dir=%s", install.CockroachNodeCertsDir)) - } +// gossipLiveness is a helper struct that allows callers to verify +// that the liveness data (`crdb_internal.gossip_liveness`) eventually +// reaches a desired state. +// +// Typical usage: +// +// newLivenessInfo(db).membershipEquals("decommissioned").eventuallyOnlyNode(n1) +// +// In this example, we assert that eventually only node `n1` has its +// membership status equal to `decommissioned`. This could be used +// after a `decommission` command. +type gossipLiveness struct { + node int + decommissioning bool + membership string } -// fullyDecommissionStep is like partialDecommissionStep, except it uses -// `--wait=all`. -func fullyDecommissionStep(target, from int, binaryVersion *clusterupgrade.Version) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - c := u.c - c.Run(ctx, option.WithNodes(c.Node(from)), clusterupgrade.CockroachPathForVersion(t, binaryVersion), "node", "decommission", - "--wait=all", strconv.Itoa(target), "--port={pgport:1}", fmt.Sprintf("--certs-dir=%s", install.CockroachNodeCertsDir)) - - // If we are decommissioning a target node from the same node, the drain - // step will be skipped. In this case, we should not consider the step done - // until the health check for the node returns non-200 OK. - // TODO(sarkesian): This could be removed after 23.2, as in these versions - // the "decommissioned" state is considered in the health check. - if target == from { - t.L().Printf("waiting for n%d to fail health check after decommission") - var healthCheckURL string - if addrs, err := c.ExternalAdminUIAddr(ctx, t.L(), c.Node(target)); err != nil { - t.Fatalf("failed to get admin ui addresses: %v", err) - } else { - healthCheckURL = fmt.Sprintf(`http://%s/health?ready=1`, addrs[0]) - } +type livenessInfo struct { + db *gosql.DB + filters []func(gossipLiveness) bool + liveness []gossipLiveness +} - if err := retry.ForDuration(testutils.DefaultSucceedsSoonDuration, func() error { - resp, err := httputil.Get(ctx, healthCheckURL) - if err != nil { - return errors.Wrapf(err, "failed to get n%d /health?ready=1 HTTP endpoint", target) - } - if resp.StatusCode == 200 { - return errors.Errorf("n%d /health?ready=1 status=%d %s "+ - "(expected 503 service unavailable after decommission)", - target, resp.StatusCode, resp.Status) - } +func newLivenessInfo(db *gosql.DB) *livenessInfo { + return &livenessInfo{db: db} +} - t.L().Printf("n%d /health?ready=1 status=%d %s", target, resp.StatusCode, resp.Status) - return nil - }); err != nil { - t.Fatal(err) - } - } - } +// addFilter adds a filter to be applied to the liveness records when +// checking for a property. Some filters are already predefined, such +// as `membershipEquals`, `isDecommissioning`, etc. +func (l *livenessInfo) addFilter(f func(gossipLiveness) bool) { + l.filters = append(l.filters, f) } -// checkOneDecommissioning checks against the `decommissioning` column in -// crdb_internal.gossip_liveness, asserting that only one node is marked as -// decommissioning. -func checkOneDecommissioning(from int) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - // We use a retry block here (and elsewhere) because we're consulting - // crdb_internal.gossip_liveness, and need to make allowances for gossip - // propagation delays. - if err := retry.ForDuration(testutils.DefaultSucceedsSoonDuration, func() error { - db := u.conn(ctx, t, from) - var count int - if err := db.QueryRow( - `select count(*) from crdb_internal.gossip_liveness where decommissioning = true;`).Scan(&count); err != nil { - t.Fatal(err) - } +// membershipEquals adds a filter so that we only look at records +// where the `membership` column matches the value passed. +func (l *livenessInfo) membershipEquals(membership string) *livenessInfo { + l.addFilter(func(rec gossipLiveness) bool { + return rec.membership == membership + }) - if count != 1 { - return errors.Newf("expected to find 1 node with decommissioning=true, found %d", count) - } + return l +} - var nodeID int - if err := db.QueryRow( - `select node_id from crdb_internal.gossip_liveness where decommissioning = true;`).Scan(&nodeID); err != nil { - t.Fatal(err) - } - t.L().Printf("n%d decommissioning=true", nodeID) - return nil - }); err != nil { - t.Fatal(err) +// membershipNotEquals adds a filter so that we only look at records +// where the `membership` column is *different* from the value passed. +func (l *livenessInfo) membershipNotEquals(membership string) *livenessInfo { + l.addFilter(func(rec gossipLiveness) bool { + return rec.membership != membership + }) + + return l +} + +// isDecommissioning adds a filter so that we only look at records +// where the `decommissioning` column is `true`. +func (l *livenessInfo) isDecommissioning() *livenessInfo { + l.addFilter(func(rec gossipLiveness) bool { + return rec.decommissioning + }) + + return l +} + +// eventuallyEmpty asserts that, eventually, the number of records in +// `crdb_internal.gossip_liveness` that match the filters used is zero. +func (l *livenessInfo) eventuallyEmpty() error { + return l.eventually(func(records []gossipLiveness) error { + if len(records) > 0 { + return errors.Newf("expected no matches, found: %#v", records) } - } + + return nil + }) } -// checkNoDecommissioning checks against the `decommissioning` column in -// crdb_internal.gossip_liveness, asserting that only no nodes are marked as -// decommissioning. -func checkNoDecommissioning(from int) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - if err := retry.ForDuration(testutils.DefaultSucceedsSoonDuration, func() error { - db := u.conn(ctx, t, from) - var count int - if err := db.QueryRow( - `select count(*) from crdb_internal.gossip_liveness where decommissioning = true;`).Scan(&count); err != nil { - t.Fatal(err) - } +// eventuallyOnlyNode asserts that, eventually, only the liveness +// record for the given `node` matches the filters used. +func (l *livenessInfo) eventuallyOnlyNode(node int) error { + return l.eventually(func(records []gossipLiveness) error { + if len(records) != 1 { + return errors.Newf("expected one liveness record, found: %#v", records) + } - if count != 0 { - return errors.Newf("expected to find 0 nodes with decommissioning=false, found %d", count) - } - return nil - }); err != nil { - t.Fatal(err) + if records[0].node != node { + return errors.Newf("expected to match n%d, found n%d", node, records[0].node) } - } + + return nil + }) } -// checkOneMembership checks against the `membership` column in -// crdb_internal.gossip_liveness, asserting that only one node is marked with -// the specified membership status. -func checkOneMembership(from int, membership string) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - if err := retry.ForDuration(testutils.DefaultSucceedsSoonDuration, func() error { - db := u.conn(ctx, t, from) - var count int - if err := db.QueryRow( - `select count(*) from crdb_internal.gossip_liveness where membership = $1;`, membership).Scan(&count); err != nil { - t.Fatal(err) - } +// eventually asserts that, eventually, the `predicate` given returns +// true when called with the records that match all the filters used. +func (l *livenessInfo) eventually(predicate func([]gossipLiveness) error) error { + return testutils.SucceedsSoonError(func() error { + if err := l.refreshLiveness(); err != nil { + return errors.Wrap(err, "refreshing liveness info") + } - if count != 1 { - return errors.Newf("expected to find 1 node with membership=%s, found %d", membership, count) + var filtered []gossipLiveness + for _, record := range l.liveness { + match := true + for _, filter := range l.filters { + if !filter(record) { + match = false + break + } } - var nodeID int - if err := db.QueryRow( - `select node_id from crdb_internal.gossip_liveness where decommissioning = true;`).Scan(&nodeID); err != nil { - t.Fatal(err) + if match { + filtered = append(filtered, record) } - t.L().Printf("n%d membership=%s", nodeID, membership) - return nil - }); err != nil { - t.Fatal(err) } - } + + return predicate(filtered) + }) } -// checkAllMembership checks against the `membership` column in -// crdb_internal.gossip_liveness, asserting that all nodes are marked with -// the specified membership status. -func checkAllMembership(from int, membership string) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - if err := retry.ForDuration(testutils.DefaultSucceedsSoonDuration, func() error { - db := u.conn(ctx, t, from) - var count int - if err := db.QueryRow( - `select count(*) from crdb_internal.gossip_liveness where membership != $1;`, membership).Scan(&count); err != nil { - t.Fatal(err) - } +func (l *livenessInfo) refreshLiveness() error { + rows, err := l.db.Query( + "SELECT node_id, decommissioning, membership FROM crdb_internal.gossip_liveness", + ) + if err != nil { + return err + } - if count != 0 { - return errors.Newf("expected to find 0 nodes with membership!=%s, found %d", membership, count) - } - return nil - }); err != nil { - t.Fatal(err) + var records []gossipLiveness + for rows.Next() { + var record gossipLiveness + if err := rows.Scan(&record.node, &record.decommissioning, &record.membership); err != nil { + return errors.Wrap(err, "failed to scan liveness row") } + + records = append(records, record) } -} -// uploadCockroachStep uploads the specified cockroach binary version on the specified -// nodes. -func uploadCockroachStep(nodes option.NodeListOption, version *clusterupgrade.Version) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - uploadCockroach(ctx, t, u.c, nodes, version) + if err := rows.Err(); err != nil { + return errors.Wrap(err, "failed to read liveness rows") } + + l.liveness = records + return nil } -// startVersion starts the specified cockroach binary version on the specified -// nodes. -func startVersion(nodes option.NodeListOption, version *clusterupgrade.Version) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - settings := install.MakeClusterSettings(install.BinaryOption( - clusterupgrade.CockroachPathForVersion(t, version), - )) - startOpts := option.DefaultStartOpts() - u.c.Start(ctx, t.L(), startOpts, settings, nodes) +func sleepCtx(ctx context.Context, duration time.Duration) { + select { + case <-ctx.Done(): + case <-time.After(duration): } } diff --git a/pkg/cmd/roachtest/tests/versionupgrade.go b/pkg/cmd/roachtest/tests/versionupgrade.go index 0e39eb51a6b8..ae0650aef359 100644 --- a/pkg/cmd/roachtest/tests/versionupgrade.go +++ b/pkg/cmd/roachtest/tests/versionupgrade.go @@ -255,26 +255,6 @@ func binaryUpgradeStep( } } -func preventAutoUpgradeStep(node int) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - db := u.conn(ctx, t, node) - _, err := db.ExecContext(ctx, `SET CLUSTER SETTING cluster.preserve_downgrade_option = $1`, u.binaryVersion(ctx, t, node).String()) - if err != nil { - t.Fatal(err) - } - } -} - -func allowAutoUpgradeStep(node int) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - db := u.conn(ctx, t, node) - _, err := db.ExecContext(ctx, `RESET CLUSTER SETTING cluster.preserve_downgrade_option`) - if err != nil { - t.Fatal(err) - } - } -} - // NB: this is intentionally kept separate from binaryUpgradeStep because we run // feature tests between the steps, and we want to expose them (at least // heuristically) to the real-world situation in which some nodes have already @@ -373,9 +353,3 @@ done `) }).run(ctx, t) } - -func sleepStep(d time.Duration) versionStep { - return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - time.Sleep(d) - } -} From e166484f2cb4a3e1f86e3e2d368e8bb3b09cc73c Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Thu, 19 Sep 2024 20:13:47 -0400 Subject: [PATCH 6/8] kvserver: add rac2 v1 integration tests Introduce several tests in `flow_control_integration_test.go`, mirroring the existing tests but applied to the replication flow control v2 machinery. The tests largely follow an identical pattern to the existing v1 tests, swapping in rac2 metrics and vtables. The following tests are added: ``` TestFlowControlBasicV2 TestFlowControlRangeSplitMergeV2 TestFlowControlBlockedAdmissionV2 TestFlowControlAdmissionPostSplitMergeV2 TestFlowControlCrashedNodeV2 TestFlowControlRaftSnapshotV2 TestFlowControlRaftMembershipV2 TestFlowControlRaftMembershipRemoveSelfV2 TestFlowControlClassPrioritizationV2 TestFlowControlQuiescedRangeV2 TestFlowControlUnquiescedRangeV2 TestFlowControlTransferLeaseV2 TestFlowControlLeaderNotLeaseholderV2 TestFlowControlGranterAdmitOneByOneV2 ``` These tests all have at least two variants: ``` V2EnabledWhenLeaderV1Encoding V2EnabledWhenLeaderV2Encoding ``` When `V2EnabledWhenLeaderV1Encoding` is run, the tests use a different testdata file, which has a `_v1_encoding` suffix. A separate file is necessary because when the protocol enablement level is `V2EnabledWhenLeaderV1Encoding`, all entries which are subject to admission control are encoded as `raftpb.LowPri`, regardless of their original priority, as we don't want to pay the cost to deserialize the raft admission meta. The v1 encoding variants retain the same comments as the v2 encoding, however any comments referring to regular tokens should be interpreted as elastic tokens instead, due to the above. Two v1 tests are not ported over to v2: ``` TestFlowControlRaftTransportBreak TestFlowControlRaftTransportCulled ``` These omitted tests behave identically to `TestFlowControlCrashedNodeV2` as rac2 is less tightly coupled to the raft transport, instead operating on replication states (e.g., `StateProbe`, `StateReplicate`). Resolves: cockroachdb#130187 Release note: None --- .../kvserver/flow_control_integration_test.go | 1736 +++++++++++++++++ .../admission_post_split_merge | 162 ++ .../admission_post_split_merge_v1_encoding | 169 ++ .../flow_control_integration_v2/basic | 96 + .../basic_v1_encoding | 103 + .../blocked_admission | 105 + .../blocked_admission_v1_encoding | 112 ++ .../class_prioritization | 103 + .../class_prioritization_v1_encoding | 110 ++ .../flow_control_integration_v2/crashed_node | 90 + .../crashed_node_v1_encoding | 97 + .../granter_admit_one_by_one | 82 + .../granter_admit_one_by_one_v1_encoding | 89 + .../leader_not_leaseholder | 160 ++ .../leader_not_leaseholder_v1_encoding | 167 ++ .../quiesced_range | 108 + .../quiesced_range_v1_encoding | 115 ++ .../raft_membership | 140 ++ .../raft_membership_remove_self | 127 ++ .../raft_membership_remove_self_v1_encoding | 134 ++ .../raft_membership_v1_encoding | 147 ++ .../flow_control_integration_v2/raft_snapshot | 225 +++ .../raft_snapshot_v1_encoding | 232 +++ .../flow_control_integration_v2/split_merge | 125 ++ .../split_merge_v1_encoding | 132 ++ .../transfer_lease | 70 + .../transfer_lease_v1_encoding | 77 + .../unquiesced_range | 111 ++ .../unquiesced_range_v1_encoding | 118 ++ 29 files changed, 5242 insertions(+) create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/admission_post_split_merge create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/admission_post_split_merge_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/basic create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/basic_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/blocked_admission create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/blocked_admission_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/class_prioritization create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/class_prioritization_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/crashed_node create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/crashed_node_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/granter_admit_one_by_one create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/granter_admit_one_by_one_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/leader_not_leaseholder create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/leader_not_leaseholder_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/quiesced_range create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/quiesced_range_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_remove_self create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_remove_self_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_snapshot create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_snapshot_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/split_merge create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/split_merge_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/transfer_lease create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/transfer_lease_v1_encoding create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/unquiesced_range create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/unquiesced_range_v1_encoding diff --git a/pkg/kv/kvserver/flow_control_integration_test.go b/pkg/kv/kvserver/flow_control_integration_test.go index 1c21b116cd01..1fb19b257ad7 100644 --- a/pkg/kv/kvserver/flow_control_integration_test.go +++ b/pkg/kv/kvserver/flow_control_integration_test.go @@ -13,6 +13,7 @@ package kvserver_test import ( "context" "fmt" + "math" "math/rand" "strconv" "strings" @@ -2148,6 +2149,1701 @@ func TestFlowControlGranterAdmitOneByOne(t *testing.T) { h.query(n1, v1FlowTokensQueryStr) } +// TestFlowControlBasicV2 runs a basic end-to-end test of the v2 kvflowcontrol +// machinery, replicating + admitting a single 1MiB regular write. The vmodule +// flags for running these tests with full logging are: +// +// --vmodule='replica_raft=1,replica_proposal_buf=1,raft_transport=2, +// kvadmission=1,work_queue=1,replica_flow_control=1, +// tracker=1,client_raft_helpers_test=1,range_controller=2, +// token_counter=2,token_tracker=2,processor=2' +func TestFlowControlBasicV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + testutils.RunTrueAndFalse(t, "always-enqueue", func(t *testing.T, alwaysEnqueue bool) { + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: alwaysEnqueue, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + // Setup the test state with 3 voters, one on each of the three + // node/stores. + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "basic")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.enableVerboseRaftMsgLoggingForRange(desc.RangeID) + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- Flow token metrics, before issuing the regular 1MiB replicated write.`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Issuing + admitting a regular 1MiB, triply replicated write...)`) + h.log("sending put request") + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + h.log("sent put request") + + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) + h.comment(` +-- Stream counts as seen by n1 post-write. We should see three {regular,elastic} +-- streams given there are three nodes and we're using a replication factor of +-- three. +`) + h.query(n1, ` + SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%stream%' +ORDER BY name ASC; +`) + + h.comment(`-- Another view of the stream count, using /inspectz-backed vtables.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(` +-- Flow token metrics from n1 after issuing the regular 1MiB replicated write, +-- and it being admitted on n1, n2 and n3. We should see 3*1MiB = 3MiB of +-- {regular,elastic} tokens deducted and returned, and {8*3=24MiB,16*3=48MiB} of +-- {regular,elastic} tokens available. Everything should be accounted for. +`) + h.query(n1, v2FlowTokensQueryStr) + }) + }) +} + +// TestFlowControlRangeSplitMergeV2 walks through what happens to flow tokens +// when a range splits/merges. +func TestFlowControlRangeSplitMergeV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "split_merge")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + h.log("sending put request to pre-split range") + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + h.log("sent put request to pre-split range") + + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) + h.comment(` +-- Flow token metrics from n1 after issuing + admitting the regular 1MiB 3x +-- replicated write to the pre-split range. There should be 3MiB of +-- {regular,elastic} tokens {deducted,returned}. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Splitting range.)`) + left, right := tc.SplitRangeOrFatal(t, k.Next()) + h.waitForConnectedStreams(ctx, right.RangeID, 3, 0 /* serverIdx */) + + h.log("sending 2MiB put request to post-split LHS") + h.put(ctx, k, 2<<20 /* 2MiB */, admissionpb.NormalPri) + h.log("sent 2MiB put request to post-split LHS") + + h.log("sending 3MiB put request to post-split RHS") + h.put(ctx, roachpb.Key(right.StartKey), 3<<20 /* 3MiB */, admissionpb.NormalPri) + h.log("sent 3MiB put request to post-split RHS") + + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) + h.comment(` +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- {deducted,returned}, which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 3MiB+15MiB=18MiB now. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- Observe the newly split off replica, with its own three streams.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Merging ranges.)`) + merged := tc.MergeRangesOrFatal(t, left.StartKey.AsRawKey()) + + h.log("sending 4MiB put request to post-merge range") + h.put(ctx, roachpb.Key(merged.StartKey), 4<<20 /* 4MiB */, admissionpb.NormalPri) + h.log("sent 4MiB put request to post-merged range") + + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) + h.comment(` +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens {deducted,returned}, +-- which comes from 4MiB*3=12MiB. So we stand at 18MiB+12MiB=30MiB now. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- Observe only the merged replica with its own three streams.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + }) +} + +// TestFlowControlBlockedAdmissionV2 tests token tracking behavior by explicitly +// blocking below-raft admission. +func TestFlowControlBlockedAdmissionV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "blocked_admission")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.enableVerboseRaftMsgLoggingForRange(desc.RangeID) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Issuing 5 regular 1MiB, 3x replicated write that's not admitted.)`) + h.log("sending put requests") + for i := 0; i < 5; i++ { + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + } + h.log("sent put requests") + + h.comment(` +-- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes +-- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- Observe the total tracked tokens per-stream on n1.`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- Observe the individual tracked tokens per-stream on the scratch range.`) + h.query(n1, ` + SELECT range_id, store_id, priority, crdb_internal.humanize_bytes(tokens::INT8) + FROM crdb_internal.kv_flow_token_deductions_v2 +`, "range_id", "store_id", "priority", "tokens") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) // wait for admission + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. We see 15MiB returns of +-- {regular,elastic} tokens, and the available capacities going back to what +-- they were. +`) + h.query(n1, v2FlowTokensQueryStr) + }) +} + +// TestFlowControlAdmissionPostSplitMergeV2 walks through what happens with flow +// tokens when a range after undergoes splits/merges. It does this by blocking +// and later unblocking below-raft admission, verifying: +// - tokens for the RHS are released at the post-merge subsuming leaseholder, +// - admission for the RHS post-merge does not cause a double return of tokens, +// - admission for the LHS can happen post-merge, +// - admission for the LHS and RHS can happen post-split. +func TestFlowControlAdmissionPostSplitMergeV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + OverrideTokenDeduction: func(tokens kvflowcontrol.Tokens) kvflowcontrol.Tokens { + // This test sends several puts, with each put potentially + // diverging by a few bytes between runs, in aggregate this + // can accumulate to enough tokens to produce a diff in + // metrics. Round the token deductions to the nearest MiB avoid + // this. + return kvflowcontrol.Tokens( + int64(math.Round(float64(tokens)/float64(1<<20))) * 1 << 20) + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "admission_post_split_merge")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.log("sending put request to pre-split range") + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + h.put(ctx, k.Next(), 1<<20 /* 1MiB */, admissionpb.NormalPri) + h.log("sent put request to pre-split range") + + h.comment(` +-- Flow token metrics from n1 after issuing a regular 2*1MiB 3x replicated write +-- that are yet to get admitted. We see 2*3*1MiB=6MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. The 2*1MiB writes +-- happened on what is soon going to be the LHS and RHS of a range being split. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Splitting range.)`) + left, right := tc.SplitRangeOrFatal(t, k.Next()) + h.waitForConnectedStreams(ctx, right.RangeID, 3, 0 /* serverIdx */) + + h.log("sending 2MiB put request to post-split LHS") + h.put(ctx, k, 2<<20 /* 2MiB */, admissionpb.NormalPri) + h.log("sent 2MiB put request to post-split LHS") + + h.log("sending 3MiB put request to post-split RHS") + h.put(ctx, roachpb.Key(right.StartKey), 3<<20 /* 3MiB */, admissionpb.NormalPri) + h.log("sent 3MiB put request to post-split RHS") + + h.comment(` +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- deducted which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 6MiB+15MiB=21MiB now. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- Observe the newly split off replica, with its own three streams.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Merging ranges.)`) + merged := tc.MergeRangesOrFatal(t, left.StartKey.AsRawKey()) + + h.log("sending 4MiB put request to post-merge range") + h.put(ctx, roachpb.Key(merged.StartKey), 4<<20 /* 4MiB */, admissionpb.NormalPri) + h.log("sent 4MiB put request to post-merged range") + + h.comment(` +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens deducted which comes +-- from 4MiB*3=12MiB. So we stand at 21MiB+12MiB=33MiB tokens deducted now. The +-- RHS of the range is gone now, and the previously 3*3MiB=9MiB of tokens +-- deducted for it are released at the subsuming LHS leaseholder. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- Observe only the merged replica with its own three streams.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) // wait for admission + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. We see all outstanding +-- {regular,elastic} tokens returned, including those from: +-- - the LHS before the merge, and +-- - the LHS and RHS before the original split. +`) + h.query(n1, v2FlowTokensQueryStr) + }) +} + +// TestFlowControlCrashedNodeV2 tests flow token behavior in the presence of +// crashed nodes. +func TestFlowControlCrashedNodeV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + kvserver.ExpirationLeasesOnly.Override(ctx, &settings.SV, true) + tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands. + RaftElectionTimeoutTicks: 1000000, + // Reduce the RangeLeaseDuration to speeds up failure detection + // below. + RangeLeaseDuration: time.Second, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return true + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "crashed_node")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) + h.waitForConnectedStreams(ctx, desc.RangeID, 2, 0 /* serverIdx */) + + h.comment(`-- (Issuing regular 5x1MiB, 2x replicated writes that are not admitted.)`) + h.log("sending put requests") + for i := 0; i < 5; i++ { + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + } + h.log("sent put requests") + + h.comment(` +-- Flow token metrics from n1 after issuing 5 regular 1MiB 2x replicated writes +-- that are yet to get admitted. We see 5*1MiB*2=10MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + h.comment(`-- Observe the per-stream tracked tokens on n1, before n2 is crashed.`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Crashing n2)`) + tc.StopServer(1) + h.waitForConnectedStreams(ctx, desc.RangeID, 1, 0 /* serverIdx */) + + h.comment(` +-- Observe the per-stream tracked tokens on n1, after n2 crashed. We're no +-- longer tracking the 5MiB held by n2. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Flow token metrics from n1 after n2 crashed. Observe that we've returned the +-- 5MiB previously held by n2. +`) + h.query(n1, v2FlowTokensQueryStr) + }) +} + +// TestFlowControlRaftSnapshotV2 tests flow token behavior when one replica +// needs to be caught up via raft snapshot. +func TestFlowControlRaftSnapshotV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + const numServers int = 5 + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + stickyServerArgs := make(map[int]base.TestServerArgs) + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + var bypassReplicaUnreachable atomic.Bool + bypassReplicaUnreachable.Store(false) + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + for i := 0; i < numServers; i++ { + stickyServerArgs[i] = base.TestServerArgs{ + Settings: settings, + StoreSpecs: []base.StoreSpec{ + { + InMemory: true, + StickyVFSID: strconv.FormatInt(int64(i), 10), + }, + }, + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + StickyVFSRegistry: fs.NewStickyRegistry(), + }, + Store: &kvserver.StoreTestingKnobs{ + RaftReportUnreachableBypass: func(_ roachpb.ReplicaID) bool { + // This test is going to crash nodes, then truncate the raft log + // and assert that tokens are returned upon an replica entering + // StateSnapshot. To avoid the stopped replicas entering + // StateProbe returning tokens, we disable reporting a replica + // as unreachable while nodes are down. + return bypassReplicaUnreachable.Load() + }, + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + OverrideTokenDeduction: func(_ kvflowcontrol.Tokens) kvflowcontrol.Tokens { + // This test makes use of (small) increment + // requests, but wants to see large token + // deductions/returns. + return kvflowcontrol.Tokens(1 << 20 /* 1MiB */) + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + RaftTransport: &kvserver.RaftTransportTestingKnobs{ + OverrideIdleTimeout: func() time.Duration { + // Effectively disable token returns due to underlying + // raft transport streams disconnecting due to + // inactivity. + return time.Hour + }, + }, + }, + } + } + + tc := testcluster.StartTestCluster(t, numServers, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: stickyServerArgs, + }) + defer tc.Stopper().Stop(ctx) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "raft_snapshot")) + + store := tc.GetFirstStoreFromServer(t, 0) + + incA := int64(5) + incB := int64(7) + incAB := incA + incB + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + tc.AddVotersOrFatal(t, k, tc.Targets(3, 4)...) + repl := store.LookupReplica(roachpb.RKey(k)) + require.NotNil(t, repl) + h.waitForConnectedStreams(ctx, repl.RangeID, 5, 0 /* serverIdx */) + + // Set up a key to replicate across the cluster. We're going to modify this + // key and truncate the raft logs from that command after killing one of the + // nodes to check that it gets the new value after it comes up. + incArgs := incrementArgs(k, incA) + if _, err := kv.SendWrappedWithAdmission(ctx, tc.Server(0).DB().NonTransactionalSender(), kvpb.Header{}, kvpb.AdmissionHeader{ + Priority: int32(admissionpb.HighPri), + Source: kvpb.AdmissionHeader_FROM_SQL, + }, incArgs); err != nil { + t.Fatal(err) + } + + h.comment(` +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. Since this test is ignoring crashed nodes for token +-- deduction purposes, we see a deduction of 5MiB {regular,elastic} tokens. + `) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- Observe the total tracked tokens per-stream on n1. 1MiB is tracked for n1-n5. + `) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + `, "range_id", "store_id", "total_tracked_tokens") + + tc.WaitForValues(t, k, []int64{incA, incA, incA, incA, incA}) + + h.comment(` +-- (Killing n2 and n3, but preventing their tokens from being returned + +-- artificially allowing tokens to get deducted.)`) + + // Kill stores 1 + 2, increment the key on the other stores and truncate + // their logs to make sure that when store 1 + 2 comes back up they will + // require a snapshot from Raft. + // + // Also prevent replicas on the killed nodes from being marked as + // unreachable, in order to prevent them from returning tokens via + // entering StateProbe, before we're able to truncate the log and assert + // on the snapshot behavior. + bypassReplicaUnreachable.Store(true) + tc.StopServer(1) + tc.StopServer(2) + + h.comment(` +-- Observe the total tracked tokens per-stream on n1. 1MiB is (still) tracked +-- for n1-n5, because they are not in StateSnapshot yet and have likely been +-- in StateProbe for less than the close timer. + `) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + `, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- (Issuing another 1MiB of 5x replicated writes while n2 and n3 are down and +-- below-raft admission is paused.) +`) + incArgs = incrementArgs(k, incB) + if _, err := kv.SendWrappedWithAdmission(ctx, tc.Server(0).DB().NonTransactionalSender(), kvpb.Header{}, kvpb.AdmissionHeader{ + Priority: int32(admissionpb.HighPri), + Source: kvpb.AdmissionHeader_FROM_SQL, + }, incArgs); err != nil { + t.Fatal(err) + } + + h.comment(` +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. We'll have deducted another 5*1MiB=5MiB worth of tokens. + `) + h.query(n1, v2FlowTokensQueryStr) + h.comment(` +-- Observe the total tracked tokens per-stream on n1. 2MiB is tracked for n1-n5; +-- see last comment for an explanation why we're still deducting for n2, n3. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + `, "range_id", "store_id", "total_tracked_tokens") + + tc.WaitForValues(t, k, []int64{incAB, 0 /* stopped */, 0 /* stopped */, incAB, incAB}) + + index := repl.GetLastIndex() + h.comment(`-- (Truncating raft log.)`) + + // Truncate the log at index+1 (log entries < N are removed, so this + // includes the increment). + truncArgs := truncateLogArgs(index+1, repl.GetRangeID()) + if _, err := kv.SendWrappedWithAdmission(ctx, tc.Server(0).DB().NonTransactionalSender(), kvpb.Header{}, kvpb.AdmissionHeader{ + Priority: int32(admissionpb.HighPri), + Source: kvpb.AdmissionHeader_FROM_SQL, + }, truncArgs); err != nil { + t.Fatal(err) + } + + h.comment(`-- (Restarting n2 and n3.)`) + require.NoError(t, tc.RestartServer(1)) + require.NoError(t, tc.RestartServer(2)) + bypassReplicaUnreachable.Store(false) + + tc.WaitForValues(t, k, []int64{incAB, incAB, incAB, incAB, incAB}) + + h.comment(` +-- Flow token metrics from n1 after restarting n2 and n3. We've returned the +-- 2MiB previously held by those nodes (2MiB each). We're reacting to it's raft +-- progress state, noting that since we've truncated our log, we need to catch +-- it up via snapshot. So we release all held tokens. + `) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- Observe the total tracked tokens per-stream on n1. There's nothing tracked +-- for n2 and n3 anymore. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + WHERE total_tracked_tokens > 0 +`, "range_id", "store_id", "total_tracked_tokens") + + h.waitForConnectedStreams(ctx, repl.RangeID, 5, 0 /* serverIdx */) + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + + h.waitForAllTokensReturned(ctx, 5, 0 /* serverIdx */) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. We see the remaining +-- 6MiB of {regular,elastic} tokens returned. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- Observe the total tracked tokens per-stream on n1; there should be nothing. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- Another view of tokens, using /inspectz-backed vtables.`) + h.query(n1, ` +SELECT store_id, + crdb_internal.humanize_bytes(available_eval_regular_tokens), + crdb_internal.humanize_bytes(available_eval_elastic_tokens) + FROM crdb_internal.kv_flow_controller_v2 + ORDER BY store_id ASC; +`, "range_id", "eval_regular_available", "eval_elastic_available") + }) +} + +// TestFlowControlRaftMembershipV2 tests flow token behavior when the raft +// membership changes. +func TestFlowControlRaftMembershipV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + tc := testcluster.StartTestCluster(t, 5, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "raft_membership")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Adding a voting replica on n4.)`) + tc.AddVotersOrFatal(t, k, tc.Target(3)) + h.waitForConnectedStreams(ctx, desc.RangeID, 4, 0 /* serverIdx */) + + h.comment(` +-- Observe the total tracked tokens per-stream on n1. s1-s3 should have 1MiB +-- tracked each, and s4 should have none.`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Issuing 1x1MiB, 4x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s3 +-- should have 2MiB tracked (they've observed 2x1MiB writes), s4 should have +-- 1MiB. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Removing voting replica from n3.)`) + tc.RemoveVotersOrFatal(t, k, tc.Target(2)) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Adding non-voting replica to n5.)`) + tc.AddNonVotersOrFatal(t, k, tc.Target(4)) + h.waitForConnectedStreams(ctx, desc.RangeID, 4, 0 /* serverIdx */) + + h.comment(`-- (Issuing 1x1MiB, 4x replicated write (w/ one non-voter) that's not admitted.`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s2 +-- should have 3MiB tracked (they've observed 3x1MiB writes), there should be +-- no s3 since it was removed, s4 and s5 should have 2MiB and 1MiB +-- respectively. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 5, 0 /* serverIdx */) + + h.comment(`-- Observe that there no tracked tokens across s1,s2,s4,s5.`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned, including from when s3 was removed as a raft +-- member. +`) + h.query(n1, v2FlowTokensQueryStr) + }) +} + +// TestFlowControlRaftMembershipRemoveSelf tests flow token behavior when the +// raft leader removes itself from the raft group. +func TestFlowControlRaftMembershipRemoveSelfV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + testutils.RunTrueAndFalse(t, "transfer-lease-first", func(t *testing.T, transferLeaseFirst bool) { + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + tc := testcluster.StartTestCluster(t, 4, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + n4 := sqlutils.MakeSQLRunner(tc.ServerConn(3)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + // Note this test behaves identically independent of we transfer the lease + // first. + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "raft_membership_remove_self")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + + // Make sure the lease is on n1 and that we're triply connected. + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Replacing current raft leader on n1 in raft group with new n4 replica.)`) + testutils.SucceedsSoon(t, func() error { + // Relocate range from n1 -> n4. + if err := tc.Servers[2].DB(). + AdminRelocateRange( + context.Background(), desc.StartKey.AsRawKey(), + tc.Targets(3, 2, 1), nil, transferLeaseFirst); err != nil { + return err + } + leaseHolder, err := tc.FindRangeLeaseHolder(desc, nil) + if err != nil { + return err + } + if !leaseHolder.Equal(tc.Target(3)) { + return errors.Errorf("expected leaseholder to be n4, found %v", leaseHolder) + } + return nil + }) + h.waitForAllTokensReturned(ctx, 4, 0 /* serverIdx */) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 3 /* serverIdx */) + + h.comment(` +-- Flow token metrics from n1 after raft leader removed itself from raft group. +-- All {regular,elastic} tokens deducted are returned. Note that the available +-- tokens increases, as n1 has seen 4 replication streams, s1,s2,s3,s4. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- n1 should have no connected streams now after transferring the lease to n4. +-- While, n4 should have 3 connected streams to s2,s3,s4. Query the stream count +-- on n1, then on n4. +-- n1 connected v2 streams: +`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- n4 connected v2 streams:`) + h.query(n4, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 4, 0 /* serverIdx */) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. Tokens were already +-- returned earlier, so there's no change. +`) + h.query(n1, v2FlowTokensQueryStr) + }) + }) +} + +// TestFlowControlClassPrioritizationV2 shows how tokens are managed for both +// regular and elastic work. It does so by replicating + admitting a single +// 1MiB {regular,elastic} write. +func TestFlowControlClassPrioritizationV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 5, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "class_prioritization")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.BulkNormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated regular write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of {regular,elastic} +-- tokens with no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned. +`) + h.query(n1, v2FlowTokensQueryStr) + }) +} + +// TestFlowControlQuiescedRangeV2 tests flow token behavior when ranges are +// quiesced. It ensures that we have timely returns of flow tokens even when +// there's no raft traffic to piggyback token returns on top of. +func TestFlowControlQuiescedRangeV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + var disableWorkQueueGranting atomic.Bool + var disableFallbackTokenDispatch atomic.Bool + disableWorkQueueGranting.Store(true) + disableFallbackTokenDispatch.Store(true) + + settings := cluster.MakeTestingClusterSettings() + // Override metamorphism to allow range quiescence. + kvserver.ExpirationLeasesOnly.Override(ctx, &settings.SV, false) + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + OverrideTokenDeduction: func(_ kvflowcontrol.Tokens) kvflowcontrol.Tokens { + // This test asserts on the exact values of tracked + // tokens. In non-test code, the tokens deducted are + // a few bytes off (give or take) from the size of + // the proposals. We don't care about such + // differences. + return kvflowcontrol.Tokens(1 << 20 /* 1MiB */) + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + RaftTransport: &kvserver.RaftTransportTestingKnobs{ + DisableFallbackFlowTokenDispatch: func() bool { + return disableFallbackTokenDispatch.Load() + }, + DisablePiggyBackedFlowTokenDispatch: func() bool { + // We'll only test using the fallback token mechanism. + return true + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "quiesced_range")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.enableVerboseRaftMsgLoggingForRange(desc.RangeID) + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.BulkNormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + // TODO(pav-kv,kvoli): When #129581 is complete, this test will fail + // because the range won't quiesce with a lagging admitted vector. Update + // the test to assert that the range doesn't quiesce then. + // + // Wait for the range to quiesce. + h.comment(`-- (Wait for range to quiesce.)`) + testutils.SucceedsSoon(t, func() error { + leader := tc.GetRaftLeader(t, roachpb.RKey(k)) + require.NotNil(t, leader) + if !leader.IsQuiescent() { + return errors.Errorf("%s not quiescent", leader) + } + return nil + }) + + h.comment(` +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.)`) + disableWorkQueueGranting.Store(false) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +`) + h.waitForTotalTrackedTokens(ctx, desc.RangeID, 2<<20 /* 2*1MiB=2MiB */, 0 /* serverIdx */) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Enable the fallback token dispatch mechanism.)`) + disableFallbackTokenDispatch.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the fallback mechanism. +`) + h.query(n1, v2FlowTokensQueryStr) + }) +} + +// TestFlowControlUnquiescedRangeV2 tests flow token behavior when ranges are +// unquiesced. It's a sort of roundabout test to ensure that flow tokens are +// returned through the raft transport piggybacking mechanism, piggybacking on +// raft heartbeats. +func TestFlowControlUnquiescedRangeV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + var disableWorkQueueGranting atomic.Bool + var disablePiggybackTokenDispatch atomic.Bool + disableWorkQueueGranting.Store(true) + disablePiggybackTokenDispatch.Store(true) + + settings := cluster.MakeTestingClusterSettings() + // Override metamorphism to allow range quiescence. + kvserver.ExpirationLeasesOnly.Override(ctx, &settings.SV, false) + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands or followers unquiescing + // ranges by calling elections. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + OverrideTokenDeduction: func(_ kvflowcontrol.Tokens) kvflowcontrol.Tokens { + // This test asserts on the exact values of tracked + // tokens. In non-test code, the tokens deducted are + // a few bytes off (give or take) from the size of + // the proposals. We don't care about such + // differences. + return kvflowcontrol.Tokens(1 << 20 /* 1MiB */) + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + RaftTransport: &kvserver.RaftTransportTestingKnobs{ + DisableFallbackFlowTokenDispatch: func() bool { + // We'll only test using the piggy-back token mechanism. + return true + }, + DisablePiggyBackedFlowTokenDispatch: func() bool { + return disablePiggybackTokenDispatch.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "unquiesced_range")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.enableVerboseRaftMsgLoggingForRange(desc.RangeID) + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.BulkNormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + // TODO(pav-kv,kvoli): When #129581 is complete, this test will fail + // because the range won't quiesce with a lagging admitted vector. Update + // the test to assert that the range doesn't quiesce then. + // + // Wait for the range to quiesce. + h.comment(`-- (Wait for range to quiesce.)`) + testutils.SucceedsSoon(t, func() error { + leader := tc.GetRaftLeader(t, roachpb.RKey(k)) + require.NotNil(t, leader) + if !leader.IsQuiescent() { + return errors.Errorf("%s not quiescent", leader) + } + return nil + }) + + h.comment(` +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.)`) + disableWorkQueueGranting.Store(false) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +`) + h.waitForTotalTrackedTokens(ctx, desc.RangeID, 2<<20 /* 2*1MiB=2MiB */, 0 /* serverIdx */) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Enable the piggyback token dispatch mechanism.)`) + disablePiggybackTokenDispatch.Store(false) + + h.comment(`-- (Unquiesce the range.)`) + testutils.SucceedsSoon(t, func() error { + _, err := tc.GetRaftLeader(t, roachpb.RKey(k)).MaybeUnquiesceAndPropose() + require.NoError(t, err) + return h.checkAllTokensReturned(ctx, 3, 0 /* serverIdx */) + }) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the piggyback mechanism. +`) + h.query(n1, v2FlowTokensQueryStr) + }) +} + +// TestFlowControlTransferLeaseV2 tests flow control behavior when the range +// lease is transferred, and the raft leadership along with it. +func TestFlowControlTransferLeaseV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 5, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "transfer_lease")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Transferring range lease to n2 and allowing leadership to follow.)`) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(1)) + testutils.SucceedsSoon(t, func() error { + if leader := tc.GetRaftLeader(t, roachpb.RKey(k)); leader.NodeID() != tc.Target(1).NodeID { + return errors.Errorf("expected raft leadership to transfer to n1, found n%d", leader.NodeID()) + } + return nil + }) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) + + h.comment(` +-- Flow token metrics from n1 having lost the lease and raft leadership. All +-- deducted tokens are returned. +`) + h.query(n1, v2FlowTokensQueryStr) + }) +} + +// TestFlowControlLeaderNotLeaseholderV2 tests flow control behavior when the +// range leaseholder is not the raft leader. +// +// NOTE: This test diverges from TestFlowControlLeaderNotLeaseholder, as v1 +// replication flow control doesn't admit via the store work queue when the +// replica is a leaseholder but not the raft leader. Tracked in #130948. +func TestFlowControlLeaderNotLeaseholderV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 5, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + // Disable leader transfers during leaseholder changes so + // that we can easily create leader-not-leaseholder + // scenarios. + DisableLeaderFollowsLeaseholder: true, + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + n2 := sqlutils.MakeSQLRunner(tc.ServerConn(1)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "leader_not_leaseholder")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Transferring only range lease, not raft leadership, to n2.)`) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(1)) + require.Equal(t, tc.GetRaftLeader(t, roachpb.RKey(k)).NodeID(), tc.Target(0).NodeID) + + h.comment(` +-- Flow token metrics from n1 having lost the lease but retained raft +-- leadership. No deducted tokens are released. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- (Allow below-raft admission to proceed. All tokens should be returned.) +`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- (Issuing another 1x1MiB, 3x replicated write that's admitted via +-- the work queue on the leaseholder. It shouldn't deduct any tokens.) +`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Looking at n1's flow token metrics, there's no change. No additional tokens +-- are deducted since the write is not being proposed here. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- Looking at n2's flow token metrics, there's no activity. n2 never acquired +-- the raft leadership. +`) + h.query(n2, v2FlowTokensQueryStr) + }) +} + +// TestFlowControlGranterAdmitOneByOneV2 is a reproduction for #105185. +// Internal admission code that relied on admitting at most one waiting request +// was in fact admitting more than one, and doing so recursively with call +// stacks as deep as the admit chain. This triggered panics (and is also just +// undesirable, design-wise). This test intentionally queues a 1000+ small +// requests, to that end. +func TestFlowControlGranterAdmitOneByOneV2(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunValues(t, "v2_enabled_when_leader_level", []kvflowcontrol.V2EnabledWhenLeaderLevel{ + kvflowcontrol.V2EnabledWhenLeaderV1Encoding, + kvflowcontrol.V2EnabledWhenLeaderV2Encoding, + }, func(t *testing.T, v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel) { + ctx := context.Background() + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return v2EnabledWhenLeaderLevel + }, + OverrideTokenDeduction: func(_ kvflowcontrol.Tokens) kvflowcontrol.Tokens { + // This test asserts on the exact values of tracked + // tokens. In non-test code, the tokens deducted are + // a few bytes off (give or take) from the size of + // the proposals. We don't care about such + // differences. + return kvflowcontrol.Tokens(1 << 10 /* 1KiB */) + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + AlwaysTryGrantWhenAdmitted: true, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelperV2(t, tc, v2EnabledWhenLeaderLevel) + h.init() + defer h.close(makeV2EnabledTestFileName(v2EnabledWhenLeaderLevel, "granter_admit_one_by_one")) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + + h.comment(`-- (Issuing regular 1024*1KiB, 3x replicated writes that are not admitted.)`) + h.log("sending put requests") + for i := 0; i < 1024; i++ { + // TODO(kvoli): This sleep is necessary because we fill up the (raft) + // send queue and delay sending + tracking. We need to determine why this + // occasionally occurs under race. + time.Sleep(1 * time.Millisecond) + h.put(ctx, k, 1<<10 /* 1KiB */, admissionpb.NormalPri) + } + h.log("sent put requests") + + h.comment(` +-- Flow token metrics from n1 after issuing 1024KiB, i.e. 1MiB 3x replicated writes +-- that are yet to get admitted. We see 3*1MiB=3MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- Observe the total tracked tokens per-stream on n1.`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */) // wait for admission + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. We see 3MiB returns of +-- {regular,elastic} tokens, and the available capacities going back to what +-- they were. In #105185, by now we would've observed panics. +`) + h.query(n1, v2FlowTokensQueryStr) + + }) +} + type flowControlTestHelper struct { t *testing.T tc *testcluster.TestCluster @@ -2189,6 +3885,17 @@ func newFlowControlTestHelperV1(t *testing.T, tc *testcluster.TestCluster) *flow ) } +func newFlowControlTestHelperV2( + t *testing.T, tc *testcluster.TestCluster, level kvflowcontrol.V2EnabledWhenLeaderLevel, +) *flowControlTestHelper { + return newFlowControlTestHelper(t, + tc, + "flow_control_integration_v2", /* testdata */ + level, + true, /* isStatic */ + ) +} + func (h *flowControlTestHelper) init() { // Reach into each server's cluster setting and override. This causes any // registered change callbacks to run immediately, which is important since @@ -2378,6 +4085,16 @@ const v1FlowTokensQueryStr = ` ORDER BY name ASC; ` +// v2FlowTokensQueryStr is the query string to fetch flow tokens metrics from +// the node metrics table. It fetches all metrics related to flow control +// tokens, distinct from v1 token metrics which only track eval tokens. +const v2FlowTokensQueryStr = ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; +` + // query runs the given SQL query against the given SQLRunner, and appends the // output to the testdata file buffer. func (h *flowControlTestHelper) query(runner *sqlutils.SQLRunner, sql string, headers ...string) { @@ -2466,3 +4183,22 @@ func (h *flowControlTestHelper) enableVerboseRaftMsgLoggingForRange(rangeID roac }) } } + +// makeV2EnabledTestFileName is a utility function which returns an updated +// filename for the testdata file based on the v2EnabledWhenLeaderLevel. +func makeV2EnabledTestFileName( + v2EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel, filename string, +) string { + var s string + switch v2EnabledWhenLeaderLevel { + case kvflowcontrol.V2NotEnabledWhenLeader: + panic("unused") + case kvflowcontrol.V2EnabledWhenLeaderV1Encoding: + s = "_v1_encoding" + case kvflowcontrol.V2EnabledWhenLeaderV2Encoding: + s = "" + default: + panic("unknown v2EnabledWhenLeaderLevel") + } + return filename + s +} diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/admission_post_split_merge b/pkg/kv/kvserver/testdata/flow_control_integration_v2/admission_post_split_merge new file mode 100644 index 000000000000..1b2dedc30625 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/admission_post_split_merge @@ -0,0 +1,162 @@ +echo +---- +---- +-- Flow token metrics from n1 after issuing a regular 2*1MiB 3x replicated write +-- that are yet to get admitted. We see 2*3*1MiB=6MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. The 2*1MiB writes +-- happened on what is soon going to be the LHS and RHS of a range being split. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 18 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 42 MiB + kvflowcontrol.tokens.eval.regular.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 18 MiB + kvflowcontrol.tokens.send.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 42 MiB + kvflowcontrol.tokens.send.regular.deducted | 6.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Splitting range.) + + +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- deducted which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 6MiB+15MiB=21MiB now. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 21 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 27 MiB + kvflowcontrol.tokens.eval.regular.deducted | 21 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 3.0 MiB + kvflowcontrol.tokens.send.elastic.deducted | 21 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 27 MiB + kvflowcontrol.tokens.send.regular.deducted | 21 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the newly split off replica, with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + 71 | 3 + + +-- (Merging ranges.) + + +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens deducted which comes +-- from 4MiB*3=12MiB. So we stand at 21MiB+12MiB=33MiB tokens deducted now. The +-- RHS of the range is gone now, and the previously 3*3MiB=9MiB of tokens +-- deducted for it are released at the subsuming LHS leaseholder. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 0 B + kvflowcontrol.tokens.eval.elastic.deducted | 33 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 24 MiB + kvflowcontrol.tokens.eval.regular.deducted | 33 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 9.0 MiB + kvflowcontrol.tokens.eval.regular.returned | 9.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 0 B + kvflowcontrol.tokens.send.elastic.deducted | 33 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 9.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 24 MiB + kvflowcontrol.tokens.send.regular.deducted | 33 MiB + kvflowcontrol.tokens.send.regular.disconnected | 9.0 MiB + kvflowcontrol.tokens.send.regular.returned | 9.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe only the merged replica with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. We see all outstanding +-- {regular,elastic} tokens returned, including those from: +-- - the LHS before the merge, and +-- - the LHS and RHS before the original split. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 33 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 33 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 33 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 9.0 MiB + kvflowcontrol.tokens.eval.regular.returned | 33 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 33 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 9.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 33 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 33 MiB + kvflowcontrol.tokens.send.regular.disconnected | 9.0 MiB + kvflowcontrol.tokens.send.regular.returned | 33 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/admission_post_split_merge_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/admission_post_split_merge_v1_encoding new file mode 100644 index 000000000000..a492335c6a93 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/admission_post_split_merge_v1_encoding @@ -0,0 +1,169 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- Flow token metrics from n1 after issuing a regular 2*1MiB 3x replicated write +-- that are yet to get admitted. We see 2*3*1MiB=6MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. The 2*1MiB writes +-- happened on what is soon going to be the LHS and RHS of a range being split. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 18 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 18 MiB + kvflowcontrol.tokens.send.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Splitting range.) + + +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- deducted which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 6MiB+15MiB=21MiB now. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 21 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 3.0 MiB + kvflowcontrol.tokens.send.elastic.deducted | 21 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the newly split off replica, with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + 71 | 3 + + +-- (Merging ranges.) + + +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens deducted which comes +-- from 4MiB*3=12MiB. So we stand at 21MiB+12MiB=33MiB tokens deducted now. The +-- RHS of the range is gone now, and the previously 3*3MiB=9MiB of tokens +-- deducted for it are released at the subsuming LHS leaseholder. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 0 B + kvflowcontrol.tokens.eval.elastic.deducted | 33 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 0 B + kvflowcontrol.tokens.send.elastic.deducted | 33 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 9.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe only the merged replica with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. We see all outstanding +-- {regular,elastic} tokens returned, including those from: +-- - the LHS before the merge, and +-- - the LHS and RHS before the original split. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 33 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 33 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 33 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 9.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 33 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/basic b/pkg/kv/kvserver/testdata/flow_control_integration_v2/basic new file mode 100644 index 000000000000..cf74b97f03d5 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/basic @@ -0,0 +1,96 @@ +echo +---- +---- +-- Flow token metrics, before issuing the regular 1MiB replicated write. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 0 B + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 0 B + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Issuing + admitting a regular 1MiB, triply replicated write...) + + +-- Stream counts as seen by n1 post-write. We should see three {regular,elastic} +-- streams given there are three nodes and we're using a replication factor of +-- three. +SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%stream%' +ORDER BY name ASC; + + kvflowcontrol.streams.eval.elastic.blocked_count | 0 + kvflowcontrol.streams.eval.elastic.total_count | 3 + kvflowcontrol.streams.eval.regular.blocked_count | 0 + kvflowcontrol.streams.eval.regular.total_count | 3 + kvflowcontrol.streams.send.elastic.blocked_count | 0 + kvflowcontrol.streams.send.elastic.total_count | 3 + kvflowcontrol.streams.send.regular.blocked_count | 0 + kvflowcontrol.streams.send.regular.total_count | 3 + + +-- Another view of the stream count, using /inspectz-backed vtables. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- Flow token metrics from n1 after issuing the regular 1MiB replicated write, +-- and it being admitted on n1, n2 and n3. We should see 3*1MiB = 3MiB of +-- {regular,elastic} tokens deducted and returned, and {8*3=24MiB,16*3=48MiB} of +-- {regular,elastic} tokens available. Everything should be accounted for. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/basic_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/basic_v1_encoding new file mode 100644 index 000000000000..50b7f93c0bf0 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/basic_v1_encoding @@ -0,0 +1,103 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- Flow token metrics, before issuing the regular 1MiB replicated write. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 0 B + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 0 B + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Issuing + admitting a regular 1MiB, triply replicated write...) + + +-- Stream counts as seen by n1 post-write. We should see three {regular,elastic} +-- streams given there are three nodes and we're using a replication factor of +-- three. +SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%stream%' +ORDER BY name ASC; + + kvflowcontrol.streams.eval.elastic.blocked_count | 0 + kvflowcontrol.streams.eval.elastic.total_count | 3 + kvflowcontrol.streams.eval.regular.blocked_count | 0 + kvflowcontrol.streams.eval.regular.total_count | 3 + kvflowcontrol.streams.send.elastic.blocked_count | 0 + kvflowcontrol.streams.send.elastic.total_count | 3 + kvflowcontrol.streams.send.regular.blocked_count | 0 + kvflowcontrol.streams.send.regular.total_count | 3 + + +-- Another view of the stream count, using /inspectz-backed vtables. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- Flow token metrics from n1 after issuing the regular 1MiB replicated write, +-- and it being admitted on n1, n2 and n3. We should see 3*1MiB = 3MiB of +-- {regular,elastic} tokens deducted and returned, and {8*3=24MiB,16*3=48MiB} of +-- {regular,elastic} tokens available. Everything should be accounted for. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/blocked_admission b/pkg/kv/kvserver/testdata/flow_control_integration_v2/blocked_admission new file mode 100644 index 000000000000..c878d1fecddf --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/blocked_admission @@ -0,0 +1,105 @@ +echo +---- +---- +-- (Issuing 5 regular 1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes +-- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 15 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 33 MiB + kvflowcontrol.tokens.eval.regular.deducted | 15 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 9.0 MiB + kvflowcontrol.tokens.send.elastic.deducted | 15 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 33 MiB + kvflowcontrol.tokens.send.regular.deducted | 15 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 5.0 MiB + 70 | 2 | 5.0 MiB + 70 | 3 | 5.0 MiB + + +-- Observe the individual tracked tokens per-stream on the scratch range. +SELECT range_id, store_id, priority, crdb_internal.humanize_bytes(tokens::INT8) + FROM crdb_internal.kv_flow_token_deductions_v2 + + range_id | store_id | priority | tokens +-----------+----------+------------+---------- + 70 | 1 | normal-pri | 1.0 MiB + 70 | 1 | normal-pri | 1.0 MiB + 70 | 1 | normal-pri | 1.0 MiB + 70 | 1 | normal-pri | 1.0 MiB + 70 | 1 | normal-pri | 1.0 MiB + 70 | 2 | normal-pri | 1.0 MiB + 70 | 2 | normal-pri | 1.0 MiB + 70 | 2 | normal-pri | 1.0 MiB + 70 | 2 | normal-pri | 1.0 MiB + 70 | 2 | normal-pri | 1.0 MiB + 70 | 3 | normal-pri | 1.0 MiB + 70 | 3 | normal-pri | 1.0 MiB + 70 | 3 | normal-pri | 1.0 MiB + 70 | 3 | normal-pri | 1.0 MiB + 70 | 3 | normal-pri | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. We see 15MiB returns of +-- {regular,elastic} tokens, and the available capacities going back to what +-- they were. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 15 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 15 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 15 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 15 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 15 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 15 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 15 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 15 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/blocked_admission_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/blocked_admission_v1_encoding new file mode 100644 index 000000000000..7e0c7c8979c2 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/blocked_admission_v1_encoding @@ -0,0 +1,112 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing 5 regular 1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes +-- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 15 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 9.0 MiB + kvflowcontrol.tokens.send.elastic.deducted | 15 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 5.0 MiB + 70 | 2 | 5.0 MiB + 70 | 3 | 5.0 MiB + + +-- Observe the individual tracked tokens per-stream on the scratch range. +SELECT range_id, store_id, priority, crdb_internal.humanize_bytes(tokens::INT8) + FROM crdb_internal.kv_flow_token_deductions_v2 + + range_id | store_id | priority | tokens +-----------+----------+----------+---------- + 70 | 1 | low-pri | 1.0 MiB + 70 | 1 | low-pri | 1.0 MiB + 70 | 1 | low-pri | 1.0 MiB + 70 | 1 | low-pri | 1.0 MiB + 70 | 1 | low-pri | 1.0 MiB + 70 | 2 | low-pri | 1.0 MiB + 70 | 2 | low-pri | 1.0 MiB + 70 | 2 | low-pri | 1.0 MiB + 70 | 2 | low-pri | 1.0 MiB + 70 | 2 | low-pri | 1.0 MiB + 70 | 3 | low-pri | 1.0 MiB + 70 | 3 | low-pri | 1.0 MiB + 70 | 3 | low-pri | 1.0 MiB + 70 | 3 | low-pri | 1.0 MiB + 70 | 3 | low-pri | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. We see 15MiB returns of +-- {regular,elastic} tokens, and the available capacities going back to what +-- they were. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 15 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 15 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 15 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 15 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/class_prioritization b/pkg/kv/kvserver/testdata/flow_control_integration_v2/class_prioritization new file mode 100644 index 000000000000..041593981123 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/class_prioritization @@ -0,0 +1,103 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Issuing 1x1MiB, 3x replicated regular write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of {regular,elastic} +-- tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 18 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 45 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 18 MiB + kvflowcontrol.tokens.send.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 45 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/class_prioritization_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/class_prioritization_v1_encoding new file mode 100644 index 000000000000..ada19c5b8b64 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/class_prioritization_v1_encoding @@ -0,0 +1,110 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Issuing 1x1MiB, 3x replicated regular write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of {regular,elastic} +-- tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 18 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 18 MiB + kvflowcontrol.tokens.send.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/crashed_node b/pkg/kv/kvserver/testdata/flow_control_integration_v2/crashed_node new file mode 100644 index 000000000000..2e2d5ed01a90 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/crashed_node @@ -0,0 +1,90 @@ +echo +---- +---- +-- (Issuing regular 5x1MiB, 2x replicated writes that are not admitted.) + + +-- Flow token metrics from n1 after issuing 5 regular 1MiB 2x replicated writes +-- that are yet to get admitted. We see 5*1MiB*2=10MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 22 MiB + kvflowcontrol.tokens.eval.regular.deducted | 10 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 6.0 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 22 MiB + kvflowcontrol.tokens.send.regular.deducted | 10 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the per-stream tracked tokens on n1, before n2 is crashed. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 5.0 MiB + 70 | 2 | 5.0 MiB + + +-- (Crashing n2) + + +-- Observe the per-stream tracked tokens on n1, after n2 crashed. We're no +-- longer tracking the 5MiB held by n2. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 5.0 MiB + + +-- Flow token metrics from n1 after n2 crashed. Observe that we've returned the +-- 5MiB previously held by n2. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 11 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 5.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 5.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 27 MiB + kvflowcontrol.tokens.eval.regular.deducted | 10 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 5.0 MiB + kvflowcontrol.tokens.eval.regular.returned | 5.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 11 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 5.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 5.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 27 MiB + kvflowcontrol.tokens.send.regular.deducted | 10 MiB + kvflowcontrol.tokens.send.regular.disconnected | 5.0 MiB + kvflowcontrol.tokens.send.regular.returned | 5.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/crashed_node_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/crashed_node_v1_encoding new file mode 100644 index 000000000000..35d8436b4cc0 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/crashed_node_v1_encoding @@ -0,0 +1,97 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing regular 5x1MiB, 2x replicated writes that are not admitted.) + + +-- Flow token metrics from n1 after issuing 5 regular 1MiB 2x replicated writes +-- that are yet to get admitted. We see 5*1MiB*2=10MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 32 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 6.0 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 32 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the per-stream tracked tokens on n1, before n2 is crashed. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 5.0 MiB + 70 | 2 | 5.0 MiB + + +-- (Crashing n2) + + +-- Observe the per-stream tracked tokens on n1, after n2 crashed. We're no +-- longer tracking the 5MiB held by n2. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 5.0 MiB + + +-- Flow token metrics from n1 after n2 crashed. Observe that we've returned the +-- 5MiB previously held by n2. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 11 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 5.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 5.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 32 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 11 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 5.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 5.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 32 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/granter_admit_one_by_one b/pkg/kv/kvserver/testdata/flow_control_integration_v2/granter_admit_one_by_one new file mode 100644 index 000000000000..9268f244f2b9 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/granter_admit_one_by_one @@ -0,0 +1,82 @@ +echo +---- +---- +-- (Issuing regular 1024*1KiB, 3x replicated writes that are not admitted.) + + +-- Flow token metrics from n1 after issuing 1024KiB, i.e. 1MiB 3x replicated writes +-- that are yet to get admitted. We see 3*1MiB=3MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 45 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 45 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. We see 3MiB returns of +-- {regular,elastic} tokens, and the available capacities going back to what +-- they were. In #105185, by now we would've observed panics. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/granter_admit_one_by_one_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/granter_admit_one_by_one_v1_encoding new file mode 100644 index 000000000000..f95a174ccdce --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/granter_admit_one_by_one_v1_encoding @@ -0,0 +1,89 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing regular 1024*1KiB, 3x replicated writes that are not admitted.) + + +-- Flow token metrics from n1 after issuing 1024KiB, i.e. 1MiB 3x replicated writes +-- that are yet to get admitted. We see 3*1MiB=3MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. We see 3MiB returns of +-- {regular,elastic} tokens, and the available capacities going back to what +-- they were. In #105185, by now we would've observed panics. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/leader_not_leaseholder b/pkg/kv/kvserver/testdata/flow_control_integration_v2/leader_not_leaseholder new file mode 100644 index 000000000000..b56971f60a5c --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/leader_not_leaseholder @@ -0,0 +1,160 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 45 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 45 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Transferring only range lease, not raft leadership, to n2.) + + +-- Flow token metrics from n1 having lost the lease but retained raft +-- leadership. No deducted tokens are released. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 45 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 45 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Allow below-raft admission to proceed. All tokens should be returned.) +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Issuing another 1x1MiB, 3x replicated write that's admitted via +-- the work queue on the leaseholder. It shouldn't deduct any tokens.) + + +-- Looking at n1's flow token metrics, there's no change. No additional tokens +-- are deducted since the write is not being proposed here. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Looking at n2's flow token metrics, there's no activity. n2 never acquired +-- the raft leadership. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 0 B + kvflowcontrol.tokens.eval.elastic.deducted | 0 B + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 0 B + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 0 B + kvflowcontrol.tokens.send.elastic.deducted | 0 B + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 0 B + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/leader_not_leaseholder_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/leader_not_leaseholder_v1_encoding new file mode 100644 index 000000000000..64ab74d81700 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/leader_not_leaseholder_v1_encoding @@ -0,0 +1,167 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Transferring only range lease, not raft leadership, to n2.) + + +-- Flow token metrics from n1 having lost the lease but retained raft +-- leadership. No deducted tokens are released. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Allow below-raft admission to proceed. All tokens should be returned.) +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Issuing another 1x1MiB, 3x replicated write that's admitted via +-- the work queue on the leaseholder. It shouldn't deduct any tokens.) + + +-- Looking at n1's flow token metrics, there's no change. No additional tokens +-- are deducted since the write is not being proposed here. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Looking at n2's flow token metrics, there's no activity. n2 never acquired +-- the raft leadership. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 0 B + kvflowcontrol.tokens.eval.elastic.deducted | 0 B + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 0 B + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 0 B + kvflowcontrol.tokens.send.elastic.deducted | 0 B + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 0 B + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/quiesced_range b/pkg/kv/kvserver/testdata/flow_control_integration_v2/quiesced_range new file mode 100644 index 000000000000..24c50ce2e7e0 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/quiesced_range @@ -0,0 +1,108 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Wait for range to quiesce.) + + +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.) + + +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 22 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 1.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 22 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 1.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Enable the fallback token dispatch mechanism.) + + +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the fallback mechanism. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/quiesced_range_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/quiesced_range_v1_encoding new file mode 100644 index 000000000000..6aae6c3b8ee4 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/quiesced_range_v1_encoding @@ -0,0 +1,115 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Wait for range to quiesce.) + + +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.) + + +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 22 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 1.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 22 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 1.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Enable the fallback token dispatch mechanism.) + + +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the fallback mechanism. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership new file mode 100644 index 000000000000..d5d70ac459ef --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership @@ -0,0 +1,140 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 45 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 45 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Adding a voting replica on n4.) + + +-- Observe the total tracked tokens per-stream on n1. s1-s3 should have 1MiB +-- tracked each, and s4 should have none. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + 70 | 4 | 0 B + + +-- (Issuing 1x1MiB, 4x replicated write that's not admitted.) + + +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s3 +-- should have 2MiB tracked (they've observed 2x1MiB writes), s4 should have +-- 1MiB. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 2.0 MiB + 70 | 2 | 2.0 MiB + 70 | 3 | 2.0 MiB + 70 | 4 | 1.0 MiB + + +-- (Removing voting replica from n3.) + + +-- (Adding non-voting replica to n5.) + + +-- (Issuing 1x1MiB, 4x replicated write (w/ one non-voter) that's not admitted. + + +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s2 +-- should have 3MiB tracked (they've observed 3x1MiB writes), there should be +-- no s3 since it was removed, s4 and s5 should have 2MiB and 1MiB +-- respectively. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 3.0 MiB + 70 | 2 | 3.0 MiB + 70 | 4 | 2.0 MiB + 70 | 5 | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Observe that there no tracked tokens across s1,s2,s4,s5. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 0 B + 70 | 2 | 0 B + 70 | 4 | 0 B + 70 | 5 | 0 B + + +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned, including from when s3 was removed as a raft +-- member. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 40 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 11 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 2.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 11 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 80 MiB + kvflowcontrol.tokens.eval.regular.deducted | 11 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 2.0 MiB + kvflowcontrol.tokens.eval.regular.returned | 11 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 40 MiB + kvflowcontrol.tokens.send.elastic.deducted | 11 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 2.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 11 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 80 MiB + kvflowcontrol.tokens.send.regular.deducted | 11 MiB + kvflowcontrol.tokens.send.regular.disconnected | 2.0 MiB + kvflowcontrol.tokens.send.regular.returned | 11 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_remove_self b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_remove_self new file mode 100644 index 000000000000..ededbf12b3f1 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_remove_self @@ -0,0 +1,127 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 45 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 45 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Replacing current raft leader on n1 in raft group with new n4 replica.) + + +-- Flow token metrics from n1 after raft leader removed itself from raft group. +-- All {regular,elastic} tokens deducted are returned. Note that the available +-- tokens increases, as n1 has seen 4 replication streams, s1,s2,s3,s4. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 32 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 64 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 32 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 64 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- n1 should have no connected streams now after transferring the lease to n4. +-- While, n4 should have 3 connected streams to s2,s3,s4. Query the stream count +-- on n1, then on n4. +-- n1 connected v2 streams: +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + + +-- n4 connected v2 streams: +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. Tokens were already +-- returned earlier, so there's no change. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 32 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 64 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 32 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 64 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_remove_self_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_remove_self_v1_encoding new file mode 100644 index 000000000000..fc09ed33bfaf --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_remove_self_v1_encoding @@ -0,0 +1,134 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Replacing current raft leader on n1 in raft group with new n4 replica.) + + +-- Flow token metrics from n1 after raft leader removed itself from raft group. +-- All {regular,elastic} tokens deducted are returned. Note that the available +-- tokens increases, as n1 has seen 4 replication streams, s1,s2,s3,s4. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 32 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 64 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 32 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 64 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- n1 should have no connected streams now after transferring the lease to n4. +-- While, n4 should have 3 connected streams to s2,s3,s4. Query the stream count +-- on n1, then on n4. +-- n1 connected v2 streams: +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + + +-- n4 connected v2 streams: +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. Tokens were already +-- returned earlier, so there's no change. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 32 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 64 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 32 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 64 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_v1_encoding new file mode 100644 index 000000000000..69ab7766fef5 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_membership_v1_encoding @@ -0,0 +1,147 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Adding a voting replica on n4.) + + +-- Observe the total tracked tokens per-stream on n1. s1-s3 should have 1MiB +-- tracked each, and s4 should have none. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + 70 | 4 | 0 B + + +-- (Issuing 1x1MiB, 4x replicated write that's not admitted.) + + +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s3 +-- should have 2MiB tracked (they've observed 2x1MiB writes), s4 should have +-- 1MiB. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 2.0 MiB + 70 | 2 | 2.0 MiB + 70 | 3 | 2.0 MiB + 70 | 4 | 1.0 MiB + + +-- (Removing voting replica from n3.) + + +-- (Adding non-voting replica to n5.) + + +-- (Issuing 1x1MiB, 4x replicated write (w/ one non-voter) that's not admitted. + + +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s2 +-- should have 3MiB tracked (they've observed 3x1MiB writes), there should be +-- no s3 since it was removed, s4 and s5 should have 2MiB and 1MiB +-- respectively. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 3.0 MiB + 70 | 2 | 3.0 MiB + 70 | 4 | 2.0 MiB + 70 | 5 | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Observe that there no tracked tokens across s1,s2,s4,s5. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 0 B + 70 | 2 | 0 B + 70 | 4 | 0 B + 70 | 5 | 0 B + + +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned, including from when s3 was removed as a raft +-- member. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 40 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 11 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 2.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 11 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 80 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 40 MiB + kvflowcontrol.tokens.send.elastic.deducted | 11 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 2.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 11 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 80 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_snapshot b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_snapshot new file mode 100644 index 000000000000..d7ea2d80730b --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_snapshot @@ -0,0 +1,225 @@ +echo +---- +---- +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. Since this test is ignoring crashed nodes for token +-- deduction purposes, we see a deduction of 5MiB {regular,elastic} tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 35 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 5.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 75 MiB + kvflowcontrol.tokens.eval.regular.deducted | 5.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 35 MiB + kvflowcontrol.tokens.send.elastic.deducted | 5.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 75 MiB + kvflowcontrol.tokens.send.regular.deducted | 5.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. 1MiB is tracked for n1-n5. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + 70 | 4 | 1.0 MiB + 70 | 5 | 1.0 MiB + + +-- (Killing n2 and n3, but preventing their tokens from being returned + +-- artificially allowing tokens to get deducted.) + + +-- Observe the total tracked tokens per-stream on n1. 1MiB is (still) tracked +-- for n1-n5, because they are not in StateSnapshot yet and have likely been +-- in StateProbe for less than the close timer. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + 70 | 4 | 1.0 MiB + 70 | 5 | 1.0 MiB + + +-- (Issuing another 1MiB of 5x replicated writes while n2 and n3 are down and +-- below-raft admission is paused.) + + +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. We'll have deducted another 5*1MiB=5MiB worth of tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 30 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 70 MiB + kvflowcontrol.tokens.eval.regular.deducted | 10 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 30 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 70 MiB + kvflowcontrol.tokens.send.regular.deducted | 10 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. 2MiB is tracked for n1-n5; +-- see last comment for an explanation why we're still deducting for n2, n3. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 2.0 MiB + 70 | 2 | 2.0 MiB + 70 | 3 | 2.0 MiB + 70 | 4 | 2.0 MiB + 70 | 5 | 2.0 MiB + + +-- (Truncating raft log.) + + +-- (Restarting n2 and n3.) + + +-- Flow token metrics from n1 after restarting n2 and n3. We've returned the +-- 2MiB previously held by those nodes (2MiB each). We're reacting to it's raft +-- progress state, noting that since we've truncated our log, we need to catch +-- it up via snapshot. So we release all held tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 34 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 4.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 4.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 74 MiB + kvflowcontrol.tokens.eval.regular.deducted | 10 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 4.0 MiB + kvflowcontrol.tokens.eval.regular.returned | 4.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 34 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 4.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 4.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 74 MiB + kvflowcontrol.tokens.send.regular.deducted | 10 MiB + kvflowcontrol.tokens.send.regular.disconnected | 4.0 MiB + kvflowcontrol.tokens.send.regular.returned | 4.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. There's nothing tracked +-- for n2 and n3 anymore. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + WHERE total_tracked_tokens > 0 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 2.0 MiB + 70 | 4 | 2.0 MiB + 70 | 5 | 2.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. We see the remaining +-- 6MiB of {regular,elastic} tokens returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 40 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 4.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 10 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 80 MiB + kvflowcontrol.tokens.eval.regular.deducted | 10 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 4.0 MiB + kvflowcontrol.tokens.eval.regular.returned | 10 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 40 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 4.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 10 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 80 MiB + kvflowcontrol.tokens.send.regular.deducted | 10 MiB + kvflowcontrol.tokens.send.regular.disconnected | 4.0 MiB + kvflowcontrol.tokens.send.regular.returned | 10 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1; there should be nothing. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 0 B + 70 | 2 | 0 B + 70 | 3 | 0 B + 70 | 4 | 0 B + 70 | 5 | 0 B + + +-- Another view of tokens, using /inspectz-backed vtables. +SELECT store_id, + crdb_internal.humanize_bytes(available_eval_regular_tokens), + crdb_internal.humanize_bytes(available_eval_elastic_tokens) + FROM crdb_internal.kv_flow_controller_v2 + ORDER BY store_id ASC; + + range_id | eval_regular_available | eval_elastic_available +-----------+------------------------+------------------------- + 1 | 16 MiB | 8.0 MiB + 2 | 16 MiB | 8.0 MiB + 3 | 16 MiB | 8.0 MiB + 4 | 16 MiB | 8.0 MiB + 5 | 16 MiB | 8.0 MiB +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_snapshot_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_snapshot_v1_encoding new file mode 100644 index 000000000000..afd8c5eb0e1a --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/raft_snapshot_v1_encoding @@ -0,0 +1,232 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. Since this test is ignoring crashed nodes for token +-- deduction purposes, we see a deduction of 5MiB {regular,elastic} tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 35 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 5.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 80 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 35 MiB + kvflowcontrol.tokens.send.elastic.deducted | 5.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 80 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. 1MiB is tracked for n1-n5. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + 70 | 4 | 1.0 MiB + 70 | 5 | 1.0 MiB + + +-- (Killing n2 and n3, but preventing their tokens from being returned + +-- artificially allowing tokens to get deducted.) + + +-- Observe the total tracked tokens per-stream on n1. 1MiB is (still) tracked +-- for n1-n5, because they are not in StateSnapshot yet and have likely been +-- in StateProbe for less than the close timer. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + 70 | 4 | 1.0 MiB + 70 | 5 | 1.0 MiB + + +-- (Issuing another 1MiB of 5x replicated writes while n2 and n3 are down and +-- below-raft admission is paused.) + + +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. We'll have deducted another 5*1MiB=5MiB worth of tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 30 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 80 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 30 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 80 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. 2MiB is tracked for n1-n5; +-- see last comment for an explanation why we're still deducting for n2, n3. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 2.0 MiB + 70 | 2 | 2.0 MiB + 70 | 3 | 2.0 MiB + 70 | 4 | 2.0 MiB + 70 | 5 | 2.0 MiB + + +-- (Truncating raft log.) + + +-- (Restarting n2 and n3.) + + +-- Flow token metrics from n1 after restarting n2 and n3. We've returned the +-- 2MiB previously held by those nodes (2MiB each). We're reacting to it's raft +-- progress state, noting that since we've truncated our log, we need to catch +-- it up via snapshot. So we release all held tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 34 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 4.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 4.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 80 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 34 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 4.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 4.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 80 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. There's nothing tracked +-- for n2 and n3 anymore. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + WHERE total_tracked_tokens > 0 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 2.0 MiB + 70 | 4 | 2.0 MiB + 70 | 5 | 2.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. We see the remaining +-- 6MiB of {regular,elastic} tokens returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 40 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 10 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 4.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 10 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 80 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 40 MiB + kvflowcontrol.tokens.send.elastic.deducted | 10 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 4.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 10 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 80 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1; there should be nothing. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 0 B + 70 | 2 | 0 B + 70 | 3 | 0 B + 70 | 4 | 0 B + 70 | 5 | 0 B + + +-- Another view of tokens, using /inspectz-backed vtables. +SELECT store_id, + crdb_internal.humanize_bytes(available_eval_regular_tokens), + crdb_internal.humanize_bytes(available_eval_elastic_tokens) + FROM crdb_internal.kv_flow_controller_v2 + ORDER BY store_id ASC; + + range_id | eval_regular_available | eval_elastic_available +-----------+------------------------+------------------------- + 1 | 16 MiB | 8.0 MiB + 2 | 16 MiB | 8.0 MiB + 3 | 16 MiB | 8.0 MiB + 4 | 16 MiB | 8.0 MiB + 5 | 16 MiB | 8.0 MiB +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/split_merge b/pkg/kv/kvserver/testdata/flow_control_integration_v2/split_merge new file mode 100644 index 000000000000..2e9895ffe3c8 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/split_merge @@ -0,0 +1,125 @@ +echo +---- +---- +-- Flow token metrics from n1 after issuing + admitting the regular 1MiB 3x +-- replicated write to the pre-split range. There should be 3MiB of +-- {regular,elastic} tokens {deducted,returned}. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Splitting range.) + + +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- {deducted,returned}, which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 3MiB+15MiB=18MiB now. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 18 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 18 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 18 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 18 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 18 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 18 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 18 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 18 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the newly split off replica, with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + 71 | 3 + + +-- (Merging ranges.) + + +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens {deducted,returned}, +-- which comes from 4MiB*3=12MiB. So we stand at 18MiB+12MiB=30MiB now. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 30 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 30 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 30 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 30 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 30 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 30 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 30 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 30 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe only the merged replica with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/split_merge_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/split_merge_v1_encoding new file mode 100644 index 000000000000..cb6edfc3fea8 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/split_merge_v1_encoding @@ -0,0 +1,132 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- Flow token metrics from n1 after issuing + admitting the regular 1MiB 3x +-- replicated write to the pre-split range. There should be 3MiB of +-- {regular,elastic} tokens {deducted,returned}. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Splitting range.) + + +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- {deducted,returned}, which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 3MiB+15MiB=18MiB now. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 18 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 18 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 18 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 18 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe the newly split off replica, with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + 71 | 3 + + +-- (Merging ranges.) + + +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens {deducted,returned}, +-- which comes from 4MiB*3=12MiB. So we stand at 18MiB+12MiB=30MiB now. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 30 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 30 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 30 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 30 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Observe only the merged replica with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/transfer_lease b/pkg/kv/kvserver/testdata/flow_control_integration_v2/transfer_lease new file mode 100644 index 000000000000..64c76a406297 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/transfer_lease @@ -0,0 +1,70 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 45 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 45 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Transferring range lease to n2 and allowing leadership to follow.) + + +-- Flow token metrics from n1 having lost the lease and raft leadership. All +-- deducted tokens are returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/transfer_lease_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/transfer_lease_v1_encoding new file mode 100644 index 000000000000..062495878bfa --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/transfer_lease_v1_encoding @@ -0,0 +1,77 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Transferring range lease to n2 and allowing leadership to follow.) + + +-- Flow token metrics from n1 having lost the lease and raft leadership. All +-- deducted tokens are returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/unquiesced_range b/pkg/kv/kvserver/testdata/flow_control_integration_v2/unquiesced_range new file mode 100644 index 000000000000..b191ff0b99c0 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/unquiesced_range @@ -0,0 +1,111 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Wait for range to quiesce.) + + +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.) + + +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 22 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 1.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 22 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 1.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Enable the piggyback token dispatch mechanism.) + + +-- (Unquiesce the range.) + + +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the piggyback mechanism. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/unquiesced_range_v1_encoding b/pkg/kv/kvserver/testdata/flow_control_integration_v2/unquiesced_range_v1_encoding new file mode 100644 index 000000000000..f35f6cbe45c0 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/unquiesced_range_v1_encoding @@ -0,0 +1,118 @@ +# -- When using v1 encoding (V2EnabledWhenLeaderV1Encoding), all entries which +# -- are subject to admission control are encoded as `raftpb.LowPri`, +# -- regardless of their original priority; to avoid the overhead of +# -- deserializing the raft admission metadata. Therefore, as the underlying +# -- test is shared between the v1 and v2 encoding testdata files, the reader +# -- should interpret any comments referring to regular tokens as referring to +# -- elastic token. +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Wait for range to quiesce.) + + +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.) + + +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 22 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 1.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 22 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 1.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Enable the piggyback token dispatch mechanism.) + + +-- (Unquiesce the range.) + + +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the piggyback mechanism. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql From abbf477276748829445a5e8f188d02d931b2b64b Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Thu, 19 Sep 2024 20:14:48 -0400 Subject: [PATCH 7/8] kvserver: add v1->v2 flow control integration test Add `TestFlowControlV1ToV2Transition`, which ratchets up the enabled version of replication flow control v2: ``` v1 protocol with v1 encoding => v2 protocol with v1 encoding => v2 protocol with v2 encoding ``` The test is structured to issue writes and wait for returned tokens whenever the protocol transitions from v1 to v2, or a leader changes. More specifically, the test takes the following steps: ``` (1) Start n1, n2, n3 with v1 protocol and v1 encoding. (2) Upgrade n1 to v2 protocol with v1 encoding. (3) Transfer the range lease to n2. (4) Upgrade n2 to v2 protocol with v1 encoding. (5) Upgrade n3 to v2 protocol with v1 encoding. (6) Upgrade n1 to v2 protocol with v2 encoding. (7) Transfer the range lease to n1. (8) Upgrade n2,n3 to v2 protocol with v2 encoding. (9) Transfer the range lease to n3. ``` Between each step, we issue writes, (un)block admission and observe the flow control metrics and vtables. Resolves: #130431 Resolves: #129276 Release note: None --- .../kvserver/flow_control_integration_test.go | 629 +++++++++- .../v1_to_v2_transition | 1031 +++++++++++++++++ 2 files changed, 1659 insertions(+), 1 deletion(-) create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration_v2/v1_to_v2_transition diff --git a/pkg/kv/kvserver/flow_control_integration_test.go b/pkg/kv/kvserver/flow_control_integration_test.go index 1fb19b257ad7..714a98792df4 100644 --- a/pkg/kv/kvserver/flow_control_integration_test.go +++ b/pkg/kv/kvserver/flow_control_integration_test.go @@ -3844,6 +3844,631 @@ func TestFlowControlGranterAdmitOneByOneV2(t *testing.T) { }) } +// TestFlowControlV1ToV2Transition exercises the transition from replication +// flow control: +// +// - v1 protocol with v1 encoding => +// - v2 protocol with v1 encoding => +// - v2 protocol with v2 encoding +// +// The test is structured as follows: +// +// (1) Start n1, n2, n3 with v1 protocol and v1 encoding. +// (2) Upgrade n1 to v2 protocol with v1 encoding. +// (3) Transfer the range lease to n2. +// (4) Upgrade n2 to v2 protocol with v1 encoding. +// (5) Upgrade n3 to v2 protocol with v1 encoding. +// (6) Upgrade n1 to v2 protocol with v2 encoding. +// (7) Transfer the range lease to n1. +// (8) Upgrade n2,n3 to v2 protocol with v2 encoding. +// (9) Transfer the range lease to n3. +// +// Between each step, we issue writes, (un)block admission and observe the flow +// control metrics and vtables. +func TestFlowControlV1ToV2Transition(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 3 + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + serverLevels := make([]atomic.Uint32, numNodes) + settings := cluster.MakeTestingClusterSettings() + + argsPerServer := make(map[int]base.TestServerArgs) + for i := range serverLevels { + // Every node starts off using the v1 protocol but we will ratchet up the + // levels on servers at different times as we go to test the transition. + serverLevels[i].Store(kvflowcontrol.V2NotEnabledWhenLeader) + argsPerServer[i] = base.TestServerArgs{ + Settings: settings, + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to deal + // with leadership changing hands unintentionally. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideV2EnabledWhenLeaderLevel: func() kvflowcontrol.V2EnabledWhenLeaderLevel { + return serverLevels[i].Load() + }, + OverrideTokenDeduction: func(tokens kvflowcontrol.Tokens) kvflowcontrol.Tokens { + // This test sends several puts, with each put potentially + // diverging by a few bytes between runs, in aggregate this can + // accumulate to enough tokens to produce a diff in metrics. + // Round the token deductions to the nearest MiB avoid this. + return kvflowcontrol.Tokens( + int64(math.Round(float64(tokens)/float64(1<<20))) * 1 << 20) + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + } + } + + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: argsPerServer, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + // We use the base constructor here because we will be modifying the enabled + // level throughout. + h := newFlowControlTestHelper( + t, tc, "flow_control_integration_v2", /* testdata */ + kvflowcontrol.V2NotEnabledWhenLeader, false, /* isStatic */ + ) + + h.init() + defer h.close("v1_to_v2_transition") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.enableVerboseRaftMsgLoggingForRange(desc.RangeID) + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + n2 := sqlutils.MakeSQLRunner(tc.ServerConn(1)) + n3 := sqlutils.MakeSQLRunner(tc.ServerConn(2)) + + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */) + h.comment(` +-- This test exercises the transition from replication flow control: +-- - v1 protocol with v1 encoding => +-- - v2 protocol with v1 encoding => +-- - v2 protocol with v2 encoding +-- The test is structured as follows: +-- (1) Start n1, n2, n3 with v1 protocol and v1 encoding. +-- (2) Upgrade n1 to v2 protocol with v1 encoding. +-- (3) Transfer the range lease to n2. +-- (4) Upgrade n2 to v2 protocol with v1 encoding. +-- (5) Upgrade n3 to v2 protocol with v1 encoding. +-- (6) Upgrade n1 to v2 protocol with v2 encoding. +-- (7) Transfer the range lease to n1. +-- (8) Upgrade n2,n3 to v2 protocol with v2 encoding. +-- (9) Transfer the range lease to n3. +-- Between each step, we issue writes, (un)block admission and observe the +-- flow control metrics and vtables. +-- +-- Start by checking that the leader (n1) has 3 connected v1 streams. +`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(`-- The v1 flow token metrics, there should be 3x1 MiB = 3 MiB of tokens deducted.`) + h.query(n1, v1FlowTokensQueryStr) + h.comment(`-- The v2 flow token metrics, there should be no tokens or deductions.`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- The v1 tracked tokens per-stream on n1 should be 1 MiB for (s1,s2,s3). +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2NotEnabledWhenLeader) + h.comment(` +-- The v1 flow token metrics on n1, there should be 3x1 MiB = 3 MiB of tokens deducted +-- and returned now. With all tokens available. +`) + h.query(n1, v1FlowTokensQueryStr) + + h.comment(` +-- The v1 tracked tokens per-stream on n1 should now be 0. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Block below-raft admission again.)`) + disableWorkQueueGranting.Store(true) + + h.comment(`-- (Issuing 1 x 1MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- The v1 tracked tokens per-stream on n1 should again be 1 MiB for (s1,s2,s3). +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-------------------------------------------------------------------------------- +-- (Upgrading n1 to v2 protocol with v1 encoding.) +-------------------------------------------------------------------------------- +`) + serverLevels[0].Store(kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2NotEnabledWhenLeader) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + + h.comment(` +-- Viewing the range's v2 connected streams, there now should be three. +-- These are lazily instantiated on the first raft event the leader +-- RangeController sees. +`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(` +-- There should also now be no connected streams for the v1 protocol, +-- at the leader n1. +`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(` +-- The v1 flow token metrics, all deducted tokens should be returned after +-- the leader switches to the rac2 protocol. +`) + h.query(n1, v1FlowTokensQueryStr) + + h.comment(`-- (Issuing 1x2MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 2<<20 /* 2MiB */, admissionpb.NormalPri) + + h.comment(` +-- The v2 flow token metrics, the 3 MiB of earlier token deductions from v1 are dropped. +-- Expect 3 * 2 MiB = 6 MiB of deductions, from the most recent write. +-- Note that the v2 protocol with v1 encoding will only ever deduct elastic tokens. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- The v2 tracked tokens per-stream on n1 should now also be 2 MiB for (s1,s2,s3). +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + h.comment(`-- The v2 flow token metrics. The 6 MiB of tokens should be returned.`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Block below-raft admission again.)`) + disableWorkQueueGranting.Store(true) + + h.comment(`-- (Issuing 1 x 1MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- The v2 tracked tokens per-stream on n1 reflect the most recent write +-- and should be 1 MiB per stream now. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- There should also be a corresponding elastic token deduction (not regular), +-- as v2 protocol with v1 encoding will only ever deduct elastic tokens. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- (Transferring range lease to n2 (running v1) and allowing leadership to follow.) +`) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(1)) + testutils.SucceedsSoon(t, func() error { + if leader := tc.GetRaftLeader(t, roachpb.RKey(k)); leader.NodeID() != tc.Target(1).NodeID { + return errors.Errorf("expected raft leadership to transfer to n2, found n%d", leader.NodeID()) + } + return nil + }) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 1 /* serverIdx */) + + h.comment(` +-- The v2 flow token metrics from n1 having lost the lease and raft leadership. +-- All deducted tokens are returned. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- Now expect to see 3 connected v1 streams on n2. +`) + h.query(n2, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Issuing 1 x 3MiB elastic, 3x replicated write that's not admitted.)`) + // We specify the serverIdx to ensure that the write is routed to n2 and not + // n1. If the write were routed to n1, it would skip flow control because + // there isn't a handle (leader isn't there) and instead block indefinitely + // on the store work queue. + h.put(ctx, k, 3<<20 /* 3MiB */, admissionpb.NormalPri, 1 /* serverIdx */) + + h.comment(` +-- The v1 tracked tokens per-stream on n2 should be 3 MiB. +`) + h.query(n2, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Corresponding v1 token metrics on the new leader n2. +-- These should reflect the 3 x 3 MiB = 9 MiB write. +`) + h.query(n2, v1FlowTokensQueryStr) + h.comment(` +-- Corresponding v2 token metrics on the new leader n2. +-- These should be unpopulated, similar to when n1 was first the leader. +`) + h.query(n2, v2FlowTokensQueryStr) + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 1 /* serverIdx */, kvflowcontrol.V2NotEnabledWhenLeader) + + h.comment(` +-- The v1 token metrics on the new leader n2 should now reflect +-- the 9 MiB write and admission, all tokens should be returned. +`) + h.query(n2, v1FlowTokensQueryStr) + + h.comment(`-- (Issuing 1 x 1MiB regular, 3x replicated write that's admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri, 1 /* serverIdx */) + + h.waitForAllTokensReturned(ctx, 3, 1 /* serverIdx */, kvflowcontrol.V2NotEnabledWhenLeader) + h.comment(` +-- The v1 token metrics on the new leader n2 should now also reflect +-- the 9 + 3 = 12 MiB write and admission, all tokens should be returned. +`) + h.query(n2, v1FlowTokensQueryStr) + + h.comment(`-- (Block below-raft admission.)`) + disableWorkQueueGranting.Store(true) + + h.comment(`-- (Issuing 1 x 4MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 4<<20 /* 4MiB */, admissionpb.NormalPri, 1 /* serverIdx */) + h.waitForTotalTrackedTokens(ctx, desc.RangeID, 12<<20 /* 12MiB */, 1, /* serverIdx */ + kvflowcontrol.V2NotEnabledWhenLeader) + + h.comment(` +-- The v1 tracked tokens per-stream on n2 should be 4 MiB. +`) + h.query(n2, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Corresponding v1 token metrics. +-- These should reflect the 3 x 4 MiB = 12 MiB write. +`) + h.query(n2, v1FlowTokensQueryStr) + + h.comment(` +-------------------------------------------------------------------------------- +-- (Upgrading n2 to v2 protocol with v1 encoding.) +-------------------------------------------------------------------------------- +`) + serverLevels[1].Store(kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + h.waitForAllTokensReturned(ctx, 3, 1 /* serverIdx */) + + h.comment(`-- (Issuing another 1x1MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri, 1 /* serverIdx */) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 1 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + + h.comment(` +-- Corresponding v1 token metrics on the new leader n2. +-- All tokens should be returned. +`) + h.query(n2, v1FlowTokensQueryStr) + + h.comment(` +-- Also expect to see 0 connected v1 streams on n2. +`) + h.query(n2, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(` +-- There should be 3 connected streams on n2 for the v2 protocol. +`) + h.query(n2, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(` +-- Corresponding v2 token metrics on the new leader n2. The most recent +-- 3 x 1 MiB = 3 MiB write should be reflected in the token deductions. +-- Recall that v2 protocol with v1 encoding will only ever deduct elastic tokens. +`) + h.query(n2, v2FlowTokensQueryStr) + + h.comment(` +-------------------------------------------------------------------------------- +-- (Upgrading n3 to v2 protocol with v1 encoding.) +-------------------------------------------------------------------------------- +`) + serverLevels[2].Store(kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 1 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + + h.comment(` +-- The v2 flow token metrics on n2. +-- The 3 MiB of elastic tokens should be returned. +`) + h.query(n2, v2FlowTokensQueryStr) + + h.comment(`-- (Block below-raft admission.)`) + disableWorkQueueGranting.Store(true) + + h.comment(`-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri, 1 /* serverIdx */) + + h.comment(` +-- The v2 tracked tokens per-stream on n2 should be 1 MiB. +`) + h.query(n2, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-------------------------------------------------------------------------------- +-- (Upgrading n1 to v2 protocol with v2 encoding.) +-------------------------------------------------------------------------------- +`) + serverLevels[0].Store(kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + + h.comment(` +-- The v2 tracked tokens per-stream on n2 should still be 1 MiB. +`) + h.query(n2, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 1 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + + h.comment(` +-- There should no longer be any tracked tokens on n2, as admission occurs. +`) + h.query(n2, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Corresponding v2 token metrics on n2. All tokens should be returned. +`) + h.query(n2, v2FlowTokensQueryStr) + + h.comment(`-- (Block below-raft admission.)`) + disableWorkQueueGranting.Store(true) + + h.comment(`-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri, 1 /* serverIdx */) + + h.comment(` +-- Corresponding v2 token metrics on n2. The 3 x 1 MiB = 3 MiB write +-- should be reflected. +`) + h.query(n2, v2FlowTokensQueryStr) + + h.comment(`-- (Transferring range lease back to n1.)`) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) + testutils.SucceedsSoon(t, func() error { + if leader := tc.GetRaftLeader(t, roachpb.RKey(k)); leader.NodeID() != tc.Target(0).NodeID { + return errors.Errorf("expected raft leadership to transfer to n1, found n%d", leader.NodeID()) + } + return nil + }) + h.waitForAllTokensReturned(ctx, 3, 1 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV1Encoding) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + + h.comment(` +-- There should no longer be any tracked tokens on n2, as it's no longer the +-- leader. +`) + h.query(n2, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Corresponding v2 token metrics on n2. All tokens should be returned. +`) + h.query(n2, v2FlowTokensQueryStr) + + h.comment(` +-- Viewing n1's v2 connected streams, there now should be three, as n1 acquired +-- the leadership and lease. +`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- The v2 tracked tokens per-stream on n1. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + h.comment(` +-- Corresponding v2 token metrics on n1. +-- All tokens should be returned via admission. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- (Block below-raft admission.)`) + disableWorkQueueGranting.Store(true) + + h.comment(`-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Corresponding v2 token metrics on n1. +-- The 3 x 1 MiB replicated write should be deducted. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- The v2 tracked tokens per-stream on n1. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- The v1 tracked tokens per-stream on n1. +-- There should be no tokens tracked. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-------------------------------------------------------------------------------- +-- (Upgrading n2 and n3 to v2 protocol with v2 encoding.) +-------------------------------------------------------------------------------- +`) + serverLevels[1].Store(kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + serverLevels[2].Store(kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + + h.comment(`-- (Issuing 2x1MiB regular, 3x replicated write that's admitted.)`) + h.put(ctx, k, 2<<20 /* 2MiB */, admissionpb.NormalPri) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + h.comment(` +-- Corresponding v2 token metrics on n1. +-- The 3 x 2 MiB replicated write should be deducted and returned. +`) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(` +-- (Transferring range lease to n3, running v2 protocol with v2 encoding, +-- and allowing leadership to follow.) +`) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(2)) + testutils.SucceedsSoon(t, func() error { + if leader := tc.GetRaftLeader(t, roachpb.RKey(k)); leader.NodeID() != tc.Target(2).NodeID { + return errors.Errorf("expected raft leadership to transfer to n2, found n%d", leader.NodeID()) + } + return nil + }) + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + h.waitForConnectedStreams(ctx, desc.RangeID, 3, 2 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + + h.comment(`-- (Issuing 1x1MiB regular, 3x replicated write that's admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri, 2 /* serverIdx */) + h.waitForAllTokensReturned(ctx, 3, 2 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + + // Ensure that there are no outstanding tokens in either protocol after + // allowing admission one last time. + // + // Note n3 was never the leader while having the v1 protocol enabled, only + // v2. + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2NotEnabledWhenLeader) + h.waitForAllTokensReturned(ctx, 3, 1 /* serverIdx */, kvflowcontrol.V2NotEnabledWhenLeader) + h.waitForAllTokensReturned(ctx, 0, 2 /* serverIdx */, kvflowcontrol.V2NotEnabledWhenLeader) + // Note all three nodes were the leader while having the v2 protocol enabled. + h.waitForAllTokensReturned(ctx, 3, 0 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + h.waitForAllTokensReturned(ctx, 3, 1 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + h.waitForAllTokensReturned(ctx, 3, 2 /* serverIdx */, kvflowcontrol.V2EnabledWhenLeaderV2Encoding) + + h.comment(` +-- The v1 and v2 flow token metrics on n3. +-- The 3 x 1 MiB write should have been deducted and returned. +`) + h.query(n3, v1FlowTokensQueryStr) + h.query(n3, v2FlowTokensQueryStr) + + h.comment(`-- The v1 and v2 flow token metrics on n1.`) + h.query(n1, v1FlowTokensQueryStr) + h.query(n1, v2FlowTokensQueryStr) + + h.comment(`-- The v1 and v2 flow token metrics on n2.`) + h.query(n2, v1FlowTokensQueryStr) + h.query(n2, v2FlowTokensQueryStr) +} + type flowControlTestHelper struct { t *testing.T tc *testcluster.TestCluster @@ -3880,6 +4505,7 @@ func newFlowControlTestHelperV1(t *testing.T, tc *testcluster.TestCluster) *flow return newFlowControlTestHelper(t, tc, "flow_control_integration", /* testdata */ + kvflowcontrol.V2NotEnabledWhenLeader, true, /* isStatic */ ) @@ -3948,9 +4574,10 @@ func (h *flowControlTestHelper) checkAllTokensReturned( typName string, ) error { if actualTokens != expTokens { - return fmt.Errorf("expected %v of %s flow tokens for %v, got %v", + return fmt.Errorf("expected %v of %s flow tokens for %v, got %v [%+v]", humanize.IBytes(uint64(expTokens)), typName, stream, humanize.IBytes(uint64(actualTokens)), + level, ) } return nil diff --git a/pkg/kv/kvserver/testdata/flow_control_integration_v2/v1_to_v2_transition b/pkg/kv/kvserver/testdata/flow_control_integration_v2/v1_to_v2_transition new file mode 100644 index 000000000000..51a5444f78b9 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/v1_to_v2_transition @@ -0,0 +1,1031 @@ +echo +---- +---- +-- This test exercises the transition from replication flow control: +-- - v1 protocol with v1 encoding => +-- - v2 protocol with v1 encoding => +-- - v2 protocol with v2 encoding +-- The test is structured as follows: +-- (1) Start n1, n2, n3 with v1 protocol and v1 encoding. +-- (2) Upgrade n1 to v2 protocol with v1 encoding. +-- (3) Transfer the range lease to n2. +-- (4) Upgrade n2 to v2 protocol with v1 encoding. +-- (5) Upgrade n3 to v2 protocol with v1 encoding. +-- (6) Upgrade n1 to v2 protocol with v2 encoding. +-- (7) Transfer the range lease to n1. +-- (8) Upgrade n2,n3 to v2 protocol with v2 encoding. +-- (9) Transfer the range lease to n3. +-- Between each step, we issue writes, (un)block admission and observe the +-- flow control metrics and vtables. +-- +-- Start by checking that the leader (n1) has 3 connected v1 streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.) + + +-- The v1 flow token metrics, there should be 3x1 MiB = 3 MiB of tokens deducted. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 21 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 45 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- The v2 flow token metrics, there should be no tokens or deductions. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 0 B + kvflowcontrol.tokens.eval.elastic.deducted | 0 B + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 0 B + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 0 B + kvflowcontrol.tokens.send.elastic.deducted | 0 B + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 0 B + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- The v1 tracked tokens per-stream on n1 should be 1 MiB for (s1,s2,s3). +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- The v1 flow token metrics on n1, there should be 3x1 MiB = 3 MiB of tokens deducted +-- and returned now. With all tokens available. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 3.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- The v1 tracked tokens per-stream on n1 should now be 0. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 0 B + 70 | 2 | 0 B + 70 | 3 | 0 B + + +-- (Block below-raft admission again.) + + +-- (Issuing 1 x 1MiB regular, 3x replicated write that's not admitted.) + + +-- The v1 tracked tokens per-stream on n1 should again be 1 MiB for (s1,s2,s3). +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + + +-------------------------------------------------------------------------------- +-- (Upgrading n1 to v2 protocol with v1 encoding.) +-------------------------------------------------------------------------------- + + +-- Viewing the range's v2 connected streams, there now should be three. +-- These are lazily instantiated on the first raft event the leader +-- RangeController sees. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- There should also now be no connected streams for the v1 protocol, +-- at the leader n1. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + + +-- The v1 flow token metrics, all deducted tokens should be returned after +-- the leader switches to the rac2 protocol. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 6.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 6.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 6.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 6.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Issuing 1x2MiB regular, 3x replicated write that's not admitted.) + + +-- The v2 flow token metrics, the 3 MiB of earlier token deductions from v1 are dropped. +-- Expect 3 * 2 MiB = 6 MiB of deductions, from the most recent write. +-- Note that the v2 protocol with v1 encoding will only ever deduct elastic tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 18 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 18 MiB + kvflowcontrol.tokens.send.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- The v2 tracked tokens per-stream on n1 should now also be 2 MiB for (s1,s2,s3). +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 2.0 MiB + 70 | 2 | 2.0 MiB + 70 | 3 | 2.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- The v2 flow token metrics. The 6 MiB of tokens should be returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Block below-raft admission again.) + + +-- (Issuing 1 x 1MiB regular, 3x replicated write that's not admitted.) + + +-- The v2 tracked tokens per-stream on n1 reflect the most recent write +-- and should be 1 MiB per stream now. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + + +-- There should also be a corresponding elastic token deduction (not regular), +-- as v2 protocol with v1 encoding will only ever deduct elastic tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Transferring range lease to n2 (running v1) and allowing leadership to follow.) + + +-- The v2 flow token metrics from n1 having lost the lease and raft leadership. +-- All deducted tokens are returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Now expect to see 3 connected v1 streams on n2. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- (Issuing 1 x 3MiB elastic, 3x replicated write that's not admitted.) + + +-- The v1 tracked tokens per-stream on n2 should be 3 MiB. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 3.0 MiB + 70 | 2 | 3.0 MiB + 70 | 3 | 3.0 MiB + + +-- Corresponding v1 token metrics on the new leader n2. +-- These should reflect the 3 x 3 MiB = 9 MiB write. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 15 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 9.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 39 MiB + kvadmission.flow_controller.regular_tokens_deducted | 9.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Corresponding v2 token metrics on the new leader n2. +-- These should be unpopulated, similar to when n1 was first the leader. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 0 B + kvflowcontrol.tokens.eval.elastic.deducted | 0 B + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 0 B + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 0 B + kvflowcontrol.tokens.send.elastic.deducted | 0 B + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 0 B + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Allow below-raft admission to proceed.) + + +-- The v1 token metrics on the new leader n2 should now reflect +-- the 9 MiB write and admission, all tokens should be returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 9.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 9.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 9.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 9.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Issuing 1 x 1MiB regular, 3x replicated write that's admitted.) + + +-- The v1 token metrics on the new leader n2 should now also reflect +-- the 9 + 3 = 12 MiB write and admission, all tokens should be returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 12 MiB + kvadmission.flow_controller.elastic_tokens_returned | 12 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 12 MiB + kvadmission.flow_controller.regular_tokens_returned | 12 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Block below-raft admission.) + + +-- (Issuing 1 x 4MiB regular, 3x replicated write that's not admitted.) + + +-- The v1 tracked tokens per-stream on n2 should be 4 MiB. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 4.0 MiB + 70 | 2 | 4.0 MiB + 70 | 3 | 4.0 MiB + + +-- Corresponding v1 token metrics. +-- These should reflect the 3 x 4 MiB = 12 MiB write. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 12 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 24 MiB + kvadmission.flow_controller.elastic_tokens_returned | 12 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 36 MiB + kvadmission.flow_controller.regular_tokens_deducted | 24 MiB + kvadmission.flow_controller.regular_tokens_returned | 12 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-------------------------------------------------------------------------------- +-- (Upgrading n2 to v2 protocol with v1 encoding.) +-------------------------------------------------------------------------------- + + +-- (Issuing another 1x1MiB regular, 3x replicated write that's not admitted.) + + +-- Corresponding v1 token metrics on the new leader n2. +-- All tokens should be returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 24 MiB + kvadmission.flow_controller.elastic_tokens_returned | 24 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 24 MiB + kvadmission.flow_controller.regular_tokens_returned | 24 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Also expect to see 0 connected v1 streams on n2. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + + +-- There should be 3 connected streams on n2 for the v2 protocol. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- Corresponding v2 token metrics on the new leader n2. The most recent +-- 3 x 1 MiB = 3 MiB write should be reflected in the token deductions. +-- Recall that v2 protocol with v1 encoding will only ever deduct elastic tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 0 B + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 0 B + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-------------------------------------------------------------------------------- +-- (Upgrading n3 to v2 protocol with v1 encoding.) +-------------------------------------------------------------------------------- + + +-- (Allow below-raft admission to proceed.) + + +-- The v2 flow token metrics on n2. +-- The 3 MiB of elastic tokens should be returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Block below-raft admission.) + + +-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.) + + +-- The v2 tracked tokens per-stream on n2 should be 1 MiB. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + + +-------------------------------------------------------------------------------- +-- (Upgrading n1 to v2 protocol with v2 encoding.) +-------------------------------------------------------------------------------- + + +-- The v2 tracked tokens per-stream on n2 should still be 1 MiB. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- There should no longer be any tracked tokens on n2, as admission occurs. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 0 B + 70 | 2 | 0 B + 70 | 3 | 0 B + + +-- Corresponding v2 token metrics on n2. All tokens should be returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 6.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Block below-raft admission.) + + +-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.) + + +-- Corresponding v2 token metrics on n2. The 3 x 1 MiB = 3 MiB write +-- should be reflected. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 6.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Transferring range lease back to n1.) + + +-- There should no longer be any tracked tokens on n2, as it's no longer the +-- leader. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + + +-- Corresponding v2 token metrics on n2. All tokens should be returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- Viewing n1's v2 connected streams, there now should be three, as n1 acquired +-- the leadership and lease. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles_v2 +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 70 | 3 + + +-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.) + + +-- The v2 tracked tokens per-stream on n1. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Corresponding v2 token metrics on n1. +-- All tokens should be returned via admission. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 12 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 12 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 12 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 12 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Block below-raft admission.) + + +-- (Issuing 1x1MiB regular, 3x replicated write that's not admitted.) + + +-- Corresponding v2 token metrics on n1. +-- The 3 x 1 MiB replicated write should be deducted. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 21 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 15 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 12 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 45 MiB + kvflowcontrol.tokens.eval.regular.deducted | 6.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 21 MiB + kvflowcontrol.tokens.send.elastic.deducted | 15 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 12 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 45 MiB + kvflowcontrol.tokens.send.regular.deducted | 6.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- The v2 tracked tokens per-stream on n1. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles_v2 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 70 | 1 | 1.0 MiB + 70 | 2 | 1.0 MiB + 70 | 3 | 1.0 MiB + + +-- The v1 tracked tokens per-stream on n1. +-- There should be no tokens tracked. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + + +-------------------------------------------------------------------------------- +-- (Upgrading n2 and n3 to v2 protocol with v2 encoding.) +-------------------------------------------------------------------------------- + + +-- (Allow below-raft admission to proceed.) + + +-- (Issuing 2x1MiB regular, 3x replicated write that's admitted.) + + +-- Corresponding v2 token metrics on n1. +-- The 3 x 2 MiB replicated write should be deducted and returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 21 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 21 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 12 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 12 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 21 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 21 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 12 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 12 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- (Transferring range lease to n3, running v2 protocol with v2 encoding, +-- and allowing leadership to follow.) + + +-- (Issuing 1x1MiB regular, 3x replicated write that's admitted.) + + +-- The v1 and v2 flow token metrics on n3. +-- The 3 x 1 MiB write should have been deducted and returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 0 B + kvadmission.flow_controller.elastic_tokens_deducted | 0 B + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 0 B + kvadmission.flow_controller.regular_tokens_deducted | 0 B + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 0 B + kvflowcontrol.tokens.eval.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 3.0 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 3.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 0 B + kvflowcontrol.tokens.send.elastic.returned | 3.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 3.0 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 3.0 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- The v1 and v2 flow token metrics on n1. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 6.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 6.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 6.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 6.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 21 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 21 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 12 MiB + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 12 MiB + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 21 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 21 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 12 MiB + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 12 MiB + kvflowcontrol.tokens.send.regular.unaccounted | 0 B + + +-- The v1 and v2 flow token metrics on n2. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 24 MiB + kvadmission.flow_controller.elastic_tokens_returned | 24 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 24 MiB + kvadmission.flow_controller.regular_tokens_returned | 24 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvflowcontrol%tokens%' +ORDER BY name ASC; + + kvflowcontrol.tokens.eval.elastic.available | 24 MiB + kvflowcontrol.tokens.eval.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.eval.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.eval.elastic.unaccounted | 0 B + kvflowcontrol.tokens.eval.regular.available | 48 MiB + kvflowcontrol.tokens.eval.regular.deducted | 0 B + kvflowcontrol.tokens.eval.regular.disconnected | 0 B + kvflowcontrol.tokens.eval.regular.returned | 0 B + kvflowcontrol.tokens.eval.regular.unaccounted | 0 B + kvflowcontrol.tokens.send.elastic.available | 24 MiB + kvflowcontrol.tokens.send.elastic.deducted | 9.0 MiB + kvflowcontrol.tokens.send.elastic.disconnected | 3.0 MiB + kvflowcontrol.tokens.send.elastic.returned | 9.0 MiB + kvflowcontrol.tokens.send.elastic.unaccounted | 0 B + kvflowcontrol.tokens.send.regular.available | 48 MiB + kvflowcontrol.tokens.send.regular.deducted | 0 B + kvflowcontrol.tokens.send.regular.disconnected | 0 B + kvflowcontrol.tokens.send.regular.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql From 8ad49fdb75651eb7eb6cb51f3abed472bb039b74 Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Wed, 25 Sep 2024 11:54:57 -0500 Subject: [PATCH 8/8] upgrades: give test an additional core under remote exec This has been timing out. Epic: none Release note: None --- pkg/upgrade/upgrades/BUILD.bazel | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index f69e21b44091..51938e35195b 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -99,6 +99,7 @@ go_test( ], data = glob(["testdata/**"]), embed = [":upgrades"], + exec_properties = {"test.Pool": "large"}, shard_count = 16, deps = [ "//pkg/base",