mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
raft: clean up bootstrap
This is the first (maybe not last) step in cleaning up the bootstrap code around StartNode. Initializing a Raft group for the first time is awkward, since a configuration has to be pulled from thin air. The way this is solved today is unclean: The app is supposed to pass peers to StartNode(), we add configuration changes for them to the log, immediately pretend that they are applied, but actually leave them unapplied (to give the app a chance to observe them, though if the app did decide to not apply them things would really go off the rails), and then return control to the app. The app will then process the initial Readys and as a result the configuration will be persisted to disk; restarts of the node then use RestartNode which doesn't take any peers. The code that did this lived awkwardly in two places fairly deep down the callstack, though it was really only necessary in StartNode(). This commit refactors things to make this more obvious: only StartNode does this dance now. In particular, RawNode does not support this at all any more; it expects the app to set up its Storage correctly. Future work may provide helpers to make this "preseeding" of the Storage more user-friendly. It isn't entirely straightforward to do so since the Storage interface doesn't provide the right accessors for this purpose. Briefly speaking, we want to make sure that a non-bootstrapped node can never catch up via the log so that we can implicitly use one of the "skipped" log entries to represent the configuration change into the bootstrap configuration. This is an invasive change that affects all consumers of raft, and it is of lower urgency since the code (post this commit) already encapsulates the complexity sufficiently.
This commit is contained in:
62
raft/node.go
62
raft/node.go
@@ -197,12 +197,63 @@ type Peer struct {
|
||||
|
||||
// StartNode returns a new Node given configuration and a list of raft peers.
|
||||
// It appends a ConfChangeAddNode entry for each given peer to the initial log.
|
||||
//
|
||||
// Peers must not be zero length; call RestartNode in that case.
|
||||
func StartNode(c *Config, peers []Peer) Node {
|
||||
rn, err := NewRawNode(c, peers)
|
||||
if len(peers) == 0 {
|
||||
panic("no peers given; use RestartNode instead")
|
||||
}
|
||||
rn, err := NewRawNode(c)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
lastIndex, err := rn.raft.raftLog.storage.LastIndex()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if lastIndex != 0 {
|
||||
panic("can't StartNode on a nonempty Storage")
|
||||
}
|
||||
|
||||
// We've faked out initial entries above, but nothing has been
|
||||
// persisted. Start with an empty HardState (thus the first Ready will
|
||||
// emit a HardState update for the app to persist).
|
||||
rn.prevHardSt = emptyState
|
||||
|
||||
// TODO(tbg): remove StartNode and give the application the right tools to
|
||||
// bootstrap the initial membership in a cleaner way.
|
||||
rn.raft.becomeFollower(1, None)
|
||||
ents := make([]pb.Entry, len(peers))
|
||||
for i, peer := range peers {
|
||||
cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
|
||||
data, err := cc.Marshal()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
ents[i] = pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: uint64(i + 1), Data: data}
|
||||
}
|
||||
rn.raft.raftLog.append(ents...)
|
||||
|
||||
// Now apply them, mainly so that the application can call Campaign
|
||||
// immediately after StartNode in tests. Note that these nodes will
|
||||
// be added to raft twice: here and when the application's Ready
|
||||
// loop calls ApplyConfChange. The calls to addNode must come after
|
||||
// all calls to raftLog.append so progress.next is set after these
|
||||
// bootstrapping entries (it is an error if we try to append these
|
||||
// entries since they have already been committed).
|
||||
// We do not set raftLog.applied so the application will be able
|
||||
// to observe all conf changes via Ready.CommittedEntries.
|
||||
//
|
||||
// TODO(bdarnell): These entries are still unstable; do we need to preserve
|
||||
// the invariant that committed < unstable?
|
||||
rn.raft.raftLog.committed = uint64(len(ents))
|
||||
for _, peer := range peers {
|
||||
rn.raft.applyConfChange(pb.ConfChange{NodeID: peer.ID, Type: pb.ConfChangeAddNode})
|
||||
}
|
||||
|
||||
n := newNode()
|
||||
n.logger = c.Logger
|
||||
|
||||
@@ -215,7 +266,14 @@ func StartNode(c *Config, peers []Peer) Node {
|
||||
// If the caller has an existing state machine, pass in the last log index that
|
||||
// has been applied to it; otherwise use zero.
|
||||
func RestartNode(c *Config) Node {
|
||||
return StartNode(c, nil)
|
||||
rn, err := NewRawNode(c)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
n := newNode()
|
||||
n.logger = c.Logger
|
||||
go n.run(rn)
|
||||
return &n
|
||||
}
|
||||
|
||||
type msgWithResult struct {
|
||||
|
||||
Reference in New Issue
Block a user