diff --git a/pkg/kv/kvserver/flow_control_integration_test.go b/pkg/kv/kvserver/flow_control_integration_test.go index 40ce02e26b7f..d910a218ef83 100644 --- a/pkg/kv/kvserver/flow_control_integration_test.go +++ b/pkg/kv/kvserver/flow_control_integration_test.go @@ -3834,6 +3834,618 @@ 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,n3 to v2 protocol with v1 encoding. +// (5) Upgrade n1 to v2 protocol with v2 encoding. +// (6) Transfer the range lease to n3. +// (7) Upgrade n2,n3 to v2 protocol with v2 encoding. +// +// Between each step, we issue writes and observe the flow control metrics. +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() + }, + }, + }, + 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. +-- (5) Upgrade n1 to v2 protocol with v2 encoding. +-- (6) Transfer the range lease to n1. +-- (7) Upgrade n2,n3 to v2 protocol with v2 encoding. +-- (8) 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.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 @@ -3870,6 +4482,7 @@ func newFlowControlTestHelperV1(t *testing.T, tc *testcluster.TestCluster) *flow return newFlowControlTestHelper(t, tc, "flow_control_integration", /* testdata */ + kvflowcontrol.V2NotEnabledWhenLeader, true, /* isStatic */ ) @@ -3938,9 +4551,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..991993b3e9b3 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration_v2/v1_to_v2_transition @@ -0,0 +1,963 @@ +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. +-- (5) Upgrade n1 to v2 protocol with v2 encoding. +-- (6) Transfer the range lease to n1. +-- (7) Upgrade n2,n3 to v2 protocol with v2 encoding. +-- (8) 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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.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.returned | 0 B + kvflowcontrol.tokens.send.regular.unaccounted | 0 B +---- +---- + +# vim:ft=sql