Skip to content

Commit

Permalink
kv: don't unquiesce uninitialized replicas
Browse files Browse the repository at this point in the history
In a [support issue](cockroachlabs/support#1340), we
saw that 10s of thousands of uninitialized replicas were being ticked regularly
and creating a large amount of background work on a node, driving up CPU. This
commit updates the Raft quiescence logic to disallow uninitialized replicas from
being unquiesced and Tick()'ing themselves.

Keeping uninitialized replicas quiesced even in the presence of Raft traffic
avoids wasted work. We could Tick() these replicas, but doing so is unnecessary
because uninitialized replicas can never win elections, so there is no reason
for them to ever call an election. In fact, uninitialized replicas do not even
know who their peers are, so there would be no way for them to call an election
or for them to send any other non-reactive message. As a result, all work
performed by an uninitialized replica is reactive and in response to incoming
messages (see processRequestQueue).

There are multiple ways for an uninitialized replica to be created and
then abandoned, and we don't do a good job garbage collecting them at a
later point (see #73424),
so it is important that they are cheap. Keeping them quiesced instead of
letting them unquiesce and tick every 200ms indefinitely avoids a
meaningful amount of periodic work for each uninitialized replica.

Release notes (bug fix): uninitialized replicas that are abandoned after an
unsuccessful snapshot no longer perform periodic background work, so they no
longer have a non-negligible cost.
  • Loading branch information
nvanbenschoten committed Jan 10, 2022
1 parent e0fb11a commit ad0a3ad
Show file tree
Hide file tree
Showing 5 changed files with 139 additions and 4 deletions.
72 changes: 72 additions & 0 deletions pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4087,6 +4087,78 @@ func TestRangeQuiescence(t *testing.T) {
}
}

// TestUninitializedReplicaRemainsQuiesced verifies that an uninitialized
// replica remains quiesced until it receives the snapshot that initializes it.
func TestUninitializedReplicaRemainsQuiesced(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
})
defer tc.Stopper().Stop(ctx)

_, desc, err := tc.Servers[0].ScratchRangeEx()
key := desc.StartKey.AsRawKey()
require.NoError(t, err)
require.NoError(t, tc.WaitForSplitAndInitialization(key))

// Block incoming snapshots on s2 until channel is signaled.
blockSnapshot := make(chan struct{})
handlerFuncs := noopRaftHandlerFuncs()
handlerFuncs.snapErr = func(header *kvserver.SnapshotRequest_Header) error {
select {
case <-blockSnapshot:
case <-tc.Stopper().ShouldQuiesce():
}
return nil
}
s2, err := tc.Server(1).GetStores().(*kvserver.Stores).GetStore(tc.Server(1).GetFirstStoreID())
require.NoError(t, err)
tc.Servers[1].RaftTransport().Listen(s2.StoreID(), &unreliableRaftHandler{
rangeID: desc.RangeID,
RaftMessageHandler: s2,
unreliableRaftHandlerFuncs: handlerFuncs,
})

// Try to up-replicate to s2. Should block on a learner snapshot after the new
// replica on s2 has been created, but before it has been initialized. While
// the replica is uninitialized, it should remain quiesced, even while it is
// receiving Raft traffic from the leader.
replicateErrChan := make(chan error)
go func() {
_, err := tc.AddVoters(key, tc.Target(1))
select {
case replicateErrChan <- err:
case <-tc.Stopper().ShouldQuiesce():
}
}()
testutils.SucceedsSoon(t, func() error {
repl, err := s2.GetReplica(desc.RangeID)
if err == nil {
// IMPORTANT: the replica should always be quiescent while uninitialized.
require.False(t, repl.IsInitialized())
require.True(t, repl.IsQuiescent())
}
return err
})

// Let the snapshot through. The up-replication attempt should succeed, the
// replica should now be initialized, and the replica should quiesce again.
close(blockSnapshot)
require.NoError(t, <-replicateErrChan)
repl, err := s2.GetReplica(desc.RangeID)
require.NoError(t, err)
require.True(t, repl.IsInitialized())
testutils.SucceedsSoon(t, func() error {
if !repl.IsQuiescent() {
return errors.Errorf("%s not quiescent", repl)
}
return nil
})
}

