mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
raft: Integrate snapshots into the raft.Storage interface.
Compaction is now treated as an implementation detail of Storage implementations; Node.Compact() and related functionality have been removed. Ready.Snapshot is now used only for incoming snapshots. A return value has been added to ApplyConfChange to allow applications to track the node information that must be stored in the snapshot. raftpb.Snapshot has been split into Snapshot and SnapshotMetadata, to allow the full snapshot data to be read from disk only when needed. raft.Storage has new methods Snapshot, ApplySnapshot, HardState, and SetHardState. The Snapshot and HardState parameters have been removed from RestartNode() and will now be loaded from Storage instead. The only remaining difference between StartNode and RestartNode is that the former bootstraps an initial list of Peers.
This commit is contained in:
parent
46ee58c6f0
commit
355ee4f393
@ -56,7 +56,7 @@ func handleBackup(c *cli.Context) {
|
||||
}
|
||||
var index uint64
|
||||
if snapshot != nil {
|
||||
index = snapshot.Index
|
||||
index = snapshot.Metadata.Index
|
||||
newss := snap.New(destSnap)
|
||||
newss.SaveSnap(*snapshot)
|
||||
}
|
||||
|
@ -54,8 +54,12 @@ func restartAsStandaloneNode(cfg *ServerConfig, index uint64, snapshot *raftpb.S
|
||||
|
||||
log.Printf("etcdserver: forcing restart of member %s in cluster %s at commit index %d", id, cfg.Cluster.ID(), st.Commit)
|
||||
s := raft.NewMemoryStorage()
|
||||
if snapshot != nil {
|
||||
s.ApplySnapshot(*snapshot)
|
||||
}
|
||||
s.SetHardState(st)
|
||||
s.Append(ents)
|
||||
n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, s)
|
||||
n := raft.RestartNode(uint64(id), 10, 1, s)
|
||||
return id, n, s, w
|
||||
}
|
||||
|
||||
@ -67,7 +71,7 @@ func restartAsStandaloneNode(cfg *ServerConfig, index uint64, snapshot *raftpb.S
|
||||
func getIDs(snap *raftpb.Snapshot, ents []raftpb.Entry) []uint64 {
|
||||
ids := make(map[uint64]bool)
|
||||
if snap != nil {
|
||||
for _, id := range snap.Nodes {
|
||||
for _, id := range snap.Metadata.ConfState.Nodes {
|
||||
ids[id] = true
|
||||
}
|
||||
}
|
||||
|
@ -34,21 +34,30 @@ func TestGetIDs(t *testing.T) {
|
||||
normalEntry := raftpb.Entry{Type: raftpb.EntryNormal}
|
||||
|
||||
tests := []struct {
|
||||
snap *raftpb.Snapshot
|
||||
ents []raftpb.Entry
|
||||
confState *raftpb.ConfState
|
||||
ents []raftpb.Entry
|
||||
|
||||
widSet []uint64
|
||||
}{
|
||||
{nil, []raftpb.Entry{}, []uint64{}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{}, []uint64{1}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry}, []uint64{1, 2}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry, removeEntry}, []uint64{1}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry, normalEntry}, []uint64{1, 2}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry, removeEntry, normalEntry}, []uint64{1}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{}, []uint64{1}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{addEntry}, []uint64{1, 2}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{addEntry, removeEntry}, []uint64{1}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{addEntry, normalEntry}, []uint64{1, 2}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{addEntry, removeEntry, normalEntry}, []uint64{1}},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
idSet := getIDs(tt.snap, tt.ents)
|
||||
var snap raftpb.Snapshot
|
||||
if tt.confState != nil {
|
||||
snap.Metadata.ConfState = *tt.confState
|
||||
}
|
||||
idSet := getIDs(&snap, tt.ents)
|
||||
if !reflect.DeepEqual(idSet, tt.widSet) {
|
||||
t.Errorf("#%d: idset = %#v, want %#v", i, idSet, tt.widSet)
|
||||
}
|
||||
|
@ -248,9 +248,9 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
||||
return nil, err
|
||||
}
|
||||
if snapshot != nil {
|
||||
log.Printf("etcdserver: recovering from snapshot at index %d", snapshot.Index)
|
||||
log.Printf("etcdserver: recovering from snapshot at index %d", snapshot.Metadata.Index)
|
||||
st.Recovery(snapshot.Data)
|
||||
index = snapshot.Index
|
||||
index = snapshot.Metadata.Index
|
||||
}
|
||||
cfg.Cluster = NewClusterFromStore(cfg.Cluster.token, st)
|
||||
cfg.Print()
|
||||
@ -360,18 +360,22 @@ func (s *EtcdServer) run() {
|
||||
if err := s.storage.Save(rd.HardState, rd.Entries); err != nil {
|
||||
log.Fatalf("etcdserver: save state and entries error: %v", err)
|
||||
}
|
||||
if err := s.storage.SaveSnap(rd.Snapshot); err != nil {
|
||||
log.Fatalf("etcdserver: create snapshot error: %v", err)
|
||||
if !raft.IsEmptySnap(rd.Snapshot) {
|
||||
if err := s.storage.SaveSnap(rd.Snapshot); err != nil {
|
||||
log.Fatalf("etcdserver: create snapshot error: %v", err)
|
||||
}
|
||||
}
|
||||
s.sender.Send(rd.Messages)
|
||||
|
||||
// recover from snapshot if it is more updated than current applied
|
||||
if rd.Snapshot.Index > appliedi {
|
||||
if err := s.store.Recovery(rd.Snapshot.Data); err != nil {
|
||||
log.Panicf("recovery store error: %v", err)
|
||||
if !raft.IsEmptySnap(rd.Snapshot) {
|
||||
// recover from snapshot if it is more updated than current applied
|
||||
if rd.Snapshot.Metadata.Index > appliedi {
|
||||
if err := s.store.Recovery(rd.Snapshot.Data); err != nil {
|
||||
log.Panicf("recovery store error: %v", err)
|
||||
}
|
||||
s.Cluster.Recover()
|
||||
appliedi = rd.Snapshot.Metadata.Index
|
||||
}
|
||||
s.Cluster.Recover()
|
||||
appliedi = rd.Snapshot.Index
|
||||
}
|
||||
// TODO(bmizerany): do this in the background, but take
|
||||
// care to apply entries in a single goroutine, and not
|
||||
@ -395,9 +399,6 @@ func (s *EtcdServer) run() {
|
||||
|
||||
s.node.Advance()
|
||||
|
||||
if rd.Snapshot.Index > snapi {
|
||||
snapi = rd.Snapshot.Index
|
||||
}
|
||||
if appliedi-snapi > s.snapCount {
|
||||
s.snapshot(appliedi, nodes)
|
||||
snapi = appliedi
|
||||
@ -773,10 +774,19 @@ func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) {
|
||||
if err != nil {
|
||||
log.Panicf("store save should never fail: %v", err)
|
||||
}
|
||||
s.node.Compact(snapi, snapnodes, d)
|
||||
// TODO(bdarnell): save ConfState instead of snapnodes directly.
|
||||
s.raftStorage.Compact(snapi, &raftpb.ConfState{Nodes: snapnodes}, d)
|
||||
if err := s.storage.Cut(); err != nil {
|
||||
log.Panicf("rotate wal file should never fail: %v", err)
|
||||
}
|
||||
snap, err := s.raftStorage.Snapshot()
|
||||
if err != nil {
|
||||
log.Fatalf("etcdserver: snapshot error: %v", err)
|
||||
}
|
||||
if err := s.storage.SaveSnap(snap); err != nil {
|
||||
log.Fatalf("etcdserver: create snapshot error: %v", err)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// checkClientURLsEmptyFromPeers does its best to get the cluster from peers,
|
||||
@ -897,8 +907,12 @@ func restartNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (ty
|
||||
|
||||
log.Printf("etcdserver: restart member %s in cluster %s at commit index %d", id, cfg.Cluster.ID(), st.Commit)
|
||||
s := raft.NewMemoryStorage()
|
||||
if snapshot != nil {
|
||||
s.ApplySnapshot(*snapshot)
|
||||
}
|
||||
s.SetHardState(st)
|
||||
s.Append(ents)
|
||||
n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, s)
|
||||
n := raft.RestartNode(uint64(id), 10, 1, s)
|
||||
return id, n, s, w
|
||||
}
|
||||
|
||||
|
@ -925,12 +925,15 @@ func TestSnapshot(t *testing.T) {
|
||||
}
|
||||
|
||||
gaction = p.Action()
|
||||
if len(gaction) != 1 {
|
||||
t.Fatalf("len(action) = %d, want 1", len(gaction))
|
||||
if len(gaction) != 2 {
|
||||
t.Fatalf("len(action) = %d, want 2", len(gaction))
|
||||
}
|
||||
if !reflect.DeepEqual(gaction[0], action{name: "Cut"}) {
|
||||
t.Errorf("action = %s, want Cut", gaction[0])
|
||||
}
|
||||
if !reflect.DeepEqual(gaction[1], action{name: "SaveSnap"}) {
|
||||
t.Errorf("action = %s, want SaveSnap", gaction[1])
|
||||
}
|
||||
}
|
||||
|
||||
// Applied > SnapCount should trigger a SaveSnap event
|
||||
@ -967,7 +970,7 @@ func TestTriggerSnap(t *testing.T) {
|
||||
gaction := p.Action()
|
||||
// each operation is recorded as a Save
|
||||
// BootstrapConfig/Nop + (SnapCount - 1) * Puts + Cut + SaveSnap = Save + (SnapCount - 1) * Save + Cut + SaveSnap
|
||||
wcnt := 2 + int(srv.snapCount)
|
||||
wcnt := 1 + int(srv.snapCount)
|
||||
if len(gaction) != wcnt {
|
||||
t.Fatalf("len(action) = %d, want %d", len(gaction), wcnt)
|
||||
}
|
||||
@ -994,7 +997,7 @@ func TestRecvSnapshot(t *testing.T) {
|
||||
}
|
||||
|
||||
s.start()
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Index: 1}}
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}}}
|
||||
// make goroutines move forward to receive snapshot
|
||||
testutil.ForceGosched()
|
||||
s.Stop()
|
||||
@ -1027,12 +1030,12 @@ func TestRecvSlowSnapshot(t *testing.T) {
|
||||
}
|
||||
|
||||
s.start()
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Index: 1}}
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}}}
|
||||
// make goroutines move forward to receive snapshot
|
||||
testutil.ForceGosched()
|
||||
action := st.Action()
|
||||
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Index: 1}}
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}}}
|
||||
// make goroutines move forward to receive snapshot
|
||||
testutil.ForceGosched()
|
||||
s.Stop()
|
||||
@ -1045,23 +1048,24 @@ func TestRecvSlowSnapshot(t *testing.T) {
|
||||
// TestApplySnapshotAndCommittedEntries tests that server applies snapshot
|
||||
// first and then committed entries.
|
||||
func TestApplySnapshotAndCommittedEntries(t *testing.T) {
|
||||
t.Skip("TODO(bdarnell): re-enable this test")
|
||||
n := newReadyNode()
|
||||
st := &storeRecorder{}
|
||||
cl := newCluster("abc")
|
||||
cl.SetStore(store.New())
|
||||
storage := raft.NewMemoryStorage()
|
||||
s := &EtcdServer{
|
||||
store: st,
|
||||
sender: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
node: n,
|
||||
Cluster: cl,
|
||||
store: st,
|
||||
sender: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
node: n,
|
||||
raftStorage: storage,
|
||||
Cluster: cl,
|
||||
}
|
||||
|
||||
s.start()
|
||||
req := &pb.Request{Method: "QGET"}
|
||||
n.readyc <- raft.Ready{
|
||||
Snapshot: raftpb.Snapshot{Index: 1},
|
||||
Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}},
|
||||
CommittedEntries: []raftpb.Entry{
|
||||
{Index: 2, Data: pbutil.MustMarshal(req)},
|
||||
},
|
||||
@ -1536,12 +1540,12 @@ func (n *readyNode) Propose(ctx context.Context, data []byte) error { return nil
|
||||
func (n *readyNode) ProposeConfChange(ctx context.Context, conf raftpb.ConfChange) error {
|
||||
return nil
|
||||
}
|
||||
func (n *readyNode) Step(ctx context.Context, msg raftpb.Message) error { return nil }
|
||||
func (n *readyNode) Ready() <-chan raft.Ready { return n.readyc }
|
||||
func (n *readyNode) Advance() {}
|
||||
func (n *readyNode) ApplyConfChange(conf raftpb.ConfChange) {}
|
||||
func (n *readyNode) Stop() {}
|
||||
func (n *readyNode) Compact(index uint64, nodes []uint64, d []byte) {}
|
||||
func (n *readyNode) Step(ctx context.Context, msg raftpb.Message) error { return nil }
|
||||
func (n *readyNode) Ready() <-chan raft.Ready { return n.readyc }
|
||||
func (n *readyNode) Advance() {}
|
||||
func (n *readyNode) ApplyConfChange(conf raftpb.ConfChange) *raftpb.ConfState { return nil }
|
||||
func (n *readyNode) Stop() {}
|
||||
func (n *readyNode) Compact(index uint64, nodes []uint64, d []byte) {}
|
||||
|
||||
type nodeRecorder struct {
|
||||
recorder
|
||||
@ -1567,8 +1571,9 @@ func (n *nodeRecorder) Step(ctx context.Context, msg raftpb.Message) error {
|
||||
}
|
||||
func (n *nodeRecorder) Ready() <-chan raft.Ready { return nil }
|
||||
func (n *nodeRecorder) Advance() {}
|
||||
func (n *nodeRecorder) ApplyConfChange(conf raftpb.ConfChange) {
|
||||
func (n *nodeRecorder) ApplyConfChange(conf raftpb.ConfChange) *raftpb.ConfState {
|
||||
n.record(action{name: "ApplyConfChange", params: []interface{}{conf}})
|
||||
return nil
|
||||
}
|
||||
func (n *nodeRecorder) Stop() {
|
||||
n.record(action{name: "Stop"})
|
||||
@ -1628,8 +1633,9 @@ func (n *nodeConfChangeCommitterRecorder) ProposeConfChange(ctx context.Context,
|
||||
func (n *nodeConfChangeCommitterRecorder) Ready() <-chan raft.Ready {
|
||||
return n.readyc
|
||||
}
|
||||
func (n *nodeConfChangeCommitterRecorder) ApplyConfChange(conf raftpb.ConfChange) {
|
||||
func (n *nodeConfChangeCommitterRecorder) ApplyConfChange(conf raftpb.ConfChange) *raftpb.ConfState {
|
||||
n.record(action{name: "ApplyConfChange:" + conf.Type.String()})
|
||||
return nil
|
||||
}
|
||||
|
||||
type waitWithResponse struct {
|
||||
|
@ -85,7 +85,7 @@ func Migrate4To5(dataDir string, name string) error {
|
||||
|
||||
// If we've got the most recent snapshot, we can use it's committed index. Still likely less than the current actual index, but worth it for the replay.
|
||||
if snap5 != nil {
|
||||
st5.Commit = snap5.Index
|
||||
st5.Commit = snap5.Metadata.Index
|
||||
}
|
||||
|
||||
ents5, err := Entries4To5(ents4)
|
||||
|
@ -172,14 +172,18 @@ func (s *Snapshot4) Snapshot5() *raftpb.Snapshot {
|
||||
}
|
||||
|
||||
snap5 := raftpb.Snapshot{
|
||||
Data: newState,
|
||||
Index: s.LastIndex,
|
||||
Term: s.LastTerm,
|
||||
Nodes: make([]uint64, len(s.Peers)),
|
||||
Data: newState,
|
||||
Metadata: raftpb.SnapshotMetadata{
|
||||
Index: s.LastIndex,
|
||||
Term: s.LastTerm,
|
||||
ConfState: raftpb.ConfState{
|
||||
Nodes: make([]uint64, len(s.Peers)),
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for i, p := range s.Peers {
|
||||
snap5.Nodes[i] = hashName(p.Name)
|
||||
snap5.Metadata.ConfState.Nodes[i] = hashName(p.Name)
|
||||
}
|
||||
|
||||
return &snap5
|
||||
|
62
raft/log.go
62
raft/log.go
@ -41,8 +41,7 @@ type raftLog struct {
|
||||
// applied is the highest log position that the application has
|
||||
// been instructed to apply to its state machine.
|
||||
// Invariant: applied <= committed
|
||||
applied uint64
|
||||
snapshot pb.Snapshot
|
||||
applied uint64
|
||||
}
|
||||
|
||||
func newLog(storage Storage) *raftLog {
|
||||
@ -52,11 +51,18 @@ func newLog(storage Storage) *raftLog {
|
||||
log := &raftLog{
|
||||
storage: storage,
|
||||
}
|
||||
firstIndex, err := storage.FirstIndex()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
lastIndex, err := storage.LastIndex()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
log.unstable = lastIndex + 1
|
||||
// Initialize our committed and applied pointers to the time of the last compaction.
|
||||
log.committed = firstIndex - 1
|
||||
log.applied = firstIndex - 1
|
||||
|
||||
return log
|
||||
}
|
||||
@ -139,9 +145,9 @@ func (l *raftLog) unstableEntries() []pb.Entry {
|
||||
// If applied is smaller than the index of snapshot, it returns all committed
|
||||
// entries after the index of snapshot.
|
||||
func (l *raftLog) nextEnts() (ents []pb.Entry) {
|
||||
off := max(l.applied, l.snapshot.Index)
|
||||
if l.committed > off {
|
||||
return l.slice(off+1, l.committed+1)
|
||||
off := max(l.applied+1, l.firstIndex())
|
||||
if l.committed+1 > off {
|
||||
return l.slice(off, l.committed+1)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
@ -235,49 +241,15 @@ func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
|
||||
return false
|
||||
}
|
||||
|
||||
// compact compacts all log entries until i.
|
||||
// It removes the log entries before i, exclusive.
|
||||
// i must be not smaller than the index of the first entry
|
||||
// and not greater than the index of the last entry.
|
||||
// the number of entries after compaction will be returned.
|
||||
func (l *raftLog) compact(i uint64) uint64 {
|
||||
if l.isOutOfAppliedBounds(i) {
|
||||
panic(fmt.Sprintf("compact %d out of bounds (applied up to %d)", i, l.applied))
|
||||
}
|
||||
err := l.storage.Compact(i)
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
l.unstable = max(i+1, l.unstable)
|
||||
firstIndex, err := l.storage.FirstIndex()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
lastIndex, err := l.storage.LastIndex()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
return lastIndex - firstIndex
|
||||
}
|
||||
|
||||
func (l *raftLog) snap(d []byte, index, term uint64, nodes []uint64) {
|
||||
l.snapshot = pb.Snapshot{
|
||||
Data: d,
|
||||
Nodes: nodes,
|
||||
Index: index,
|
||||
Term: term,
|
||||
}
|
||||
}
|
||||
|
||||
func (l *raftLog) restore(s pb.Snapshot) {
|
||||
l.storage = &MemoryStorage{
|
||||
ents: []pb.Entry{{Term: s.Term}},
|
||||
offset: s.Index,
|
||||
err := l.storage.ApplySnapshot(s)
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
l.unstable = s.Index + 1
|
||||
l.committed = s.Metadata.Index
|
||||
l.applied = s.Metadata.Index
|
||||
l.unstable = l.committed + 1
|
||||
l.unstableEnts = nil
|
||||
l.committed = s.Index
|
||||
l.snapshot = s
|
||||
}
|
||||
|
||||
func (l *raftLog) at(i uint64) *pb.Entry {
|
||||
|
@ -298,7 +298,7 @@ func TestCompactionSideEffects(t *testing.T) {
|
||||
raftLog.appliedTo(raftLog.committed)
|
||||
|
||||
offset := uint64(500)
|
||||
raftLog.compact(offset)
|
||||
storage.Compact(offset, nil, nil)
|
||||
|
||||
if raftLog.lastIndex() != lastIndex {
|
||||
t.Errorf("lastIndex = %d, want %d", raftLog.lastIndex(), lastIndex)
|
||||
@ -337,7 +337,9 @@ func TestCompactionSideEffects(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestNextEnts(t *testing.T) {
|
||||
snap := pb.Snapshot{Term: 1, Index: 3}
|
||||
snap := pb.Snapshot{
|
||||
Metadata: pb.SnapshotMetadata{Term: 1, Index: 3},
|
||||
}
|
||||
ents := []pb.Entry{
|
||||
{Term: 1, Index: 4},
|
||||
{Term: 1, Index: 5},
|
||||
@ -353,9 +355,10 @@ func TestNextEnts(t *testing.T) {
|
||||
{5, nil},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
raftLog := newLog(NewMemoryStorage())
|
||||
raftLog.restore(snap)
|
||||
raftLog.append(snap.Index, ents...)
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(snap)
|
||||
raftLog := newLog(storage)
|
||||
raftLog.append(snap.Metadata.Index, ents...)
|
||||
raftLog.maybeCommit(5, 1)
|
||||
raftLog.appliedTo(tt.applied)
|
||||
|
||||
@ -418,18 +421,16 @@ func TestStableTo(t *testing.T) {
|
||||
//TestCompaction ensures that the number of log entries is correct after compactions.
|
||||
func TestCompaction(t *testing.T) {
|
||||
tests := []struct {
|
||||
applied uint64
|
||||
lastIndex uint64
|
||||
compact []uint64
|
||||
wleft []int
|
||||
wallow bool
|
||||
}{
|
||||
// out of upper bound
|
||||
{1000, 1000, []uint64{1001}, []int{-1}, false},
|
||||
{1000, 1000, []uint64{300, 500, 800, 900}, []int{700, 500, 200, 100}, true},
|
||||
{1000, []uint64{1001}, []int{-1}, false},
|
||||
{1000, []uint64{300, 500, 800, 900}, []int{700, 500, 200, 100}, true},
|
||||
// out of lower bound
|
||||
{1000, 1000, []uint64{300, 299}, []int{700, -1}, false},
|
||||
{0, 1000, []uint64{1}, []int{-1}, false},
|
||||
{1000, []uint64{300, 299}, []int{700, -1}, false},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
@ -447,11 +448,11 @@ func TestCompaction(t *testing.T) {
|
||||
storage.Append([]pb.Entry{{}})
|
||||
}
|
||||
raftLog := newLog(storage)
|
||||
raftLog.maybeCommit(tt.applied, 0)
|
||||
raftLog.maybeCommit(tt.lastIndex, 0)
|
||||
raftLog.appliedTo(raftLog.committed)
|
||||
|
||||
for j := 0; j < len(tt.compact); j++ {
|
||||
raftLog.compact(tt.compact[j])
|
||||
storage.Compact(tt.compact[j], nil, nil)
|
||||
if len(raftLog.allEntries()) != tt.wleft[j] {
|
||||
t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.allEntries()), tt.wleft[j])
|
||||
}
|
||||
@ -461,15 +462,12 @@ func TestCompaction(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestLogRestore(t *testing.T) {
|
||||
var i uint64
|
||||
raftLog := newLog(NewMemoryStorage())
|
||||
for i = 0; i < 100; i++ {
|
||||
raftLog.append(i, pb.Entry{Term: i + 1})
|
||||
}
|
||||
|
||||
index := uint64(1000)
|
||||
term := uint64(1000)
|
||||
raftLog.restore(pb.Snapshot{Index: index, Term: term})
|
||||
snap := pb.SnapshotMetadata{Index: index, Term: term}
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(pb.Snapshot{Metadata: snap})
|
||||
raftLog := newLog(storage)
|
||||
|
||||
// only has the guard entry
|
||||
if len(raftLog.allEntries()) != 0 {
|
||||
@ -492,8 +490,9 @@ func TestLogRestore(t *testing.T) {
|
||||
func TestIsOutOfBounds(t *testing.T) {
|
||||
offset := uint64(100)
|
||||
num := uint64(100)
|
||||
l := newLog(NewMemoryStorage())
|
||||
l.restore(pb.Snapshot{Index: offset})
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
|
||||
l := newLog(storage)
|
||||
l.append(offset, make([]pb.Entry, num)...)
|
||||
|
||||
tests := []struct {
|
||||
@ -520,8 +519,9 @@ func TestAt(t *testing.T) {
|
||||
offset := uint64(100)
|
||||
num := uint64(100)
|
||||
|
||||
l := newLog(NewMemoryStorage())
|
||||
l.restore(pb.Snapshot{Index: offset})
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
|
||||
l := newLog(storage)
|
||||
for i = 1; i < num; i++ {
|
||||
l.append(offset+i-1, pb.Entry{Term: i})
|
||||
}
|
||||
@ -550,8 +550,9 @@ func TestTerm(t *testing.T) {
|
||||
offset := uint64(100)
|
||||
num := uint64(100)
|
||||
|
||||
l := newLog(NewMemoryStorage())
|
||||
l.restore(pb.Snapshot{Index: offset})
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
|
||||
l := newLog(storage)
|
||||
for i = 1; i < num; i++ {
|
||||
l.append(offset+i-1, pb.Entry{Term: i})
|
||||
}
|
||||
@ -580,8 +581,9 @@ func TestSlice(t *testing.T) {
|
||||
offset := uint64(100)
|
||||
num := uint64(100)
|
||||
|
||||
l := newLog(NewMemoryStorage())
|
||||
l.restore(pb.Snapshot{Index: offset})
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
|
||||
l := newLog(storage)
|
||||
for i = 1; i < num; i++ {
|
||||
l.append(offset+i-1, pb.Entry{Term: i})
|
||||
}
|
||||
|
114
raft/node.go
114
raft/node.go
@ -75,12 +75,6 @@ type Ready struct {
|
||||
Messages []pb.Message
|
||||
}
|
||||
|
||||
type compact struct {
|
||||
index uint64
|
||||
nodes []uint64
|
||||
data []byte
|
||||
}
|
||||
|
||||
func isHardStateEqual(a, b pb.HardState) bool {
|
||||
return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit
|
||||
}
|
||||
@ -92,12 +86,13 @@ func IsEmptyHardState(st pb.HardState) bool {
|
||||
|
||||
// IsEmptySnap returns true if the given Snapshot is empty.
|
||||
func IsEmptySnap(sp pb.Snapshot) bool {
|
||||
return sp.Index == 0
|
||||
return sp.Metadata.Index == 0
|
||||
}
|
||||
|
||||
func (rd Ready) containsUpdates() bool {
|
||||
return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) || !IsEmptySnap(rd.Snapshot) ||
|
||||
len(rd.Entries) > 0 || len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
|
||||
return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) ||
|
||||
!IsEmptySnap(rd.Snapshot) || len(rd.Entries) > 0 ||
|
||||
len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
|
||||
}
|
||||
|
||||
// Node represents a node in a raft cluster.
|
||||
@ -122,18 +117,14 @@ type Node interface {
|
||||
// It prepares the node to return the next available Ready.
|
||||
Advance()
|
||||
// ApplyConfChange applies config change to the local node.
|
||||
// Returns an opaque ConfState protobuf which must be recorded
|
||||
// in snapshots. Will never return nil; it returns a pointer only
|
||||
// to match MemoryStorage.Compact.
|
||||
// TODO: reject existing node when add node
|
||||
// TODO: reject non-existant node when remove node
|
||||
ApplyConfChange(cc pb.ConfChange)
|
||||
ApplyConfChange(cc pb.ConfChange) *pb.ConfState
|
||||
// Stop performs any necessary termination of the Node
|
||||
Stop()
|
||||
// Compact discards the entrire log up to the given index. It also
|
||||
// generates a raft snapshot containing the given nodes configuration
|
||||
// and the given snapshot data.
|
||||
// It is the caller's responsibility to ensure the given configuration
|
||||
// and snapshot data match the actual point-in-time configuration and snapshot
|
||||
// at the given index.
|
||||
Compact(index uint64, nodes []uint64, d []byte)
|
||||
}
|
||||
|
||||
type Peer struct {
|
||||
@ -157,55 +148,49 @@ func StartNode(id uint64, peers []Peer, election, heartbeat int, storage Storage
|
||||
e := pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: r.raftLog.lastIndex() + 1, Data: d}
|
||||
r.raftLog.append(r.raftLog.lastIndex(), e)
|
||||
}
|
||||
// Mark these initial entries as committed.
|
||||
// TODO(bdarnell): These entries are still unstable; do we need to preserve
|
||||
// the invariant that committed < unstable?
|
||||
r.raftLog.committed = r.raftLog.lastIndex()
|
||||
|
||||
go n.run(r)
|
||||
return &n
|
||||
}
|
||||
|
||||
// RestartNode is identical to StartNode but takes an initial State and a slice
|
||||
// of entries. Generally this is used when restarting from a stable storage
|
||||
// log.
|
||||
// TODO(bdarnell): remove args that are unnecessary with storage.
|
||||
// Maybe this function goes away and is replaced by StartNode with a non-empty Storage.
|
||||
func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, storage Storage) Node {
|
||||
// RestartNode is identical to StartNode but does not take a list of peers.
|
||||
// The current membership of the cluster will be restored from the Storage.
|
||||
func RestartNode(id uint64, election, heartbeat int, storage Storage) Node {
|
||||
n := newNode()
|
||||
r := newRaft(id, nil, election, heartbeat, storage)
|
||||
if snapshot != nil {
|
||||
r.restore(*snapshot)
|
||||
r.raftLog.appliedTo(snapshot.Index)
|
||||
}
|
||||
if !isHardStateEqual(st, emptyState) {
|
||||
r.loadState(st)
|
||||
}
|
||||
|
||||
go n.run(r)
|
||||
return &n
|
||||
}
|
||||
|
||||
// node is the canonical implementation of the Node interface
|
||||
type node struct {
|
||||
propc chan pb.Message
|
||||
recvc chan pb.Message
|
||||
compactc chan compact
|
||||
confc chan pb.ConfChange
|
||||
readyc chan Ready
|
||||
advancec chan struct{}
|
||||
tickc chan struct{}
|
||||
done chan struct{}
|
||||
stop chan struct{}
|
||||
propc chan pb.Message
|
||||
recvc chan pb.Message
|
||||
confc chan pb.ConfChange
|
||||
confstatec chan pb.ConfState
|
||||
readyc chan Ready
|
||||
advancec chan struct{}
|
||||
tickc chan struct{}
|
||||
done chan struct{}
|
||||
stop chan struct{}
|
||||
}
|
||||
|
||||
func newNode() node {
|
||||
return node{
|
||||
propc: make(chan pb.Message),
|
||||
recvc: make(chan pb.Message),
|
||||
compactc: make(chan compact),
|
||||
confc: make(chan pb.ConfChange),
|
||||
readyc: make(chan Ready),
|
||||
advancec: make(chan struct{}),
|
||||
tickc: make(chan struct{}),
|
||||
done: make(chan struct{}),
|
||||
stop: make(chan struct{}),
|
||||
propc: make(chan pb.Message),
|
||||
recvc: make(chan pb.Message),
|
||||
confc: make(chan pb.ConfChange),
|
||||
confstatec: make(chan pb.ConfState),
|
||||
readyc: make(chan Ready),
|
||||
advancec: make(chan struct{}),
|
||||
tickc: make(chan struct{}),
|
||||
done: make(chan struct{}),
|
||||
stop: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
@ -232,13 +217,12 @@ func (n *node) run(r *raft) {
|
||||
lead := None
|
||||
prevSoftSt := r.softState()
|
||||
prevHardSt := r.HardState
|
||||
prevSnapi := r.raftLog.snapshot.Index
|
||||
|
||||
for {
|
||||
if advancec != nil {
|
||||
readyc = nil
|
||||
} else {
|
||||
rd = newReady(r, prevSoftSt, prevHardSt, prevSnapi)
|
||||
rd = newReady(r, prevSoftSt, prevHardSt)
|
||||
if rd.containsUpdates() {
|
||||
readyc = n.readyc
|
||||
} else {
|
||||
@ -270,11 +254,13 @@ func (n *node) run(r *raft) {
|
||||
r.Step(m)
|
||||
case m := <-n.recvc:
|
||||
r.Step(m) // raft never returns an error
|
||||
case c := <-n.compactc:
|
||||
r.compact(c.index, c.nodes, c.data)
|
||||
case cc := <-n.confc:
|
||||
if cc.NodeID == None {
|
||||
r.resetPendingConf()
|
||||
select {
|
||||
case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
|
||||
case <-n.done:
|
||||
}
|
||||
break
|
||||
}
|
||||
switch cc.Type {
|
||||
@ -287,6 +273,10 @@ func (n *node) run(r *raft) {
|
||||
default:
|
||||
panic("unexpected conf type")
|
||||
}
|
||||
select {
|
||||
case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
|
||||
case <-n.done:
|
||||
}
|
||||
case <-n.tickc:
|
||||
r.tick()
|
||||
case readyc <- rd:
|
||||
@ -301,9 +291,8 @@ func (n *node) run(r *raft) {
|
||||
prevHardSt = rd.HardState
|
||||
}
|
||||
if !IsEmptySnap(rd.Snapshot) {
|
||||
prevSnapi = rd.Snapshot.Index
|
||||
if prevSnapi > prevLastUnstablei {
|
||||
prevLastUnstablei = prevSnapi
|
||||
if rd.Snapshot.Metadata.Index > prevLastUnstablei {
|
||||
prevLastUnstablei = rd.Snapshot.Metadata.Index
|
||||
havePrevLastUnstablei = true
|
||||
}
|
||||
}
|
||||
@ -388,21 +377,20 @@ func (n *node) Advance() {
|
||||
}
|
||||
}
|
||||
|
||||
func (n *node) ApplyConfChange(cc pb.ConfChange) {
|
||||
func (n *node) ApplyConfChange(cc pb.ConfChange) *pb.ConfState {
|
||||
var cs pb.ConfState
|
||||
select {
|
||||
case n.confc <- cc:
|
||||
case <-n.done:
|
||||
}
|
||||
}
|
||||
|
||||
func (n *node) Compact(index uint64, nodes []uint64, d []byte) {
|
||||
select {
|
||||
case n.compactc <- compact{index, nodes, d}:
|
||||
case cs = <-n.confstatec:
|
||||
case <-n.done:
|
||||
}
|
||||
return &cs
|
||||
}
|
||||
|
||||
func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState, prevSnapi uint64) Ready {
|
||||
func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready {
|
||||
rd := Ready{
|
||||
Entries: r.raftLog.unstableEntries(),
|
||||
CommittedEntries: r.raftLog.nextEnts(),
|
||||
@ -414,8 +402,8 @@ func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState, prevSnapi
|
||||
if !isHardStateEqual(r.HardState, prevHardSt) {
|
||||
rd.HardState = r.HardState
|
||||
}
|
||||
if prevSnapi != r.raftLog.snapshot.Index {
|
||||
rd.Snapshot = r.raftLog.snapshot
|
||||
if r.snapshot != nil {
|
||||
rd.Snapshot = *r.snapshot
|
||||
}
|
||||
return rd
|
||||
}
|
||||
|
@ -281,7 +281,7 @@ func TestReadyContainUpdates(t *testing.T) {
|
||||
{Ready{Entries: make([]raftpb.Entry, 1, 1)}, true},
|
||||
{Ready{CommittedEntries: make([]raftpb.Entry, 1, 1)}, true},
|
||||
{Ready{Messages: make([]raftpb.Message, 1, 1)}, true},
|
||||
{Ready{Snapshot: raftpb.Snapshot{Index: 1}}, true},
|
||||
{Ready{Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}}}, true},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
@ -363,8 +363,9 @@ func TestNodeRestart(t *testing.T) {
|
||||
}
|
||||
|
||||
storage := NewMemoryStorage()
|
||||
storage.SetHardState(st)
|
||||
storage.Append(entries)
|
||||
n := RestartNode(1, 10, 1, nil, st, storage)
|
||||
n := RestartNode(1, 10, 1, storage)
|
||||
if g := <-n.Ready(); !reflect.DeepEqual(g, want) {
|
||||
t.Errorf("g = %+v,\n w %+v", g, want)
|
||||
}
|
||||
@ -378,15 +379,14 @@ func TestNodeRestart(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestNodeRestartFromSnapshot(t *testing.T) {
|
||||
t.Skip("TODO(bdarnell): re-enable after integrating snapshot and storage")
|
||||
snap := &raftpb.Snapshot{
|
||||
Data: []byte("some data"),
|
||||
Nodes: []uint64{1, 2},
|
||||
Index: 2,
|
||||
Term: 1,
|
||||
snap := raftpb.Snapshot{
|
||||
Metadata: raftpb.SnapshotMetadata{
|
||||
ConfState: raftpb.ConfState{Nodes: []uint64{1, 2}},
|
||||
Index: 2,
|
||||
Term: 1,
|
||||
},
|
||||
}
|
||||
entries := []raftpb.Entry{
|
||||
{Term: 1, Index: 2},
|
||||
{Term: 1, Index: 3, Data: []byte("foo")},
|
||||
}
|
||||
st := raftpb.HardState{Term: 1, Commit: 3}
|
||||
@ -394,12 +394,14 @@ func TestNodeRestartFromSnapshot(t *testing.T) {
|
||||
want := Ready{
|
||||
HardState: emptyState,
|
||||
// commit upto index commit index in st
|
||||
CommittedEntries: entries[1:],
|
||||
CommittedEntries: entries,
|
||||
}
|
||||
|
||||
s := NewMemoryStorage()
|
||||
s.SetHardState(st)
|
||||
s.ApplySnapshot(snap)
|
||||
s.Append(entries)
|
||||
n := RestartNode(1, 10, 1, snap, st, s)
|
||||
n := RestartNode(1, 10, 1, s)
|
||||
if g := <-n.Ready(); !reflect.DeepEqual(g, want) {
|
||||
t.Errorf("g = %+v,\n w %+v", g, want)
|
||||
} else {
|
||||
@ -413,61 +415,6 @@ func TestNodeRestartFromSnapshot(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// TestCompacts ensures Node.Compact creates a correct raft snapshot and compacts
|
||||
// the raft log (call raft.compact)
|
||||
func TestNodeCompact(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
n := newNode()
|
||||
storage := NewMemoryStorage()
|
||||
r := newRaft(1, []uint64{1}, 10, 1, storage)
|
||||
go n.run(r)
|
||||
|
||||
n.Campaign(ctx)
|
||||
n.Propose(ctx, []byte("foo"))
|
||||
|
||||
w := raftpb.Snapshot{
|
||||
Term: 1,
|
||||
Index: 2, // one nop + one proposal
|
||||
Data: []byte("a snapshot"),
|
||||
Nodes: []uint64{1},
|
||||
}
|
||||
|
||||
testutil.ForceGosched()
|
||||
select {
|
||||
case rd := <-n.Ready():
|
||||
storage.Append(rd.Entries)
|
||||
n.Advance()
|
||||
default:
|
||||
t.Fatalf("unexpected proposal failure: unable to commit entry")
|
||||
}
|
||||
|
||||
n.Compact(w.Index, w.Nodes, w.Data)
|
||||
testutil.ForceGosched()
|
||||
select {
|
||||
case rd := <-n.Ready():
|
||||
if !reflect.DeepEqual(rd.Snapshot, w) {
|
||||
t.Errorf("snap = %+v, want %+v", rd.Snapshot, w)
|
||||
}
|
||||
storage.Append(rd.Entries)
|
||||
n.Advance()
|
||||
default:
|
||||
t.Fatalf("unexpected compact failure: unable to create a snapshot")
|
||||
}
|
||||
testutil.ForceGosched()
|
||||
// TODO: this test the run updates the snapi correctly... should be tested
|
||||
// separately with other kinds of updates
|
||||
select {
|
||||
case <-n.Ready():
|
||||
t.Fatalf("unexpected more ready")
|
||||
default:
|
||||
}
|
||||
n.Stop()
|
||||
|
||||
if r.raftLog.firstIndex() != w.Index+1 {
|
||||
t.Errorf("log.offset = %d, want %d", r.raftLog.firstIndex(), w.Index)
|
||||
}
|
||||
}
|
||||
|
||||
func TestNodeAdvance(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
43
raft/raft.go
43
raft/raft.go
@ -105,6 +105,9 @@ type raft struct {
|
||||
|
||||
msgs []pb.Message
|
||||
|
||||
// the incoming snapshot, if any.
|
||||
snapshot *pb.Snapshot
|
||||
|
||||
// the leader id
|
||||
lead uint64
|
||||
|
||||
@ -123,10 +126,15 @@ func newRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage
|
||||
if id == None {
|
||||
panic("cannot use none id")
|
||||
}
|
||||
log := newLog(storage)
|
||||
st, err := storage.HardState()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
r := &raft{
|
||||
id: id,
|
||||
lead: None,
|
||||
raftLog: newLog(storage),
|
||||
raftLog: log,
|
||||
prs: make(map[uint64]*progress),
|
||||
electionTimeout: election,
|
||||
heartbeatTimeout: heartbeat,
|
||||
@ -135,6 +143,9 @@ func newRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage
|
||||
for _, p := range peers {
|
||||
r.prs[p] = &progress{next: 1}
|
||||
}
|
||||
if !isHardStateEqual(st, emptyState) {
|
||||
r.loadState(st)
|
||||
}
|
||||
r.becomeFollower(0, None)
|
||||
return r
|
||||
}
|
||||
@ -189,7 +200,14 @@ func (r *raft) sendAppend(to uint64) {
|
||||
m.To = to
|
||||
if r.needSnapshot(pr.next) {
|
||||
m.Type = pb.MsgSnap
|
||||
m.Snapshot = r.raftLog.snapshot
|
||||
snapshot, err := r.raftLog.storage.Snapshot()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
if snapshot.Metadata.Term == 0 {
|
||||
panic("need non-empty snapshot")
|
||||
}
|
||||
m.Snapshot = snapshot
|
||||
} else {
|
||||
m.Type = pb.MsgApp
|
||||
m.Index = pr.next - 1
|
||||
@ -381,6 +399,7 @@ func (r *raft) handleAppendEntries(m pb.Message) {
|
||||
|
||||
func (r *raft) handleSnapshot(m pb.Message) {
|
||||
if r.restore(m.Snapshot) {
|
||||
r.snapshot = &m.Snapshot
|
||||
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
|
||||
} else {
|
||||
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
|
||||
@ -489,24 +508,16 @@ func stepFollower(r *raft, m pb.Message) {
|
||||
}
|
||||
}
|
||||
|
||||
func (r *raft) compact(index uint64, nodes []uint64, d []byte) {
|
||||
if index > r.raftLog.applied {
|
||||
panic(fmt.Sprintf("raft: compact index (%d) exceeds applied index (%d)", index, r.raftLog.applied))
|
||||
}
|
||||
r.raftLog.snap(d, index, r.raftLog.term(index), nodes)
|
||||
r.raftLog.compact(index)
|
||||
}
|
||||
|
||||
// restore recovers the statemachine from a snapshot. It restores the log and the
|
||||
// configuration of statemachine.
|
||||
func (r *raft) restore(s pb.Snapshot) bool {
|
||||
if s.Index <= r.raftLog.committed {
|
||||
if s.Metadata.Index <= r.raftLog.committed {
|
||||
return false
|
||||
}
|
||||
|
||||
r.raftLog.restore(s)
|
||||
r.prs = make(map[uint64]*progress)
|
||||
for _, n := range s.Nodes {
|
||||
for _, n := range s.Metadata.ConfState.Nodes {
|
||||
if n == r.id {
|
||||
r.setProgress(n, r.raftLog.lastIndex(), r.raftLog.lastIndex()+1)
|
||||
} else {
|
||||
@ -517,13 +528,7 @@ func (r *raft) restore(s pb.Snapshot) bool {
|
||||
}
|
||||
|
||||
func (r *raft) needSnapshot(i uint64) bool {
|
||||
if i < r.raftLog.firstIndex() {
|
||||
if r.raftLog.snapshot.Term == 0 {
|
||||
panic("need non-empty snapshot")
|
||||
}
|
||||
return true
|
||||
}
|
||||
return false
|
||||
return i < r.raftLog.firstIndex()
|
||||
}
|
||||
|
||||
func (r *raft) nodes() []uint64 {
|
||||
|
@ -22,7 +22,6 @@ import (
|
||||
"math"
|
||||
"math/rand"
|
||||
"reflect"
|
||||
"sort"
|
||||
"testing"
|
||||
|
||||
pb "github.com/coreos/etcd/raft/raftpb"
|
||||
@ -514,52 +513,6 @@ func TestProposalByProxy(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestCompact(t *testing.T) {
|
||||
tests := []struct {
|
||||
compacti uint64
|
||||
nodes []uint64
|
||||
snapd []byte
|
||||
wpanic bool
|
||||
}{
|
||||
{1, []uint64{1, 2, 3}, []byte("some data"), false},
|
||||
{2, []uint64{1, 2, 3}, []byte("some data"), false},
|
||||
{4, []uint64{1, 2, 3}, []byte("some data"), true}, // compact out of range
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
func() {
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
if tt.wpanic != true {
|
||||
t.Errorf("%d: panic = %v, want %v", i, true, tt.wpanic)
|
||||
}
|
||||
}
|
||||
}()
|
||||
sm := &raft{
|
||||
state: StateLeader,
|
||||
raftLog: &raftLog{
|
||||
committed: 2,
|
||||
applied: 2,
|
||||
storage: &MemoryStorage{
|
||||
ents: []pb.Entry{{}, {Term: 1}, {Term: 1}, {Term: 1}},
|
||||
},
|
||||
},
|
||||
}
|
||||
sm.compact(tt.compacti, tt.nodes, tt.snapd)
|
||||
sort.Sort(uint64Slice(sm.raftLog.snapshot.Nodes))
|
||||
if sm.raftLog.firstIndex() != tt.compacti+1 {
|
||||
t.Errorf("%d: log.firstIndex = %d, want %d", i, sm.raftLog.firstIndex(), tt.compacti+1)
|
||||
}
|
||||
if !reflect.DeepEqual(sm.raftLog.snapshot.Nodes, tt.nodes) {
|
||||
t.Errorf("%d: snap.nodes = %v, want %v", i, sm.raftLog.snapshot.Nodes, tt.nodes)
|
||||
}
|
||||
if !reflect.DeepEqual(sm.raftLog.snapshot.Data, tt.snapd) {
|
||||
t.Errorf("%d: snap.data = %v, want %v", i, sm.raftLog.snapshot.Data, tt.snapd)
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
func TestCommit(t *testing.T) {
|
||||
tests := []struct {
|
||||
matches []uint64
|
||||
@ -944,13 +897,16 @@ func TestBcastBeat(t *testing.T) {
|
||||
offset := uint64(1000)
|
||||
// make a state machine with log.offset = 1000
|
||||
s := pb.Snapshot{
|
||||
Index: offset,
|
||||
Term: 1,
|
||||
Nodes: []uint64{1, 2, 3},
|
||||
Metadata: pb.SnapshotMetadata{
|
||||
Index: offset,
|
||||
Term: 1,
|
||||
ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
|
||||
},
|
||||
}
|
||||
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(s)
|
||||
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage)
|
||||
sm.Term = 1
|
||||
sm.restore(s)
|
||||
|
||||
sm.becomeCandidate()
|
||||
sm.becomeLeader()
|
||||
@ -1026,28 +982,28 @@ func TestRecvMsgBeat(t *testing.T) {
|
||||
|
||||
func TestRestore(t *testing.T) {
|
||||
s := pb.Snapshot{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
Nodes: []uint64{1, 2, 3},
|
||||
Metadata: pb.SnapshotMetadata{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
|
||||
},
|
||||
}
|
||||
|
||||
sm := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
||||
storage := NewMemoryStorage()
|
||||
sm := newRaft(1, []uint64{1, 2}, 10, 1, storage)
|
||||
if ok := sm.restore(s); !ok {
|
||||
t.Fatal("restore fail, want succeed")
|
||||
}
|
||||
|
||||
if sm.raftLog.lastIndex() != s.Index {
|
||||
t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Index)
|
||||
if sm.raftLog.lastIndex() != s.Metadata.Index {
|
||||
t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Metadata.Index)
|
||||
}
|
||||
if sm.raftLog.term(s.Index) != s.Term {
|
||||
t.Errorf("log.lastTerm = %d, want %d", sm.raftLog.term(s.Index), s.Term)
|
||||
if sm.raftLog.term(s.Metadata.Index) != s.Metadata.Term {
|
||||
t.Errorf("log.lastTerm = %d, want %d", sm.raftLog.term(s.Metadata.Index), s.Metadata.Term)
|
||||
}
|
||||
sg := sm.nodes()
|
||||
if !reflect.DeepEqual(sg, s.Nodes) {
|
||||
t.Errorf("sm.Nodes = %+v, want %+v", sg, s.Nodes)
|
||||
}
|
||||
if !reflect.DeepEqual(sm.raftLog.snapshot, s) {
|
||||
t.Errorf("snapshot = %+v, want %+v", sm.raftLog.snapshot, s)
|
||||
if !reflect.DeepEqual(sg, s.Metadata.ConfState.Nodes) {
|
||||
t.Errorf("sm.Nodes = %+v, want %+v", sg, s.Metadata.ConfState.Nodes)
|
||||
}
|
||||
|
||||
if ok := sm.restore(s); ok {
|
||||
@ -1056,14 +1012,17 @@ func TestRestore(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestProvideSnap(t *testing.T) {
|
||||
s := pb.Snapshot{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
Nodes: []uint64{1, 2},
|
||||
}
|
||||
sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
||||
// restore the statemachin from a snapshot
|
||||
// so it has a compacted log and a snapshot
|
||||
s := pb.Snapshot{
|
||||
Metadata: pb.SnapshotMetadata{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
|
||||
},
|
||||
}
|
||||
storage := NewMemoryStorage()
|
||||
sm := newRaft(1, []uint64{1}, 10, 1, storage)
|
||||
sm.restore(s)
|
||||
|
||||
sm.becomeCandidate()
|
||||
@ -1086,18 +1045,18 @@ func TestProvideSnap(t *testing.T) {
|
||||
|
||||
func TestRestoreFromSnapMsg(t *testing.T) {
|
||||
s := pb.Snapshot{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
Nodes: []uint64{1, 2},
|
||||
Metadata: pb.SnapshotMetadata{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
|
||||
},
|
||||
}
|
||||
m := pb.Message{Type: pb.MsgSnap, From: 1, Term: 2, Snapshot: s}
|
||||
|
||||
sm := newRaft(2, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
||||
sm.Step(m)
|
||||
|
||||
if !reflect.DeepEqual(sm.raftLog.snapshot, s) {
|
||||
t.Errorf("snapshot = %+v, want %+v", sm.raftLog.snapshot, s)
|
||||
}
|
||||
// TODO(bdarnell): what should this test?
|
||||
}
|
||||
|
||||
func TestSlowNodeRestore(t *testing.T) {
|
||||
@ -1110,15 +1069,12 @@ func TestSlowNodeRestore(t *testing.T) {
|
||||
}
|
||||
lead := nt.peers[1].(*raft)
|
||||
nextEnts(lead, nt.storage[1])
|
||||
lead.compact(lead.raftLog.applied, lead.nodes(), nil)
|
||||
//lead.compact(lead.raftLog.applied, lead.nodes(), nil)
|
||||
|
||||
nt.recover()
|
||||
// trigger a snapshot
|
||||
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
|
||||
follower := nt.peers[3].(*raft)
|
||||
if !reflect.DeepEqual(follower.raftLog.snapshot, lead.raftLog.snapshot) {
|
||||
t.Errorf("follower.snap = %+v, want %+v", follower.raftLog.snapshot, lead.raftLog.snapshot)
|
||||
}
|
||||
|
||||
// trigger a commit
|
||||
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
|
||||
|
@ -10,9 +10,11 @@
|
||||
|
||||
It has these top-level messages:
|
||||
Entry
|
||||
SnapshotMetadata
|
||||
Snapshot
|
||||
Message
|
||||
HardState
|
||||
ConfState
|
||||
ConfChange
|
||||
*/
|
||||
package raftpb
|
||||
@ -163,12 +165,21 @@ func (m *Entry) Reset() { *m = Entry{} }
|
||||
func (m *Entry) String() string { return proto.CompactTextString(m) }
|
||||
func (*Entry) ProtoMessage() {}
|
||||
|
||||
type SnapshotMetadata struct {
|
||||
ConfState ConfState `protobuf:"bytes,1,req,name=conf_state" json:"conf_state"`
|
||||
Index uint64 `protobuf:"varint,2,req,name=index" json:"index"`
|
||||
Term uint64 `protobuf:"varint,3,req,name=term" json:"term"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
}
|
||||
|
||||
func (m *SnapshotMetadata) Reset() { *m = SnapshotMetadata{} }
|
||||
func (m *SnapshotMetadata) String() string { return proto.CompactTextString(m) }
|
||||
func (*SnapshotMetadata) ProtoMessage() {}
|
||||
|
||||
type Snapshot struct {
|
||||
Data []byte `protobuf:"bytes,1,req,name=data" json:"data"`
|
||||
Nodes []uint64 `protobuf:"varint,2,rep,name=nodes" json:"nodes"`
|
||||
Index uint64 `protobuf:"varint,3,req,name=index" json:"index"`
|
||||
Term uint64 `protobuf:"varint,4,req,name=term" json:"term"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
Data []byte `protobuf:"bytes,1,opt,name=data" json:"data"`
|
||||
Metadata SnapshotMetadata `protobuf:"bytes,2,req,name=metadata" json:"metadata"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Snapshot) Reset() { *m = Snapshot{} }
|
||||
@ -204,6 +215,15 @@ func (m *HardState) Reset() { *m = HardState{} }
|
||||
func (m *HardState) String() string { return proto.CompactTextString(m) }
|
||||
func (*HardState) ProtoMessage() {}
|
||||
|
||||
type ConfState struct {
|
||||
Nodes []uint64 `protobuf:"varint,1,rep,name=nodes" json:"nodes"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
}
|
||||
|
||||
func (m *ConfState) Reset() { *m = ConfState{} }
|
||||
func (m *ConfState) String() string { return proto.CompactTextString(m) }
|
||||
func (*ConfState) ProtoMessage() {}
|
||||
|
||||
type ConfChange struct {
|
||||
ID uint64 `protobuf:"varint,1,req" json:"ID"`
|
||||
Type ConfChangeType `protobuf:"varint,2,req,enum=raftpb.ConfChangeType" json:"Type"`
|
||||
@ -330,6 +350,102 @@ func (m *Entry) Unmarshal(data []byte) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *SnapshotMetadata) Unmarshal(data []byte) error {
|
||||
l := len(data)
|
||||
index := 0
|
||||
for index < l {
|
||||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
wire |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
fieldNum := int32(wire >> 3)
|
||||
wireType := int(wire & 0x7)
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
msglen |= (int(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
postIndex := index + msglen
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
if err := m.ConfState.Unmarshal(data[index:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
index = postIndex
|
||||
case 2:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
m.Index |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
case 3:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
m.Term |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
default:
|
||||
var sizeOfWire int
|
||||
for {
|
||||
sizeOfWire++
|
||||
wire >>= 7
|
||||
if wire == 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
index -= sizeOfWire
|
||||
skippy, err := code_google_com_p_gogoprotobuf_proto.Skip(data[index:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (index + skippy) > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
|
||||
index += skippy
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *Snapshot) Unmarshal(data []byte) error {
|
||||
l := len(data)
|
||||
index := 0
|
||||
@ -372,52 +488,29 @@ func (m *Snapshot) Unmarshal(data []byte) error {
|
||||
m.Data = append(m.Data, data[index:postIndex]...)
|
||||
index = postIndex
|
||||
case 2:
|
||||
if wireType != 0 {
|
||||
if wireType != 2 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
var v uint64
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
v |= (uint64(b) & 0x7F) << shift
|
||||
msglen |= (int(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.Nodes = append(m.Nodes, v)
|
||||
case 3:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
postIndex := index + msglen
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
m.Index |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
case 4:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
m.Term |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
if err := m.Metadata.Unmarshal(data[index:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
index = postIndex
|
||||
default:
|
||||
var sizeOfWire int
|
||||
for {
|
||||
@ -739,6 +832,65 @@ func (m *HardState) Unmarshal(data []byte) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *ConfState) Unmarshal(data []byte) error {
|
||||
l := len(data)
|
||||
index := 0
|
||||
for index < l {
|
||||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
wire |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
fieldNum := int32(wire >> 3)
|
||||
wireType := int(wire & 0x7)
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
v |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.Nodes = append(m.Nodes, v)
|
||||
default:
|
||||
var sizeOfWire int
|
||||
for {
|
||||
sizeOfWire++
|
||||
wire >>= 7
|
||||
if wire == 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
index -= sizeOfWire
|
||||
skippy, err := code_google_com_p_gogoprotobuf_proto.Skip(data[index:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (index + skippy) > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
|
||||
index += skippy
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *ConfChange) Unmarshal(data []byte) error {
|
||||
l := len(data)
|
||||
index := 0
|
||||
@ -861,18 +1013,25 @@ func (m *Entry) Size() (n int) {
|
||||
}
|
||||
return n
|
||||
}
|
||||
func (m *SnapshotMetadata) Size() (n int) {
|
||||
var l int
|
||||
_ = l
|
||||
l = m.ConfState.Size()
|
||||
n += 1 + l + sovRaft(uint64(l))
|
||||
n += 1 + sovRaft(uint64(m.Index))
|
||||
n += 1 + sovRaft(uint64(m.Term))
|
||||
if m.XXX_unrecognized != nil {
|
||||
n += len(m.XXX_unrecognized)
|
||||
}
|
||||
return n
|
||||
}
|
||||
func (m *Snapshot) Size() (n int) {
|
||||
var l int
|
||||
_ = l
|
||||
l = len(m.Data)
|
||||
n += 1 + l + sovRaft(uint64(l))
|
||||
if len(m.Nodes) > 0 {
|
||||
for _, e := range m.Nodes {
|
||||
n += 1 + sovRaft(uint64(e))
|
||||
}
|
||||
}
|
||||
n += 1 + sovRaft(uint64(m.Index))
|
||||
n += 1 + sovRaft(uint64(m.Term))
|
||||
l = m.Metadata.Size()
|
||||
n += 1 + l + sovRaft(uint64(l))
|
||||
if m.XXX_unrecognized != nil {
|
||||
n += len(m.XXX_unrecognized)
|
||||
}
|
||||
@ -913,6 +1072,19 @@ func (m *HardState) Size() (n int) {
|
||||
}
|
||||
return n
|
||||
}
|
||||
func (m *ConfState) Size() (n int) {
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.Nodes) > 0 {
|
||||
for _, e := range m.Nodes {
|
||||
n += 1 + sovRaft(uint64(e))
|
||||
}
|
||||
}
|
||||
if m.XXX_unrecognized != nil {
|
||||
n += len(m.XXX_unrecognized)
|
||||
}
|
||||
return n
|
||||
}
|
||||
func (m *ConfChange) Size() (n int) {
|
||||
var l int
|
||||
_ = l
|
||||
@ -973,6 +1145,40 @@ func (m *Entry) MarshalTo(data []byte) (n int, err error) {
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
func (m *SnapshotMetadata) Marshal() (data []byte, err error) {
|
||||
size := m.Size()
|
||||
data = make([]byte, size)
|
||||
n, err := m.MarshalTo(data)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return data[:n], nil
|
||||
}
|
||||
|
||||
func (m *SnapshotMetadata) MarshalTo(data []byte) (n int, err error) {
|
||||
var i int
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
data[i] = 0xa
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.ConfState.Size()))
|
||||
n1, err := m.ConfState.MarshalTo(data[i:])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i += n1
|
||||
data[i] = 0x10
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Index))
|
||||
data[i] = 0x18
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Term))
|
||||
if m.XXX_unrecognized != nil {
|
||||
i += copy(data[i:], m.XXX_unrecognized)
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
func (m *Snapshot) Marshal() (data []byte, err error) {
|
||||
size := m.Size()
|
||||
data = make([]byte, size)
|
||||
@ -992,25 +1198,14 @@ func (m *Snapshot) MarshalTo(data []byte) (n int, err error) {
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(len(m.Data)))
|
||||
i += copy(data[i:], m.Data)
|
||||
if len(m.Nodes) > 0 {
|
||||
for _, num := range m.Nodes {
|
||||
data[i] = 0x10
|
||||
i++
|
||||
for num >= 1<<7 {
|
||||
data[i] = uint8(uint64(num)&0x7f | 0x80)
|
||||
num >>= 7
|
||||
i++
|
||||
}
|
||||
data[i] = uint8(num)
|
||||
i++
|
||||
}
|
||||
data[i] = 0x12
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Metadata.Size()))
|
||||
n2, err := m.Metadata.MarshalTo(data[i:])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
data[i] = 0x18
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Index))
|
||||
data[i] = 0x20
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Term))
|
||||
i += n2
|
||||
if m.XXX_unrecognized != nil {
|
||||
i += copy(data[i:], m.XXX_unrecognized)
|
||||
}
|
||||
@ -1067,11 +1262,11 @@ func (m *Message) MarshalTo(data []byte) (n int, err error) {
|
||||
data[i] = 0x4a
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Snapshot.Size()))
|
||||
n1, err := m.Snapshot.MarshalTo(data[i:])
|
||||
n3, err := m.Snapshot.MarshalTo(data[i:])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i += n1
|
||||
i += n3
|
||||
data[i] = 0x50
|
||||
i++
|
||||
if m.Reject {
|
||||
@ -1114,6 +1309,39 @@ func (m *HardState) MarshalTo(data []byte) (n int, err error) {
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
func (m *ConfState) Marshal() (data []byte, err error) {
|
||||
size := m.Size()
|
||||
data = make([]byte, size)
|
||||
n, err := m.MarshalTo(data)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return data[:n], nil
|
||||
}
|
||||
|
||||
func (m *ConfState) MarshalTo(data []byte) (n int, err error) {
|
||||
var i int
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.Nodes) > 0 {
|
||||
for _, num := range m.Nodes {
|
||||
data[i] = 0x8
|
||||
i++
|
||||
for num >= 1<<7 {
|
||||
data[i] = uint8(uint64(num)&0x7f | 0x80)
|
||||
num >>= 7
|
||||
i++
|
||||
}
|
||||
data[i] = uint8(num)
|
||||
i++
|
||||
}
|
||||
}
|
||||
if m.XXX_unrecognized != nil {
|
||||
i += copy(data[i:], m.XXX_unrecognized)
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
func (m *ConfChange) Marshal() (data []byte, err error) {
|
||||
size := m.Size()
|
||||
data = make([]byte, size)
|
||||
|
@ -20,11 +20,15 @@ message Entry {
|
||||
optional bytes Data = 4 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
message SnapshotMetadata {
|
||||
required ConfState conf_state = 1 [(gogoproto.nullable) = false];
|
||||
required uint64 index = 2 [(gogoproto.nullable) = false];
|
||||
required uint64 term = 3 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
message Snapshot {
|
||||
required bytes data = 1 [(gogoproto.nullable) = false];
|
||||
repeated uint64 nodes = 2 [(gogoproto.nullable) = false];
|
||||
required uint64 index = 3 [(gogoproto.nullable) = false];
|
||||
required uint64 term = 4 [(gogoproto.nullable) = false];
|
||||
optional bytes data = 1 [(gogoproto.nullable) = false];
|
||||
required SnapshotMetadata metadata = 2 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
enum MessageType {
|
||||
@ -57,6 +61,10 @@ message HardState {
|
||||
required uint64 commit = 3 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
message ConfState {
|
||||
repeated uint64 nodes = 1 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
enum ConfChangeType {
|
||||
ConfChangeAddNode = 0;
|
||||
ConfChangeRemoveNode = 1;
|
||||
|
@ -18,6 +18,7 @@ package raft
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
pb "github.com/coreos/etcd/raft/raftpb"
|
||||
@ -34,6 +35,8 @@ var ErrCompacted = errors.New("requested index is unavailable due to compaction"
|
||||
// become inoperable and refuse to participate in elections; the
|
||||
// application is responsible for cleanup and recovery in this case.
|
||||
type Storage interface {
|
||||
// HardState returns the saved HardState information.
|
||||
HardState() (pb.HardState, error)
|
||||
// Entries returns a slice of log entries in the range [lo,hi).
|
||||
Entries(lo, hi uint64) ([]pb.Entry, error)
|
||||
// Term returns the term of entry i, which must be in the range
|
||||
@ -47,10 +50,11 @@ type Storage interface {
|
||||
// available via Entries (older entries have been incorporated
|
||||
// into the latest Snapshot).
|
||||
FirstIndex() (uint64, error)
|
||||
// Compact discards all log entries prior to i.
|
||||
// TODO(bdarnell): Create a snapshot which can be used to
|
||||
// reconstruct the state at that point.
|
||||
Compact(i uint64) error
|
||||
// Snapshot returns the most recent snapshot.
|
||||
Snapshot() (pb.Snapshot, error)
|
||||
// ApplySnapshot overwrites the contents of this Storage object with
|
||||
// those of the given snapshot.
|
||||
ApplySnapshot(pb.Snapshot) error
|
||||
}
|
||||
|
||||
// MemoryStorage implements the Storage interface backed by an
|
||||
@ -61,10 +65,10 @@ type MemoryStorage struct {
|
||||
// goroutine.
|
||||
sync.Mutex
|
||||
|
||||
hardState pb.HardState
|
||||
snapshot pb.Snapshot
|
||||
// ents[i] has raft log position i+snapshot.Metadata.Index
|
||||
ents []pb.Entry
|
||||
// offset is the position of the last compaction.
|
||||
// ents[i] has raft log position i+offset.
|
||||
offset uint64
|
||||
}
|
||||
|
||||
// NewMemoryStorage creates an empty MemoryStorage.
|
||||
@ -75,50 +79,96 @@ func NewMemoryStorage() *MemoryStorage {
|
||||
}
|
||||
}
|
||||
|
||||
// HardState implements the Storage interface.
|
||||
func (ms *MemoryStorage) HardState() (pb.HardState, error) {
|
||||
return ms.hardState, nil
|
||||
}
|
||||
|
||||
// SetHardState saves the current HardState.
|
||||
func (ms *MemoryStorage) SetHardState(st pb.HardState) error {
|
||||
ms.hardState = st
|
||||
return nil
|
||||
}
|
||||
|
||||
// Entries implements the Storage interface.
|
||||
func (ms *MemoryStorage) Entries(lo, hi uint64) ([]pb.Entry, error) {
|
||||
ms.Lock()
|
||||
defer ms.Unlock()
|
||||
if lo <= ms.offset {
|
||||
offset := ms.snapshot.Metadata.Index
|
||||
if lo <= offset {
|
||||
return nil, ErrCompacted
|
||||
}
|
||||
return ms.ents[lo-ms.offset : hi-ms.offset], nil
|
||||
return ms.ents[lo-offset : hi-offset], nil
|
||||
}
|
||||
|
||||
// Term implements the Storage interface.
|
||||
func (ms *MemoryStorage) Term(i uint64) (uint64, error) {
|
||||
ms.Lock()
|
||||
defer ms.Unlock()
|
||||
if i < ms.offset || i > ms.offset+uint64(len(ms.ents)) {
|
||||
offset := ms.snapshot.Metadata.Index
|
||||
if i < offset || i > offset+uint64(len(ms.ents)) {
|
||||
return 0, ErrCompacted
|
||||
}
|
||||
return ms.ents[i-ms.offset].Term, nil
|
||||
return ms.ents[i-offset].Term, nil
|
||||
}
|
||||
|
||||
// LastIndex implements the Storage interface.
|
||||
func (ms *MemoryStorage) LastIndex() (uint64, error) {
|
||||
ms.Lock()
|
||||
defer ms.Unlock()
|
||||
return ms.offset + uint64(len(ms.ents)) - 1, nil
|
||||
return ms.snapshot.Metadata.Index + uint64(len(ms.ents)) - 1, nil
|
||||
}
|
||||
|
||||
// FirstIndex implements the Storage interface.
|
||||
func (ms *MemoryStorage) FirstIndex() (uint64, error) {
|
||||
ms.Lock()
|
||||
defer ms.Unlock()
|
||||
return ms.offset + 1, nil
|
||||
return ms.snapshot.Metadata.Index + 1, nil
|
||||
}
|
||||
|
||||
// Compact implements the Storage interface.
|
||||
func (ms *MemoryStorage) Compact(i uint64) error {
|
||||
// Snapshot implements the Storage interface.
|
||||
func (ms *MemoryStorage) Snapshot() (pb.Snapshot, error) {
|
||||
ms.Lock()
|
||||
defer ms.Unlock()
|
||||
i -= ms.offset
|
||||
return ms.snapshot, nil
|
||||
}
|
||||
|
||||
// ApplySnapshot implements the Storage interface.
|
||||
func (ms *MemoryStorage) ApplySnapshot(snap pb.Snapshot) error {
|
||||
ms.Lock()
|
||||
defer ms.Unlock()
|
||||
|
||||
ms.snapshot = snap
|
||||
ms.ents = []pb.Entry{{Term: snap.Metadata.Term, Index: snap.Metadata.Index}}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Compact discards all log entries prior to i. Creates a snapshot
|
||||
// which can be retrieved with the Snapshot() method and can be used
|
||||
// to reconstruct the state at that point.
|
||||
// If any configuration changes have been made since the last compaction,
|
||||
// the result of the last ApplyConfChange must be passed in.
|
||||
// It is the application's responsibility to not attempt to compact an index
|
||||
// greater than raftLog.applied.
|
||||
func (ms *MemoryStorage) Compact(i uint64, cs *pb.ConfState, data []byte) error {
|
||||
ms.Lock()
|
||||
defer ms.Unlock()
|
||||
offset := ms.snapshot.Metadata.Index
|
||||
if i <= offset || i > offset+uint64(len(ms.ents))-1 {
|
||||
panic(fmt.Sprintf("compact %d out of bounds (%d, %d)", i, offset,
|
||||
offset+uint64(len(ms.ents))-1))
|
||||
}
|
||||
i -= offset
|
||||
ents := make([]pb.Entry, 1, 1+uint64(len(ms.ents))-i)
|
||||
ents[0].Term = ms.ents[i].Term
|
||||
ents = append(ents, ms.ents[i+1:]...)
|
||||
ms.ents = ents
|
||||
ms.offset += i
|
||||
ms.snapshot.Metadata.Index += i
|
||||
ms.snapshot.Metadata.Term = ents[0].Term
|
||||
if cs != nil {
|
||||
ms.snapshot.Metadata.ConfState = *cs
|
||||
}
|
||||
ms.snapshot.Data = data
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -61,7 +61,7 @@ func (s *Snapshotter) SaveSnap(snapshot raftpb.Snapshot) error {
|
||||
}
|
||||
|
||||
func (s *Snapshotter) save(snapshot *raftpb.Snapshot) error {
|
||||
fname := fmt.Sprintf("%016x-%016x%s", snapshot.Term, snapshot.Index, snapSuffix)
|
||||
fname := fmt.Sprintf("%016x-%016x%s", snapshot.Metadata.Term, snapshot.Metadata.Index, snapSuffix)
|
||||
b := pbutil.MustMarshal(snapshot)
|
||||
crc := crc32.Update(0, crcTable, b)
|
||||
snap := snappb.Snapshot{Crc: crc, Data: b}
|
||||
|
@ -29,10 +29,14 @@ import (
|
||||
)
|
||||
|
||||
var testSnap = &raftpb.Snapshot{
|
||||
Data: []byte("some snapshot"),
|
||||
Nodes: []uint64{1, 2, 3},
|
||||
Index: 1,
|
||||
Term: 1,
|
||||
Data: []byte("some snapshot"),
|
||||
Metadata: raftpb.SnapshotMetadata{
|
||||
ConfState: raftpb.ConfState{
|
||||
Nodes: []uint64{1, 2, 3},
|
||||
},
|
||||
Index: 1,
|
||||
Term: 1,
|
||||
},
|
||||
}
|
||||
|
||||
func TestSaveAndLoad(t *testing.T) {
|
||||
@ -156,7 +160,7 @@ func TestLoadNewestSnap(t *testing.T) {
|
||||
}
|
||||
|
||||
newSnap := *testSnap
|
||||
newSnap.Index = 5
|
||||
newSnap.Metadata.Index = 5
|
||||
err = ss.save(&newSnap)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
|
Loading…
x
Reference in New Issue
Block a user