raft: internally support joint consensus
This commit introduces machinery to safely apply joint consensus configuration changes to Raft. The main contribution is the new package, `confchange`, which offers the primitives `Simple`, `EnterJoint`, and `LeaveJoint`. The first two take a list of configuration changes. `Simple` only declares success if these configuration changes (applied atomically) change the set of voters by at most one (i.e. it's fine to add or remove any number of learners, but change only one voter). `EnterJoint` makes the configuration joint and then applies the changes to it, in preparation of the caller returning later and transitioning out of the joint config into the final desired configuration via `LeaveJoint()`. This commit streamlines the conversion between voters and learners, which is now generally allowed whenever the above conditions are upheld (i.e. it's not possible to demote a voter and add a new voter in the context of a Simple configuration change, but it is possible via EnterJoint). Previously, we had the artificial restriction that a voter could not be demoted to a learner, but had to be removed first. Even though demoting a learner is generally less useful than promoting a learner (the latter is used to catch up future voters), demotions could see use in improved handling of temporary node unavailability, where it is desired to remove voting power from a down node, but to preserve its data should it return. An additional change that was made in this commit is to prevent the use of empty commit quorums, which was previously possible but for no good reason; this: Closes #10884. The work left to do in a future PR is to actually expose joint configurations to the applications using Raft. This will entail mostly API design and the addition of suitable testing, which to be carried out ergonomically is likely to motivate a larger refactor. Touches #7625.
This commit is contained in:
71
raft/raft.go
71
raft/raft.go
@ -24,6 +24,7 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/raft/confchange"
|
||||
"go.etcd.io/etcd/raft/quorum"
|
||||
pb "go.etcd.io/etcd/raft/raftpb"
|
||||
"go.etcd.io/etcd/raft/tracker"
|
||||
@ -356,15 +357,11 @@ func newRaft(c *Config) *raft {
|
||||
}
|
||||
for _, p := range peers {
|
||||
// Add node to active config.
|
||||
r.prs.InitProgress(p, 0 /* match */, 1 /* next */, false /* isLearner */)
|
||||
r.applyConfChange(pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: p})
|
||||
}
|
||||
for _, p := range learners {
|
||||
// Add learner to active config.
|
||||
r.prs.InitProgress(p, 0 /* match */, 1 /* next */, true /* isLearner */)
|
||||
|
||||
if r.id == p {
|
||||
r.isLearner = true
|
||||
}
|
||||
r.applyConfChange(pb.ConfChange{Type: pb.ConfChangeAddLearnerNode, NodeID: p})
|
||||
}
|
||||
|
||||
if !isHardStateEqual(hs, emptyState) {
|
||||
@ -1401,55 +1398,15 @@ func (r *raft) promotable() bool {
|
||||
}
|
||||
|
||||
func (r *raft) applyConfChange(cc pb.ConfChange) pb.ConfState {
|
||||
addNodeOrLearnerNode := func(id uint64, isLearner bool) {
|
||||
// NB: this method is intentionally hidden from view. All mutations of
|
||||
// the conf state must call applyConfChange directly.
|
||||
pr := r.prs.Progress[id]
|
||||
if pr == nil {
|
||||
r.prs.InitProgress(id, 0, r.raftLog.lastIndex()+1, isLearner)
|
||||
} else {
|
||||
if isLearner && !pr.IsLearner {
|
||||
// Can only change Learner to Voter.
|
||||
//
|
||||
// TODO(tbg): why?
|
||||
r.logger.Infof("%x ignored addLearner: do not support changing %x from raft peer to learner.", r.id, id)
|
||||
return
|
||||
}
|
||||
|
||||
if isLearner == pr.IsLearner {
|
||||
// Ignore any redundant addNode calls (which can happen because the
|
||||
// initial bootstrapping entries are applied twice).
|
||||
return
|
||||
}
|
||||
|
||||
// Change Learner to Voter, use origin Learner progress.
|
||||
r.prs.RemoveAny(id)
|
||||
r.prs.InitProgress(id, 0 /* match */, 1 /* next */, false /* isLearner */)
|
||||
pr.IsLearner = false
|
||||
*r.prs.Progress[id] = *pr
|
||||
}
|
||||
|
||||
// When a node is first added, we should mark it as recently active.
|
||||
// Otherwise, CheckQuorum may cause us to step down if it is invoked
|
||||
// before the added node has had a chance to communicate with us.
|
||||
r.prs.Progress[id].RecentActive = true
|
||||
}
|
||||
|
||||
var removed int
|
||||
if cc.NodeID != None {
|
||||
switch cc.Type {
|
||||
case pb.ConfChangeAddNode:
|
||||
addNodeOrLearnerNode(cc.NodeID, false /* isLearner */)
|
||||
case pb.ConfChangeAddLearnerNode:
|
||||
addNodeOrLearnerNode(cc.NodeID, true /* isLearner */)
|
||||
case pb.ConfChangeRemoveNode:
|
||||
removed++
|
||||
r.prs.RemoveAny(cc.NodeID)
|
||||
case pb.ConfChangeUpdateNode:
|
||||
default:
|
||||
panic("unexpected conf type")
|
||||
}
|
||||
cfg, prs, err := confchange.Changer{
|
||||
Tracker: r.prs,
|
||||
LastIndex: r.raftLog.lastIndex(),
|
||||
}.Simple(cc)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
r.prs.Config = cfg
|
||||
r.prs.Progress = prs
|
||||
|
||||
r.logger.Infof("%x switched to configuration %s", r.id, r.prs.Config)
|
||||
// Now that the configuration is updated, handle any side effects.
|
||||
@ -1479,12 +1436,10 @@ func (r *raft) applyConfChange(cc pb.ConfChange) pb.ConfState {
|
||||
if r.state != StateLeader || len(cs.Nodes) == 0 {
|
||||
return cs
|
||||
}
|
||||
if removed > 0 {
|
||||
if r.maybeCommit() {
|
||||
// The quorum size may have been reduced (but not to zero), so see if
|
||||
// any pending entries can be committed.
|
||||
if r.maybeCommit() {
|
||||
r.bcastAppend()
|
||||
}
|
||||
r.bcastAppend()
|
||||
}
|
||||
// If the the leadTransferee was removed, abort the leadership transfer.
|
||||
if _, tOK := r.prs.Progress[r.leadTransferee]; !tOK && r.leadTransferee != 0 {
|
||||
|
Reference in New Issue
Block a user