Skip to content

Commit

Permalink
kv: remove kv.snapshot_recovery.max_rate setting
Browse files Browse the repository at this point in the history
Fixes: cockroachdb#63728

There were two settings that controlled snapshot transfer rates, however
this has caused numerous problems when they are set differently as
the code makes timeout assumptions based on this rate.

Epic: none

Release note: This change removes a user configurable setting
`kv.snapshot_recovery.max_rate`. Guidance to customers has been to
always set this equal to `kv.snapshot_rebalance.max_rate` and this
change will start using that setting for all snapshots. If the customer
previously set `kv.snapshot_recovery.max_rate` it will be cleared after
this is released and future attempts to set it will fail with:
"ERROR: unknown cluster setting 'kv.snapshot_recovery.max_rate'
  • Loading branch information
andrewbaptist committed Apr 28, 2023
1 parent e875499 commit f39f9db
Show file tree
Hide file tree
Showing 15 changed files with 19 additions and 147 deletions.
1 change: 0 additions & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,6 @@
<tr><td><div id="setting-kv-replica-stats-addsst-request-size-factor" class="anchored"><code>kv.replica_stats.addsst_request_size_factor</code></div></td><td>integer</td><td><code>50000</code></td><td>the divisor that is applied to addsstable request sizes, then recorded in a leaseholders QPS; 0 means all requests are treated as cost 1</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-replication-reports-interval" class="anchored"><code>kv.replication_reports.interval</code></div></td><td>duration</td><td><code>1m0s</code></td><td>the frequency for generating the replication_constraint_stats, replication_stats_report and replication_critical_localities reports (set to 0 to disable)</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-snapshot-rebalance-max-rate" class="anchored"><code>kv.snapshot_rebalance.max_rate</code></div></td><td>byte size</td><td><code>32 MiB</code></td><td>the rate limit (bytes/sec) to use for rebalance and upreplication snapshots</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-snapshot-recovery-max-rate" class="anchored"><code>kv.snapshot_recovery.max_rate</code></div></td><td>byte size</td><td><code>32 MiB</code></td><td>the rate limit (bytes/sec) to use for recovery snapshots</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-transaction-max-intents-bytes" class="anchored"><code>kv.transaction.max_intents_bytes</code></div></td><td>integer</td><td><code>4194304</code></td><td>maximum number of bytes used to track locks in transactions</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-transaction-max-refresh-spans-bytes" class="anchored"><code>kv.transaction.max_refresh_spans_bytes</code></div></td><td>integer</td><td><code>4194304</code></td><td>maximum number of bytes used to track refresh spans in serializable transactions</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-transaction-reject-over-max-intents-budget-enabled" class="anchored"><code>kv.transaction.reject_over_max_intents_budget.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if set, transactions that exceed their lock tracking budget (kv.transaction.max_intents_bytes) are rejected instead of having their lock spans imprecisely compressed</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
Expand Down
2 changes: 1 addition & 1 deletion docs/tech-notes/rebalancing.md
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ are:
up-replicate or rebalance, because their snapshots can't complete in time
and just get canceled on every attempt. This shouldn't happen with the
default settings of `kv.snapshot_rebalance.max_rate`,
`kv.snapshot_recovery.max_rate`, and `ZoneConfig.RangeMaxBytes`, but
and `ZoneConfig.RangeMaxBytes`, but
modifications to one or more of them can put a cluster in danger.
* We limit lease transfers away from each node to one per second. This is a
very long-standing policy that hasn't been reconsidered in a long time, but
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,10 +80,6 @@ func registerSnapshotOverload(r registry.Registry) {
ctx, "SET CLUSTER SETTING kv.snapshot_rebalance.max_rate = '256MiB'"); err != nil {
t.Fatalf("failed to set kv.snapshot_rebalance.max_rate: %v", err)
}
if _, err := db.ExecContext(
ctx, "SET CLUSTER SETTING kv.snapshot_recovery.max_rate = '256MiB'"); err != nil {
t.Fatalf("failed to set kv.snapshot_recovery.max_rate: %v", err)
}
}

