-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
kvserver: avoid replicating to followers that are I/O overloaded
This commit implements the stop-gap solution to raft traffic contributing to I/O overload that was discussed[^1] in #79215. The newly introduced `admission.kv.pause_replication_io_threshold` cluster setting can be used to define an I/O overload score above which raft leaders will attempt to pause replication to nonessential followers, to given them a chance at tidying up their LSM. A follower is "nonessential" if it is either a non-voter, or if we think (and are fairly certain that) quorum can be reached without it. If there are multiple overloaded followers, we pick as many as we can without losing quorum, and we try to do so in a way that's a) random, i.e. different raft leaders will pick a different set, to give each store a share of the relief, and b) stable, i.e. each raft leader will pick the same set at least for a little while, to avoid interrupting a quorum by rapidly switching the set of active followers (but see here[^2]). The implementation of this is as follows: - on Store, materialize (on each tick) a `storeOverloadMap` from gossip - each Replica, on tick, from this map compute the set of followers to pause, and store it in a per-Replica map. - consult this latter map - when sending `MsgApp` from `handleRaftReady` - when releasing proposal quota. This commit by default disables this new functionality by setting the cluster setting to zero. This has the effect of an empty `storeOverloadMap` and thus, after one round of gossip and subsequent tick (i.e. within seconds), an empty per-Replica paused followers map. Additionally, it's worth pointing out the mixed-version behavior: the old nodes' stores will be observed as gossiping a zero IOThreshold, which is considered not overloaded, i.e. replication streams to old nodes will never be paused. Fixes #79215. [^1]: #79215 (comment) [^2]: #83920 (comment) Release note (ops change): the `admission.kv.pause_replication_io_threshold` cluster setting can be set to a nonzero value to reduce I/O throughput on followers that are driven towards an inverted LSM by replication traffic. The functionality is disabled by default. A suggested value is 0.8, meaning that replication traffic to nonessential followers is paused before these followers will begin throttling their foreground traffic.
- Loading branch information
Showing
21 changed files
with
901 additions
and
35 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,101 @@ | ||
// Copyright 2022 The Cockroach Authors. | ||
// | ||
// Use of this software is governed by the Business Source License | ||
// included in the file licenses/BSL.txt. | ||
// | ||
// As of the Change Date specified in that file, in accordance with | ||
// the Business Source License, use of this software will be governed | ||
// by the Apache License, Version 2.0, included in the file | ||
// licenses/APL.txt. | ||
// | ||
|
||
package kvserver_test | ||
|
||
import ( | ||
"context" | ||
"sync/atomic" | ||
"testing" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/base" | ||
"github.com/cockroachdb/cockroach/pkg/kv/kvserver" | ||
"github.com/cockroachdb/cockroach/pkg/roachpb" | ||
"github.com/cockroachdb/cockroach/pkg/testutils" | ||
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster" | ||
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" | ||
"github.com/cockroachdb/cockroach/pkg/util/leaktest" | ||
"github.com/cockroachdb/cockroach/pkg/util/log" | ||
"github.com/cockroachdb/errors" | ||
"github.com/stretchr/testify/require" | ||
) | ||
|
||
// TestReplicaRaftOverload is an end-to-end test verifying that leaseholders | ||
// will "pause" followers that are on overloaded stores, and will unpause when | ||
// the overload ends. | ||
// | ||
// This primarily tests the gossip signal as well as the mechanics of pausing, | ||
// and does not check that "pausing" really results in no traffic being sent | ||
// to the followers. | ||
func TestReplicaRaftOverload(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
defer log.Scope(t).Close(t) | ||
ctx := context.Background() | ||
|
||
var on atomic.Value // bool | ||
on.Store(false) | ||
var args base.TestClusterArgs | ||
args.ReplicationMode = base.ReplicationManual | ||
args.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{StoreGossipIntercept: func(descriptor *roachpb.StoreDescriptor) { | ||
if !on.Load().(bool) || descriptor.StoreID != 3 { | ||
return | ||
} | ||
descriptor.Capacity.IOThreshold = admissionpb.IOThreshold{ | ||
L0NumSubLevels: 1000000, | ||
L0NumSubLevelsThreshold: 1, | ||
L0NumFiles: 1000000, | ||
L0NumFilesThreshold: 1, | ||
} | ||
}} | ||
tc := testcluster.StartTestCluster(t, 3, args) | ||
defer tc.Stopper().Stop(ctx) | ||
|
||
{ | ||
_, err := tc.ServerConn(0).Exec(`SET CLUSTER SETTING admission.kv.pause_replication_io_threshold = 1.0`) | ||
require.NoError(t, err) | ||
} | ||
k := tc.ScratchRange(t) | ||
tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) | ||
// Also split off another range that is on s3 so that we can verify that a | ||
// quiesced range doesn't do anything unexpected, like not release the paused | ||
// follower from its map (and thus messing with metrics when overload recovers). | ||
tc.SplitRangeOrFatal(t, k.Next()) | ||
|
||
// Gossip faux IO overload from s3. s1 should pick up on that and pause followers. | ||
on.Store(true) | ||
require.NoError(t, tc.GetFirstStoreFromServer(t, 2 /* n3 */).GossipStore(ctx, false /* useCached */)) | ||
testutils.SucceedsSoon(t, func() error { | ||
// Touch the one range that is on s3 since it's likely quiesced, and wouldn't unquiesce | ||
// if s3 becomes overloaded. Note that we do no such thing for the right sibling range, | ||
// so it may or may not contribute here (depending on when it quiesces). | ||
// | ||
// See: https://github.com/cockroachdb/cockroach/issues/84252 | ||
require.NoError(t, tc.Servers[0].DB().Put(ctx, tc.ScratchRange(t), "foo")) | ||
s1 := tc.GetFirstStoreFromServer(t, 0) | ||
require.NoError(t, s1.ComputeMetrics(ctx, 0 /* tick */)) | ||
if n := s1.Metrics().RaftPausedFollowerCount.Value(); n == 0 { | ||
return errors.New("no paused followers") | ||
} | ||
return nil | ||
}) | ||
|
||
// Now remove the gossip intercept and check again. The follower should un-pause immediately. | ||
on.Store(false) | ||
require.NoError(t, tc.GetFirstStoreFromServer(t, 2 /* n3 */).GossipStore(ctx, false /* useCached */)) | ||
testutils.SucceedsSoon(t, func() error { | ||
s1 := tc.GetFirstStoreFromServer(t, 0) | ||
require.NoError(t, s1.ComputeMetrics(ctx, 0 /* tick */)) | ||
if n := s1.Metrics().RaftPausedFollowerCount.Value(); n > 0 { | ||
return errors.Errorf("%d paused followers", n) | ||
} | ||
return nil | ||
}) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.