// TestInitRaftGroupOnRequest verifies that an uninitialized Raft group
// is initialized if a request is received, even if the current range
// lease points to a different replica.
Expand Down
9 changes: 9 additions & 0 deletions pkg/kv/kvserver/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -291,6 +291,15 @@ type Replica struct {
destroyStatus
// Is the range quiescent? Quiescent ranges are not Tick()'d and unquiesce
// whenever a Raft operation is performed.
//
// Replica objects always begin life in a quiescent state, as the field is
// set to true in the Replica constructor newUnloadedReplica. They unquiesce
// and set the field to false in either unquiesceAndWakeLeaderLocked or
// unquiesceWithOptionsLocked, which are called in response to Raft traffic.
//
// Only initialized replicas that have a non-nil internalRaftGroup are
// allowed to unquiesce and be Tick()'d. See canUnquiesceRLocked for an
// explanation of these conditions.
quiescent bool
// laggingFollowersOnQuiesce is the set of dead replicas that are not
// up-to-date with the rest of the quiescent Raft group. Nil if !quiescent.
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -466,8 +466,8 @@ func (r *Replica) handleRaftReady(
return r.handleRaftReadyRaftMuLocked(ctx, inSnap)
}

// handleRaftReadyLocked is the same as handleRaftReady but requires that the
// replica's raftMu be held.
// handleRaftReadyRaftMuLocked is the same as handleRaftReady but requires that
// the replica's raftMu be held.
//
// The returned string is nonzero whenever an error is returned to give a
// non-sensitive cue as to what happened.
Expand Down
33 changes: 31 additions & 2 deletions pkg/kv/kvserver/replica_raft_quiesce.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func (r *Replica) unquiesceLocked() {
}

func (r *Replica) unquiesceWithOptionsLocked(campaignOnWake bool) {
if r.mu.quiescent && r.mu.internalRaftGroup != nil {
if r.canUnquiesceRLocked() {
ctx := r.AnnotateCtx(context.TODO())
if log.V(3) {
log.Infof(ctx, "unquiescing %d", r.RangeID)
Expand All @@ -75,7 +75,7 @@ func (r *Replica) unquiesceWithOptionsLocked(campaignOnWake bool) {
}

func (r *Replica) unquiesceAndWakeLeaderLocked() {
if r.mu.quiescent && r.mu.internalRaftGroup != nil {
if r.canUnquiesceRLocked() {
ctx := r.AnnotateCtx(context.TODO())
if log.V(3) {
log.Infof(ctx, "unquiescing %d: waking leader", r.RangeID)
Expand All @@ -93,6 +93,35 @@ func (r *Replica) unquiesceAndWakeLeaderLocked() {
}
}

func (r *Replica) canUnquiesceRLocked() bool {
return r.mu.quiescent &&
// If the replica is uninitialized (i.e. it contains no replicated state),
// it is not allowed to unquiesce and begin Tick()'ing itself.
//
// Keeping uninitialized replicas quiesced even in the presence of Raft
// traffic avoids wasted work. We could Tick() these replicas, but doing so
// is unnecessary because uninitialized replicas can never win elections, so
// there is no reason for them to ever call an election. In fact,
// uninitialized replicas do not even know who their peers are, so there
// would be no way for them to call an election or for them to send any
// other non-reactive message. As a result, all work performed by an
// uninitialized replica is reactive and in response to incoming messages
// (see processRequestQueue).
//
// There are multiple ways for an uninitialized replica to be created and
// then abandoned, and we don't do a good job garbage collecting them at a
// later point (see https://github.com/cockroachdb/cockroach/issues/73424),
// so it is important that they are cheap. Keeping them quiesced instead of
// letting them unquiesce and tick every 200ms indefinitely avoids a
// meaningful amount of periodic work for each uninitialized replica.
r.isInitializedRLocked() &&
// A replica's Raft group begins in a dormant state and is initialized
// lazily in response to any Raft traffic (see stepRaftGroup) or KV request
// traffic (see maybeInitializeRaftGroup). If it has yet to be initialized,
// let it remain quiesced. The Raft group will be initialized soon enough.
r.mu.internalRaftGroup != nil
}

// maybeQuiesceLocked checks to see if the replica is quiescable and initiates
// quiescence if it is. Returns true if the replica has been quiesced and false
// otherwise.
Expand Down
25 changes: 25 additions & 0 deletions pkg/kv/kvserver/store_create_replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -344,6 +344,31 @@ func (s *Store) maybeMarkReplicaInitializedLockedReplLocked(
it.item.key(), it)
}

// Unquiesce the replica. We don't allow uninitialized replicas to unquiesce,
// but now that the replica has been initialized, we unquiesce it as soon as
// possible. This replica was initialized in response to the reception of a
// snapshot from another replica. This means that the other replica is not
// quiesced, so we don't need to campaign or wake the leader. We just want
// to start ticking.
//
// NOTE: The fact that this replica is being initialized in response to the
// receipt of a snapshot means that its r.mu.internalRaftGroup must not be
// nil.
//
// NOTE: Unquiescing the replica here is not strictly necessary. As of the
// time of writing, this function is only ever called below handleRaftReady,
// which will always unquiesce any eligible replicas before completing. So in
// marking this replica as initialized, we have made it eligible to unquiesce.
// However, there is still a benefit to unquiecing here instead of letting
// handleRaftReady do it for us. The benefit is that handleRaftReady cannot
// make assumptions about the state of the other replicas in the range when it
// unquieces a replica, so when it does so, it also instructs the replica to
// campaign and to wake the leader (by calling unquiesceAndWakeLeaderLocked).
// We have more information here (see "This means that the other replica ..."
// above) and can make assumptions about the state of the other replicas in
// the range, so we can unquiesce without campaigning or waking the leader.
lockedRepl.unquiesceWithOptionsLocked(false /* campaignOnWake */)

// Add the range to metrics and maybe gossip on capacity change.
s.metrics.ReplicaCount.Inc(1)
s.maybeGossipOnCapacityChange(ctx, rangeAddEvent)
Expand Down

0 comments on commit ad0a3ad

Please sign in to comment.