github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/kv/kvserver/replica_raft_quiesce.go (about)

     1  // Copyright 2019 The Cockroach Authors.
     2  //
     3  // Use of this software is governed by the Business Source License
     4  // included in the file licenses/BSL.txt.
     5  //
     6  // As of the Change Date specified in that file, in accordance with
     7  // the Business Source License, use of this software will be governed
     8  // by the Apache License, Version 2.0, included in the file
     9  // licenses/APL.txt.
    10  
    11  package kvserver
    12  
    13  import (
    14  	"context"
    15  
    16  	"github.com/cockroachdb/cockroach/pkg/roachpb"
    17  	"github.com/cockroachdb/cockroach/pkg/util/hlc"
    18  	"github.com/cockroachdb/cockroach/pkg/util/log"
    19  	"github.com/cockroachdb/cockroach/pkg/util/timeutil"
    20  	"go.etcd.io/etcd/raft"
    21  	"go.etcd.io/etcd/raft/raftpb"
    22  )
    23  
    24  func (r *Replica) quiesceLocked() bool {
    25  	ctx := r.AnnotateCtx(context.TODO())
    26  	if r.hasPendingProposalsRLocked() {
    27  		if log.V(3) {
    28  			log.Infof(ctx, "not quiescing: pending commands")
    29  		}
    30  		return false
    31  	}
    32  	// Note that we're not calling r.hasPendingProposalQuotaRLocked(). That has
    33  	// been checked on the leaseholder before deciding to quiesce. There's no
    34  	// point in checking it followers since the quota is reset when the lease
    35  	// changes.
    36  
    37  	if !r.mu.quiescent {
    38  		if log.V(3) {
    39  			log.Infof(ctx, "quiescing %d", r.RangeID)
    40  		}
    41  		r.mu.quiescent = true
    42  		r.store.unquiescedReplicas.Lock()
    43  		delete(r.store.unquiescedReplicas.m, r.RangeID)
    44  		r.store.unquiescedReplicas.Unlock()
    45  	} else if log.V(4) {
    46  		log.Infof(ctx, "already quiesced")
    47  	}
    48  	return true
    49  }
    50  
    51  func (r *Replica) unquiesce() {
    52  	r.mu.Lock()
    53  	defer r.mu.Unlock()
    54  	r.unquiesceLocked()
    55  }
    56  
    57  func (r *Replica) unquiesceLocked() {
    58  	r.unquiesceWithOptionsLocked(true /* campaignOnWake */)
    59  }
    60  
    61  func (r *Replica) unquiesceWithOptionsLocked(campaignOnWake bool) {
    62  	if r.mu.quiescent && r.mu.internalRaftGroup != nil {
    63  		ctx := r.AnnotateCtx(context.TODO())
    64  		if log.V(3) {
    65  			log.Infof(ctx, "unquiescing %d", r.RangeID)
    66  		}
    67  		r.mu.quiescent = false
    68  		r.store.unquiescedReplicas.Lock()
    69  		r.store.unquiescedReplicas.m[r.RangeID] = struct{}{}
    70  		r.store.unquiescedReplicas.Unlock()
    71  		if campaignOnWake {
    72  			r.maybeCampaignOnWakeLocked(ctx)
    73  		}
    74  		// NB: we know there's a non-nil RaftStatus because internalRaftGroup isn't nil.
    75  		r.mu.lastUpdateTimes.updateOnUnquiesce(
    76  			r.mu.state.Desc.Replicas().All(), r.raftStatusRLocked().Progress, timeutil.Now(),
    77  		)
    78  	}
    79  }
    80  
    81  func (r *Replica) unquiesceAndWakeLeaderLocked() {
    82  	if r.mu.quiescent && r.mu.internalRaftGroup != nil {
    83  		ctx := r.AnnotateCtx(context.TODO())
    84  		if log.V(3) {
    85  			log.Infof(ctx, "unquiescing %d: waking leader", r.RangeID)
    86  		}
    87  		r.mu.quiescent = false
    88  		r.store.unquiescedReplicas.Lock()
    89  		r.store.unquiescedReplicas.m[r.RangeID] = struct{}{}
    90  		r.store.unquiescedReplicas.Unlock()
    91  		r.maybeCampaignOnWakeLocked(ctx)
    92  		// Propose an empty command which will wake the leader.
    93  		data := encodeRaftCommand(raftVersionStandard, makeIDKey(), nil)
    94  		_ = r.mu.internalRaftGroup.Propose(data)
    95  	}
    96  }
    97  
    98  // maybeQuiesceLocked checks to see if the replica is quiescable and initiates
    99  // quiescence if it is. Returns true if the replica has been quiesced and false
   100  // otherwise.
   101  //
   102  // A quiesced range is not ticked and thus doesn't create MsgHeartbeat requests
   103  // or cause elections. The Raft leader for a range checks various
   104  // pre-conditions: no pending raft commands, no pending raft ready, all of the
   105  // followers are up to date, etc. Quiescence is initiated by a special
   106  // MsgHeartbeat that is tagged as Quiesce. Upon receipt (see
   107  // Store.processRaftRequestWithReplica), the follower checks to see if the
   108  // term/commit matches and marks the local replica as quiescent. If the
   109  // term/commit do not match the MsgHeartbeat is passed through to Raft which
   110  // will generate a MsgHeartbeatResp that will unquiesce the sender.
   111  //
   112  // Any Raft operation on the local replica will unquiesce the Replica. For
   113  // example, a Raft operation initiated on a follower will unquiesce the
   114  // follower which will send a MsgProp to the leader that will unquiesce the
   115  // leader. If the leader of a quiesced range dies, followers will not notice,
   116  // though any request directed to the range will eventually end up on a
   117  // follower which will unquiesce the follower and lead to an election. When a
   118  // follower unquiesces for a reason other than receiving a raft message or
   119  // proposing a raft command (for example the concurrent enqueuing of a tick),
   120  // it wakes the leader by sending an empty message proposal. This avoids
   121  // unnecessary elections due to bugs in which a follower is left unquiesced
   122  // while the leader is quiesced.
   123  //
   124  // Note that both the quiesce and wake-the-leader messages can be dropped or
   125  // reordered by the transport. The wake-the-leader message is termless so it
   126  // won't affect elections and, while it triggers reproprosals that won't cause
   127  // problems on reordering. If the wake-the-leader message is dropped the leader
   128  // won't be woken and the follower will eventually call an election.
   129  //
   130  // If the quiesce message is dropped the follower which missed it will not
   131  // quiesce and will eventually cause an election. The quiesce message is tagged
   132  // with the current term and commit index. If the quiesce message is reordered
   133  // it will either still apply to the recipient or the recipient will have moved
   134  // forward and the quiesce message will fall back to being a heartbeat.
   135  //
   136  // The supplied livenessMap maps from node ID to a boolean indicating
   137  // liveness. A range may be quiesced in the presence of non-live
   138  // replicas if the remaining live replicas all meet the quiesce
   139  // requirements. When a node considered non-live becomes live, the
   140  // node liveness instance invokes a callback which causes all nodes to
   141  // wakes up any ranges containing replicas owned by the newly-live
   142  // node, allowing the out-of-date replicas to be brought back up to date.
   143  // If livenessMap is nil, liveness data will not be used, meaning no range
   144  // will quiesce if any replicas are behind, whether or not they are live.
   145  // If any entry in the livenessMap is nil, then the missing node ID is
   146  // treated as not live.
   147  //
   148  // TODO(peter): There remains a scenario in which a follower is left unquiesced
   149  // while the leader is quiesced: the follower's receive queue is full and the
   150  // "quiesce" message is dropped. This seems very very unlikely because if the
   151  // follower isn't keeping up with raft messages it is unlikely that the leader
   152  // would quiesce. The fallout from this situation are undesirable raft
   153  // elections which will cause throughput hiccups to the range, but not
   154  // correctness issues.
   155  func (r *Replica) maybeQuiesceLocked(ctx context.Context, livenessMap IsLiveMap) bool {
   156  	status, ok := shouldReplicaQuiesce(ctx, r, r.store.Clock().Now(), livenessMap)
   157  	if !ok {
   158  		return false
   159  	}
   160  	return r.quiesceAndNotifyLocked(ctx, status)
   161  }
   162  
   163  type quiescer interface {
   164  	descRLocked() *roachpb.RangeDescriptor
   165  	raftStatusRLocked() *raft.Status
   166  	raftLastIndexLocked() (uint64, error)
   167  	hasRaftReadyRLocked() bool
   168  	hasPendingProposalsRLocked() bool
   169  	hasPendingProposalQuotaRLocked() bool
   170  	ownsValidLeaseRLocked(ts hlc.Timestamp) bool
   171  	mergeInProgressRLocked() bool
   172  	isDestroyedRLocked() (DestroyReason, error)
   173  }
   174  
   175  // shouldReplicaQuiesce determines if a replica should be quiesced. All of the
   176  // access to Replica internals are gated by the quiescer interface to
   177  // facilitate testing. Returns the raft.Status and true on success, and (nil,
   178  // false) on failure.
   179  //
   180  // Deciding to quiesce can race with requests being evaluated and their
   181  // proposals. Any proposal happening after the range has quiesced will
   182  // un-quiesce the range.
   183  //
   184  // A replica should quiesce if all the following hold:
   185  // a) The leaseholder and the leader are collocated. We don't want to quiesce
   186  // otherwise as we don't want to quiesce while a leader election is in progress,
   187  // and also we don't want to quiesce if another replica might have commands
   188  // pending that require this leader for proposing them. Note that, after the
   189  // leaseholder decides to quiesce, followers can still refuse quiescing if they
   190  // have pending commands.
   191  // b) There are no commands in-flight proposed by this leaseholder. Clients can
   192  // be waiting for results while there's pending proposals.
   193  // c) There is no outstanding proposal quota. Quiescing while there's quota
   194  // outstanding can lead to deadlock. See #46699.
   195  // d) All the live followers are caught up. We don't want to quiesce when
   196  // followers are behind because then they might not catch up until we
   197  // un-quiesce. We like it when everybody is caught up because otherwise
   198  // failovers can take longer.
   199  //
   200  // NOTE: The last 3 conditions are fairly, but not completely, overlapping.
   201  func shouldReplicaQuiesce(
   202  	ctx context.Context, q quiescer, now hlc.Timestamp, livenessMap IsLiveMap,
   203  ) (*raft.Status, bool) {
   204  	if testingDisableQuiescence {
   205  		return nil, false
   206  	}
   207  	if q.hasPendingProposalsRLocked() {
   208  		if log.V(4) {
   209  			log.Infof(ctx, "not quiescing: proposals pending")
   210  		}
   211  		return nil, false
   212  	}
   213  	// Don't quiesce if there's outstanding quota - it can lead to deadlock. This
   214  	// condition is largely subsumed by the upcoming replication state check,
   215  	// except that the conditions for replica availability are different, and the
   216  	// timing of releasing quota is unrelated to this function.
   217  	if q.hasPendingProposalQuotaRLocked() {
   218  		if log.V(4) {
   219  			log.Infof(ctx, "not quiescing: replication quota outstanding")
   220  		}
   221  		return nil, false
   222  	}
   223  	if q.mergeInProgressRLocked() {
   224  		if log.V(4) {
   225  			log.Infof(ctx, "not quiescing: merge in progress")
   226  		}
   227  		return nil, false
   228  	}
   229  	if _, err := q.isDestroyedRLocked(); err != nil {
   230  		if log.V(4) {
   231  			log.Infof(ctx, "not quiescing: replica destroyed")
   232  		}
   233  		return nil, false
   234  	}
   235  	status := q.raftStatusRLocked()
   236  	if status == nil {
   237  		if log.V(4) {
   238  			log.Infof(ctx, "not quiescing: dormant Raft group")
   239  		}
   240  		return nil, false
   241  	}
   242  	if status.SoftState.RaftState != raft.StateLeader {
   243  		if log.V(4) {
   244  			log.Infof(ctx, "not quiescing: not leader")
   245  		}
   246  		return nil, false
   247  	}
   248  	if status.LeadTransferee != 0 {
   249  		if log.V(4) {
   250  			log.Infof(ctx, "not quiescing: leader transfer to %d in progress", status.LeadTransferee)
   251  		}
   252  		return nil, false
   253  	}
   254  	// Only quiesce if this replica is the leaseholder as well;
   255  	// otherwise the replica which is the valid leaseholder may have
   256  	// pending commands which it's waiting on this leader to propose.
   257  	if !q.ownsValidLeaseRLocked(now) {
   258  		if log.V(4) {
   259  			log.Infof(ctx, "not quiescing: not leaseholder")
   260  		}
   261  		return nil, false
   262  	}
   263  	// We need all of Applied, Commit, LastIndex and Progress.Match indexes to be
   264  	// equal in order to quiesce.
   265  	if status.Applied != status.Commit {
   266  		if log.V(4) {
   267  			log.Infof(ctx, "not quiescing: applied (%d) != commit (%d)",
   268  				status.Applied, status.Commit)
   269  		}
   270  		return nil, false
   271  	}
   272  	lastIndex, err := q.raftLastIndexLocked()
   273  	if err != nil {
   274  		if log.V(4) {
   275  			log.Infof(ctx, "not quiescing: %v", err)
   276  		}
   277  		return nil, false
   278  	}
   279  	if status.Commit != lastIndex {
   280  		if log.V(4) {
   281  			log.Infof(ctx, "not quiescing: commit (%d) != lastIndex (%d)",
   282  				status.Commit, lastIndex)
   283  		}
   284  		return nil, false
   285  	}
   286  
   287  	var foundSelf bool
   288  	for _, rep := range q.descRLocked().Replicas().All() {
   289  		if uint64(rep.ReplicaID) == status.ID {
   290  			foundSelf = true
   291  		}
   292  		if progress, ok := status.Progress[uint64(rep.ReplicaID)]; !ok {
   293  			if log.V(4) {
   294  				log.Infof(ctx, "not quiescing: could not locate replica %d in progress: %+v",
   295  					rep.ReplicaID, progress)
   296  			}
   297  			return nil, false
   298  		} else if progress.Match != status.Applied {
   299  			// Skip any node in the descriptor which is not live.
   300  			if livenessMap != nil && !livenessMap[rep.NodeID].IsLive {
   301  				if log.V(4) {
   302  					log.Infof(ctx, "skipping node %d because not live. Progress=%+v",
   303  						rep.NodeID, progress)
   304  				}
   305  				continue
   306  			}
   307  			if log.V(4) {
   308  				log.Infof(ctx, "not quiescing: replica %d match (%d) != applied (%d)",
   309  					rep.ReplicaID, progress.Match, status.Applied)
   310  			}
   311  			return nil, false
   312  		}
   313  	}
   314  	if !foundSelf {
   315  		if log.V(4) {
   316  			log.Infof(ctx, "not quiescing: %d not found in progress: %+v",
   317  				status.ID, status.Progress)
   318  		}
   319  		return nil, false
   320  	}
   321  	if q.hasRaftReadyRLocked() {
   322  		if log.V(4) {
   323  			log.Infof(ctx, "not quiescing: raft ready")
   324  		}
   325  		return nil, false
   326  	}
   327  	return status, true
   328  }
   329  
   330  func (r *Replica) quiesceAndNotifyLocked(ctx context.Context, status *raft.Status) bool {
   331  	fromReplica, fromErr := r.getReplicaDescriptorByIDRLocked(r.mu.replicaID, r.mu.lastToReplica)
   332  	if fromErr != nil {
   333  		if log.V(4) {
   334  			log.Infof(ctx, "not quiescing: cannot find from replica (%d)", r.mu.replicaID)
   335  		}
   336  		return false
   337  	}
   338  
   339  	if !r.quiesceLocked() {
   340  		return false
   341  	}
   342  
   343  	for id, prog := range status.Progress {
   344  		if roachpb.ReplicaID(id) == r.mu.replicaID {
   345  			continue
   346  		}
   347  		toReplica, toErr := r.getReplicaDescriptorByIDRLocked(
   348  			roachpb.ReplicaID(id), r.mu.lastFromReplica)
   349  		if toErr != nil {
   350  			if log.V(4) {
   351  				log.Infof(ctx, "failed to quiesce: cannot find to replica (%d)", id)
   352  			}
   353  			r.unquiesceLocked()
   354  			return false
   355  		}
   356  
   357  		// Attach the commit as min(prog.Match, status.Commit). This is exactly
   358  		// the same as what raft.sendHeartbeat does. See the comment there for
   359  		// an explanation.
   360  		//
   361  		// If the follower is behind, we don't tell it that we're quiescing.
   362  		// This ensures that if the follower receives the heartbeat then it will
   363  		// unquiesce the Range and be caught up by the leader. Remember that we
   364  		// only allow Ranges to quiesce with straggling Replicas if we believe
   365  		// those Replicas are on dead nodes.
   366  		commit := status.Commit
   367  		quiesce := true
   368  		if prog.Match < status.Commit {
   369  			commit = prog.Match
   370  			quiesce = false
   371  		}
   372  		msg := raftpb.Message{
   373  			From:   uint64(r.mu.replicaID),
   374  			To:     id,
   375  			Type:   raftpb.MsgHeartbeat,
   376  			Term:   status.Term,
   377  			Commit: commit,
   378  		}
   379  
   380  		if !r.maybeCoalesceHeartbeat(ctx, msg, toReplica, fromReplica, quiesce) {
   381  			log.Fatalf(ctx, "failed to coalesce known heartbeat: %v", msg)
   382  		}
   383  	}
   384  	return true
   385  }