t.Status(fmt.Sprintf("setting up prometheus/grafana (<%s)", 2*time.Minute))
Expand Down
4 changes: 0 additions & 4 deletions pkg/cmd/roachtest/tests/decommission.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,6 @@ func runDrainAndDecommission(

// Speed up the decommissioning.
run(`SET CLUSTER SETTING kv.snapshot_rebalance.max_rate='2GiB'`)
run(`SET CLUSTER SETTING kv.snapshot_recovery.max_rate='2GiB'`)

// Wait for initial up-replication.
err := WaitForReplication(ctx, t, db, defaultReplicationFactor)
Expand Down Expand Up @@ -999,8 +998,6 @@ func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster)
// Increase the speed of decommissioning.
err = run(db, `SET CLUSTER SETTING kv.snapshot_rebalance.max_rate='2GiB'`)
require.NoError(t, err)
err = run(db, `SET CLUSTER SETTING kv.snapshot_recovery.max_rate='2GiB'`)
require.NoError(t, err)

// Wait for initial up-replication.
err := WaitFor3XReplication(ctx, t, db)
Expand Down Expand Up @@ -1095,7 +1092,6 @@ func runDecommissionSlow(ctx context.Context, t test.Test, c cluster.Cluster) {

// Increase the speed of decommissioning.
run(db, `SET CLUSTER SETTING kv.snapshot_rebalance.max_rate='2GiB'`)
run(db, `SET CLUSTER SETTING kv.snapshot_recovery.max_rate='2GiB'`)

// Wait for initial up-replication.
err := WaitForReplication(ctx, t, db, replicationFactor)
Expand Down
4 changes: 1 addition & 3 deletions pkg/cmd/roachtest/tests/decommissionbench.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ const (
bytesUsedMetric = "targetBytesUsed"

// Used to calculate estimated decommission time. Should remain in sync with
// setting `kv.snapshot_recovery.max_rate` in store_snapshot.go.
// setting `kv.snapshot_rebalance.max_rate` in store_snapshot.go.
defaultSnapshotRateMb = 32

// Skip message for tests not meant to be run nightly.
Expand Down Expand Up @@ -391,8 +391,6 @@ func setupDecommissionBench(
for _, stmt := range []string{
fmt.Sprintf(`SET CLUSTER SETTING kv.snapshot_rebalance.max_rate='%dMiB'`,
benchSpec.snapshotRate),
fmt.Sprintf(`SET CLUSTER SETTING kv.snapshot_recovery.max_rate='%dMiB'`,
benchSpec.snapshotRate),
} {
t.Status(stmt)
_, err := db.ExecContext(ctx, stmt)
Expand Down
2 changes: 0 additions & 2 deletions pkg/cmd/roachtest/tests/loss_of_quorum_recovery.go
Original file line number Diff line number Diff line change
Expand Up @@ -604,12 +604,10 @@ func setDBRangeLimits(ctx context.Context, db *gosql.DB, dbName string, size int
func setSnapshotRate(ctx context.Context, db *gosql.DB, sizeMB int64) error {
queries := []string{
"RESET CLUSTER SETTING kv.snapshot_rebalance.max_rate",
"RESET CLUSTER SETTING kv.snapshot_recovery.max_rate",
}
if sizeMB > 0 {
queries = []string{
fmt.Sprintf("SET CLUSTER SETTING kv.snapshot_rebalance.max_rate = '%dMiB'", sizeMB),
fmt.Sprintf("SET CLUSTER SETTING kv.snapshot_recovery.max_rate = '%dMiB'", sizeMB),
}
}
for _, query := range queries {
Expand Down
3 changes: 0 additions & 3 deletions pkg/cmd/roachtest/tests/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -309,9 +309,6 @@ func runLargeRangeSplits(ctx context.Context, t test.Test, c cluster.Cluster, si
if _, err := db.ExecContext(ctx, `SET CLUSTER SETTING kv.snapshot_rebalance.max_rate='512MiB'`); err != nil {
return err
}
if _, err := db.ExecContext(ctx, `SET CLUSTER SETTING kv.snapshot_recovery.max_rate='512MiB'`); err != nil {
return err
}
// Set the range size to a multiple of what we expect the size of the
// bank table to be. This should result in the table fitting
// inside a single range.
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/merge_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,7 @@ func newMergeQueue(store *Store, db *kv.DB) *mergeQueue {
// hard to determine ahead of time. An alternative would be to calculate
// the timeout with a function that additionally considers the replication
// factor.
processTimeoutFunc: makeRateLimitedTimeoutFunc(rebalanceSnapshotRate, recoverySnapshotRate),
processTimeoutFunc: makeRateLimitedTimeoutFunc(rebalanceSnapshotRate),
needsLease: true,
needsSpanConfigs: true,
acceptsUnsplitRanges: false,
Expand Down
16 changes: 5 additions & 11 deletions pkg/kv/kvserver/queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,8 +78,8 @@ func defaultProcessTimeoutFunc(cs *cluster.Settings, _ replicaInQueue) time.Dura
// after them in the queue to time-out.
//
// The parameter controls which rate(s) to use.
func makeRateLimitedTimeoutFunc(rateSettings ...*settings.ByteSizeSetting) queueProcessTimeoutFunc {
return makeRateLimitedTimeoutFuncByPermittedSlowdown(permittedRangeScanSlowdown, rateSettings...)
func makeRateLimitedTimeoutFunc(rateSettings *settings.ByteSizeSetting) queueProcessTimeoutFunc {
return makeRateLimitedTimeoutFuncByPermittedSlowdown(permittedRangeScanSlowdown, rateSettings)
}

// permittedRangeScanSlowdown is the factor of the above the estimated duration
Expand All @@ -91,24 +91,18 @@ const permittedRangeScanSlowdown = 10
// slowdown factor on the estimated queue processing duration based on the given rate settings.
// See makeRateLimitedTimeoutFunc for more information.
func makeRateLimitedTimeoutFuncByPermittedSlowdown(
permittedSlowdown int, rateSettings ...*settings.ByteSizeSetting,
permittedSlowdown int, rateSettings *settings.ByteSizeSetting,
) queueProcessTimeoutFunc {
return func(cs *cluster.Settings, r replicaInQueue) time.Duration {
minimumTimeout := queueGuaranteedProcessingTimeBudget.Get(&cs.SV)
// NB: In production code this will type assertion will always succeed.
// Some tests set up a fake implementation of replicaInQueue in which
// case we fall back to the configured minimum timeout.
repl, ok := r.(interface{ GetMVCCStats() enginepb.MVCCStats })
if !ok || len(rateSettings) == 0 {
if !ok {
return minimumTimeout
}
minSnapshotRate := rateSettings[0].Get(&cs.SV)
for i := 1; i < len(rateSettings); i++ {
snapshotRate := rateSettings[i].Get(&cs.SV)
if snapshotRate < minSnapshotRate {
minSnapshotRate = snapshotRate
}
}
minSnapshotRate := rateSettings.Get(&cs.SV)
estimatedDuration := time.Duration(repl.GetMVCCStats().Total()/minSnapshotRate) * time.Second
timeout := estimatedDuration * time.Duration(permittedSlowdown)
if timeout < minimumTimeout {
Expand Down
34 changes: 3 additions & 31 deletions pkg/kv/kvserver/queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1001,7 +1001,6 @@ func TestQueueRateLimitedTimeoutFunc(t *testing.T) {
ctx := context.Background()
type testCase struct {
guaranteedProcessingTime time.Duration
recoverySnapshotRate int64 // bytes/s
rebalanceSnapshotRate int64 // bytes/s
replicaSize int64 // bytes
expectedTimeout time.Duration
Expand All @@ -1010,9 +1009,8 @@ func TestQueueRateLimitedTimeoutFunc(t *testing.T) {
return fmt.Sprintf("%+v", tc), func(t *testing.T) {
st := cluster.MakeTestingClusterSettings()
queueGuaranteedProcessingTimeBudget.Override(ctx, &st.SV, tc.guaranteedProcessingTime)
recoverySnapshotRate.Override(ctx, &st.SV, tc.recoverySnapshotRate)
rebalanceSnapshotRate.Override(ctx, &st.SV, tc.rebalanceSnapshotRate)
tf := makeRateLimitedTimeoutFunc(recoverySnapshotRate, rebalanceSnapshotRate)
tf := makeRateLimitedTimeoutFunc(rebalanceSnapshotRate)
repl := mvccStatsReplicaInQueue{
size: tc.replicaSize,
}
Expand All @@ -1022,56 +1020,30 @@ func TestQueueRateLimitedTimeoutFunc(t *testing.T) {
for _, tc := range []testCase{
{
guaranteedProcessingTime: time.Minute,
recoverySnapshotRate: 1 << 30,
rebalanceSnapshotRate: 1 << 20, // minimum rate for timeout calculation.
replicaSize: 1 << 20,
expectedTimeout: time.Minute, // the minimum timeout (guaranteedProcessingTime).
},
{
guaranteedProcessingTime: time.Minute,
recoverySnapshotRate: 1 << 20, // minimum rate for timeout calculation.
rebalanceSnapshotRate: 1 << 30,
rebalanceSnapshotRate: 1 << 20,
replicaSize: 1 << 20,
expectedTimeout: time.Minute, // the minimum timeout (guaranteedProcessingTime).
},
{
guaranteedProcessingTime: time.Minute,
recoverySnapshotRate: 1 << 20, // minimum rate for timeout calculation.
rebalanceSnapshotRate: 2 << 20,
replicaSize: 100 << 20,
expectedTimeout: 100 * time.Second * permittedRangeScanSlowdown,
},
{
guaranteedProcessingTime: time.Minute,
recoverySnapshotRate: 2 << 20,
rebalanceSnapshotRate: 1 << 20, // minimum rate for timeout calculation.
replicaSize: 100 << 20,
expectedTimeout: 100 * time.Second * permittedRangeScanSlowdown,
},
{
guaranteedProcessingTime: time.Hour,
recoverySnapshotRate: 1 << 20, // minimum rate for timeout calculation.
rebalanceSnapshotRate: 1 << 30,
replicaSize: 100 << 20,
expectedTimeout: time.Hour, // the minimum timeout (guaranteedProcessingTime).
},
{
guaranteedProcessingTime: time.Hour,
recoverySnapshotRate: 1 << 30,
rebalanceSnapshotRate: 1 << 20, // minimum rate for timeout calculation.
replicaSize: 100 << 20,
expectedTimeout: time.Hour, // the minimum timeout (guaranteedProcessingTime).
},
{
guaranteedProcessingTime: time.Minute,
recoverySnapshotRate: 1 << 10, // minimum rate for timeout calculation.
rebalanceSnapshotRate: 1 << 20,
rebalanceSnapshotRate: 1 << 10,
replicaSize: 100 << 20,
expectedTimeout: 100 * (1 << 10) * time.Second * permittedRangeScanSlowdown,
},
{
guaranteedProcessingTime: time.Minute,
recoverySnapshotRate: 1 << 20,
rebalanceSnapshotRate: 1 << 10, // minimum rate for timeout calculation.
replicaSize: 100 << 20,
expectedTimeout: 100 * (1 << 10) * time.Second * permittedRangeScanSlowdown,
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/raft_snapshot_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func newRaftSnapshotQueue(store *Store) *raftSnapshotQueue {
needsLease: false,
needsSpanConfigs: false,
acceptsUnsplitRanges: true,
processTimeoutFunc: makeRateLimitedTimeoutFunc(recoverySnapshotRate, rebalanceSnapshotRate),
processTimeoutFunc: makeRateLimitedTimeoutFunc(rebalanceSnapshotRate),
successes: store.metrics.RaftSnapshotQueueSuccesses,
failures: store.metrics.RaftSnapshotQueueFailures,
pending: store.metrics.RaftSnapshotQueuePending,
Expand Down
9 changes: 0 additions & 9 deletions pkg/kv/kvserver/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -2784,15 +2784,6 @@ func (r *Replica) sendSnapshotUsingDelegate(
return
}

// Don't send a queue name or priority if the receiver may not understand
// them or the setting is disabled. TODO(baptist): Remove the version flag in
// v23.1. Consider removing the cluster setting once we have verified this
// works as expected in all cases.
if !r.store.ClusterSettings().Version.IsActive(ctx, clusterversion.TODODelete_V22_2PrioritizeSnapshots) ||
!snapshotPrioritizationEnabled.Get(&r.store.ClusterSettings().SV) {
senderQueueName = 0
senderQueuePriority = 0
}
snapUUID := uuid.MakeV4()
appliedIndex, cleanup := r.addSnapshotLogTruncationConstraint(ctx, snapUUID, snapType == kvserverpb.SnapshotRequest_INITIAL, recipient.StoreID)
// The cleanup function needs to be called regardless of success or failure of
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replicate_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -559,7 +559,7 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica
purgCh: time.NewTicker(replicateQueuePurgatoryCheckInterval).C,
updateCh: make(chan time.Time, 1),
logTracesThresholdFunc: makeRateLimitedTimeoutFuncByPermittedSlowdown(
permittedRangeScanSlowdown/2, rebalanceSnapshotRate, recoverySnapshotRate,
permittedRangeScanSlowdown/2, rebalanceSnapshotRate,
),
}
store.metrics.registry.AddMetricStruct(&rq.metrics)
Expand All @@ -574,7 +574,7 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica
// so we use the raftSnapshotQueueTimeoutFunc. This function sets a
// timeout based on the range size and the sending rate in addition
// to consulting the setting which controls the minimum timeout.
processTimeoutFunc: makeRateLimitedTimeoutFunc(rebalanceSnapshotRate, recoverySnapshotRate),
processTimeoutFunc: makeRateLimitedTimeoutFunc(rebalanceSnapshotRate),
successes: store.metrics.ReplicateQueueSuccesses,
failures: store.metrics.ReplicateQueueFailures,
pending: store.metrics.ReplicateQueuePending,
Expand Down
55 changes: 2 additions & 53 deletions pkg/kv/kvserver/store_snapshot.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,16 +66,6 @@ const (
DefaultSnapshotApplyLimit = 1
)

// snapshotPrioritizationEnabled will allow the sender and receiver of snapshots
// to prioritize the snapshots. If disabled, the behavior will be FIFO on both
// send and receive sides.
var snapshotPrioritizationEnabled = settings.RegisterBoolSetting(
settings.SystemOnly,
"kv.snapshot_prioritization.enabled",
"if true, then prioritize enqueued snapshots on both the send or receive sides",
true,
)

// snapshotMetrics contains metrics on the number and size of snapshots in
// progress or in the snapshot queue.
type snapshotMetrics struct {
Expand Down Expand Up @@ -1172,9 +1162,7 @@ type SnapshotStorePool interface {
// limit of 1mb/s, a 512mb snapshot will take just under 9 minutes to send.
const minSnapshotRate = 1 << 20 // 1mb/s

// rebalanceSnapshotRate is the rate at which snapshots can be sent in the
// context of up-replication or rebalancing (i.e. any snapshot that was not
// requested by raft itself, to which `kv.snapshot_recovery.max_rate` applies).
// rebalanceSnapshotRate is the rate at which all snapshots are sent.
var rebalanceSnapshotRate = settings.RegisterByteSizeSetting(
settings.SystemOnly,
"kv.snapshot_rebalance.max_rate",
Expand All @@ -1189,29 +1177,6 @@ var rebalanceSnapshotRate = settings.RegisterByteSizeSetting(
},
).WithPublic()

// recoverySnapshotRate is the rate at which Raft-initiated snapshot can be
// sent. Ideally, one would never see a Raft-initiated snapshot; we'd like all
// replicas to start out as learners or via splits, and to never be cut off from
// the log. However, it has proved unfeasible to completely get rid of them.
//
// TODO(tbg): The existence of this rate, separate from rebalanceSnapshotRate,
// does not make a whole lot of sense. Both sources of snapshots compete thanks
// to a semaphore at the receiver, and so the slower one ultimately determines
// the pace at which things can move along.
var recoverySnapshotRate = settings.RegisterByteSizeSetting(
settings.SystemOnly,
"kv.snapshot_recovery.max_rate",
"the rate limit (bytes/sec) to use for recovery snapshots",
32<<20, // 32mb/s
func(v int64) error {
if v < minSnapshotRate {
return errors.Errorf("snapshot rate cannot be set to a value below %s: %s",
humanizeutil.IBytes(minSnapshotRate), humanizeutil.IBytes(v))
}
return nil
},
).WithPublic()

// snapshotSenderBatchSize is the size that key-value batches are allowed to
// grow to during Range snapshots before being sent to the receiver. This limit
// places an upper-bound on the memory footprint of the sender of a Range
Expand Down Expand Up @@ -1339,19 +1304,6 @@ var snapshotSSTWriteSyncRate = settings.RegisterByteSizeSetting(
settings.PositiveInt,
)

func snapshotRateLimit(
st *cluster.Settings, priority kvserverpb.SnapshotRequest_Priority,
) (rate.Limit, error) {
switch priority {
case kvserverpb.SnapshotRequest_RECOVERY:
return rate.Limit(recoverySnapshotRate.Get(&st.SV)), nil
case kvserverpb.SnapshotRequest_REBALANCE:
return rate.Limit(rebalanceSnapshotRate.Get(&st.SV)), nil
default:
return 0, errors.Errorf("unknown snapshot priority: %s", priority)
}
}

// SendEmptySnapshot creates an OutgoingSnapshot for the input range
// descriptor and seeds it with an empty range. Then, it sends this
// snapshot to the replica specified in the input.
Expand Down Expand Up @@ -1558,10 +1510,7 @@ func sendSnapshot(
start = timeutil.Now()

// Consult cluster settings to determine rate limits and batch sizes.
targetRate, err := snapshotRateLimit(st, header.Priority)
if err != nil {
return errors.Wrapf(err, "%s", to)
}
targetRate := rate.Limit(rebalanceSnapshotRate.Get(&st.SV))
batchSize := snapshotSenderBatchSize.Get(&st.SV)

// Convert the bytes/sec rate limit to batches/sec.
Expand Down
24 changes: 3 additions & 21 deletions pkg/kv/kvserver/store_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,6 @@ import (
"go.etcd.io/raft/v3"
"go.etcd.io/raft/v3/raftpb"
"golang.org/x/sync/errgroup"
"golang.org/x/time/rate"
)

var testIdent = roachpb.StoreIdent{
Expand Down Expand Up @@ -3394,26 +3393,9 @@ func TestSnapshotRateLimit(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

testCases := []struct {
priority kvserverpb.SnapshotRequest_Priority
expectedLimit rate.Limit
expectedErr string
}{
{kvserverpb.SnapshotRequest_UNKNOWN, 0, "unknown snapshot priority"},
{kvserverpb.SnapshotRequest_RECOVERY, 32 << 20, ""},
{kvserverpb.SnapshotRequest_REBALANCE, 32 << 20, ""},
}
for _, c := range testCases {
t.Run(c.priority.String(), func(t *testing.T) {
limit, err := snapshotRateLimit(cluster.MakeTestingClusterSettings(), c.priority)
if !testutils.IsError(err, c.expectedErr) {
t.Fatalf("expected \"%s\", but found %v", c.expectedErr, err)
}
if c.expectedLimit != limit {
t.Fatalf("expected %v, but found %v", c.expectedLimit, limit)
}
})
}
st := cluster.MakeTestingClusterSettings()
limit := rebalanceSnapshotRate.Get(&st.SV)
require.Equal(t, int64(32<<20), limit)
}

type mockSpanConfigReader struct {
Expand Down

0 comments on commit f39f9db

Please sign in to comment.