From 25b65905475b519bb0a9dcc1ddefd6f6c30dc2eb Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Mon, 3 Nov 2014 15:04:24 -0500 Subject: [PATCH 01/52] raft: introduce log storage interface. This change splits the raftLog.entries array into an in-memory "unstable" list and a pluggable interface for retrieving entries that have been persisted to disk. An in-memory implementation of this interface is provided which behaves the same as the old version; in a future commit etcdserver could replace the MemoryStorage with one backed by the WAL. --- etcdserver/force_cluster.go | 7 +- etcdserver/server.go | 42 ++++++---- etcdserver/server_test.go | 158 ++++++++++++++++++++---------------- raft/diff_test.go | 2 +- raft/example_test.go | 2 +- raft/log.go | 156 +++++++++++++++++++++++++++-------- raft/log_test.go | 98 ++++++++++++---------- raft/node.go | 12 +-- raft/node_bench_test.go | 2 +- raft/node_test.go | 26 +++--- raft/raft.go | 6 +- raft/raft_paper_test.go | 51 ++++++------ raft/raft_test.go | 133 ++++++++++++++++++++---------- raft/storage.go | 108 ++++++++++++++++++++++++ raft/util.go | 54 ++++++++++++ test | 2 +- 16 files changed, 605 insertions(+), 254 deletions(-) create mode 100644 raft/storage.go create mode 100644 raft/util.go diff --git a/etcdserver/force_cluster.go b/etcdserver/force_cluster.go index b0d1c8085..233648539 100644 --- a/etcdserver/force_cluster.go +++ b/etcdserver/force_cluster.go @@ -28,7 +28,7 @@ import ( "github.com/coreos/etcd/wal" ) -func restartAsStandaloneNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (types.ID, raft.Node, *wal.WAL) { +func restartAsStandaloneNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (types.ID, raft.Node, *raft.MemoryStorage, *wal.WAL) { w, id, cid, st, ents := readWAL(cfg.WALDir(), index) cfg.Cluster.SetID(cid) @@ -53,8 +53,9 @@ 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) - n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents) - return id, n, w + s := raft.NewMemoryStorage() + n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents, s) + return id, n, s, w } // getIDs returns an ordered set of IDs included in the given snapshot and diff --git a/etcdserver/server.go b/etcdserver/server.go index a33f3c227..24115944b 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -156,8 +156,9 @@ type EtcdServer struct { Cluster *Cluster - node raft.Node - store store.Store + node raft.Node + raftStorage *raft.MemoryStorage + store store.Store stats *stats.ServerStats lstats *stats.LeaderStats @@ -190,6 +191,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { st := store.New() var w *wal.WAL var n raft.Node + var s *raft.MemoryStorage var id types.ID haveWAL := wal.Exist(cfg.WALDir()) switch { @@ -204,7 +206,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { } cfg.Cluster.SetID(cl.id) cfg.Cluster.SetStore(st) - id, n, w = startNode(cfg, nil) + id, n, s, w = startNode(cfg, nil) case !haveWAL && cfg.NewCluster: if err := cfg.VerifyBootstrapConfig(); err != nil { return nil, err @@ -221,7 +223,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { } cfg.Cluster.SetStore(st) log.Printf("etcdserver: initial cluster members: %s", cfg.Cluster) - id, n, w = startNode(cfg, cfg.Cluster.MemberIDs()) + id, n, s, w = startNode(cfg, cfg.Cluster.MemberIDs()) case haveWAL: if cfg.ShouldDiscover() { log.Printf("etcdserver: warn: ignoring discovery: etcd has already been initialized and has a valid log in %q", cfg.WALDir()) @@ -241,9 +243,9 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { log.Printf("etcdserver: loaded peers from snapshot: %s", cfg.Cluster) } if !cfg.ForceNewCluster { - id, n, w = restartNode(cfg, index, snapshot) + id, n, s, w = restartNode(cfg, index, snapshot) } else { - id, n, w = restartAsStandaloneNode(cfg, index, snapshot) + id, n, s, w = restartAsStandaloneNode(cfg, index, snapshot) } default: return nil, fmt.Errorf("unsupported bootstrap config") @@ -256,12 +258,13 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { lstats := stats.NewLeaderStats(id.String()) shub := newSendHub(cfg.Transport, cfg.Cluster, sstats, lstats) - s := &EtcdServer{ - store: st, - node: n, - id: id, - attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()}, - Cluster: cfg.Cluster, + srv := &EtcdServer{ + store: st, + node: n, + raftStorage: s, + id: id, + attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()}, + Cluster: cfg.Cluster, storage: struct { *wal.WAL *snap.Snapshotter @@ -273,7 +276,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { SyncTicker: time.Tick(500 * time.Millisecond), snapCount: cfg.SnapCount, } - return s, nil + return srv, nil } // Start prepares and starts server in a new goroutine. It is no longer safe to @@ -327,6 +330,7 @@ func (s *EtcdServer) run() { } } + s.raftStorage.Append(rd.Entries) if err := s.storage.Save(rd.HardState, rd.Entries); err != nil { log.Fatalf("etcdserver: save state and entries error: %v", err) } @@ -722,7 +726,7 @@ func GetClusterFromPeers(urls []string) (*Cluster, error) { return nil, fmt.Errorf("etcdserver: could not retrieve cluster information from the given urls") } -func startNode(cfg *ServerConfig, ids []types.ID) (id types.ID, n raft.Node, w *wal.WAL) { +func startNode(cfg *ServerConfig, ids []types.ID) (id types.ID, n raft.Node, s *raft.MemoryStorage, w *wal.WAL) { var err error member := cfg.Cluster.MemberByName(cfg.Name) metadata := pbutil.MustMarshal( @@ -744,7 +748,8 @@ func startNode(cfg *ServerConfig, ids []types.ID) (id types.ID, n raft.Node, w * } id = member.ID log.Printf("etcdserver: start node %s in cluster %s", id, cfg.Cluster.ID()) - n = raft.StartNode(uint64(id), peers, 10, 1) + s = raft.NewMemoryStorage() + n = raft.StartNode(uint64(id), peers, 10, 1, s) return } @@ -762,13 +767,14 @@ func getOtherPeerURLs(cl ClusterInfo, self string) []string { return us } -func restartNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (types.ID, raft.Node, *wal.WAL) { +func restartNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (types.ID, raft.Node, *raft.MemoryStorage, *wal.WAL) { w, id, cid, st, ents := readWAL(cfg.WALDir(), index) cfg.Cluster.SetID(cid) log.Printf("etcdserver: restart member %s in cluster %s at commit index %d", id, cfg.Cluster.ID(), st.Commit) - n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents) - return id, n, w + s := raft.NewMemoryStorage() + n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents, s) + return id, n, s, w } func readWAL(waldir string, index uint64) (w *wal.WAL, id, cid types.ID, st raftpb.HardState, ents []raftpb.Entry) { diff --git a/etcdserver/server_test.go b/etcdserver/server_test.go index 25fa25bc9..8b505a056 100644 --- a/etcdserver/server_test.go +++ b/etcdserver/server_test.go @@ -519,24 +519,30 @@ func testServer(t *testing.T, ns uint64) { members := mustMakePeerSlice(t, ids...) for i := uint64(0); i < ns; i++ { id := i + 1 - n := raft.StartNode(id, members, 10, 1) + s := raft.NewMemoryStorage() + n := raft.StartNode(id, members, 10, 1, s) tk := time.NewTicker(10 * time.Millisecond) defer tk.Stop() st := store.New() cl := newCluster("abc") cl.SetStore(st) srv := &EtcdServer{ - node: n, - store: st, - sender: &fakeSender{ss}, - storage: &storageRecorder{}, - Ticker: tk.C, - Cluster: cl, + node: n, + raftStorage: s, + store: st, + sender: &fakeSender{ss}, + storage: &storageRecorder{}, + Ticker: tk.C, + Cluster: cl, } - srv.start() ss[i] = srv } + // Start the servers after they're all created to avoid races in send(). + for i := uint64(0); i < ns; i++ { + ss[i].start() + } + for i := 1; i <= 10; i++ { r := pb.Request{ Method: "PUT", @@ -587,7 +593,8 @@ func TestDoProposal(t *testing.T) { for i, tt := range tests { ctx, _ := context.WithCancel(context.Background()) - n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1) + s := raft.NewMemoryStorage() + n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s) st := &storeRecorder{} tk := make(chan time.Time) // this makes <-tk always successful, which accelerates internal clock @@ -595,12 +602,13 @@ func TestDoProposal(t *testing.T) { cl := newCluster("abc") cl.SetStore(store.New()) srv := &EtcdServer{ - node: n, - store: st, - sender: &nopSender{}, - storage: &storageRecorder{}, - Ticker: tk, - Cluster: cl, + node: n, + raftStorage: s, + store: st, + sender: &nopSender{}, + storage: &storageRecorder{}, + Ticker: tk, + Cluster: cl, } srv.start() resp, err := srv.Do(ctx, tt) @@ -623,14 +631,16 @@ func TestDoProposal(t *testing.T) { func TestDoProposalCancelled(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) // node cannot make any progress because there are two nodes - n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0, 0xBAD1), 10, 1) + s := raft.NewMemoryStorage() + n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0, 0xBAD1), 10, 1, s) st := &storeRecorder{} wait := &waitRecorder{} srv := &EtcdServer{ // TODO: use fake node for better testability - node: n, - store: st, - w: wait, + node: n, + raftStorage: s, + store: st, + w: wait, } done := make(chan struct{}) @@ -671,18 +681,20 @@ func TestDoProposalStopped(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() // node cannot make any progress because there are two nodes - n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0, 0xBAD1), 10, 1) + s := raft.NewMemoryStorage() + n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0, 0xBAD1), 10, 1, s) st := &storeRecorder{} tk := make(chan time.Time) // this makes <-tk always successful, which accelarates internal clock close(tk) srv := &EtcdServer{ // TODO: use fake node for better testability - node: n, - store: st, - sender: &nopSender{}, - storage: &storageRecorder{}, - Ticker: tk, + node: n, + raftStorage: s, + store: st, + sender: &nopSender{}, + storage: &storageRecorder{}, + Ticker: tk, } srv.start() @@ -788,11 +800,12 @@ func TestSyncTrigger(t *testing.T) { } st := make(chan time.Time, 1) srv := &EtcdServer{ - node: n, - store: &storeRecorder{}, - sender: &nopSender{}, - storage: &storageRecorder{}, - SyncTicker: st, + node: n, + raftStorage: raft.NewMemoryStorage(), + store: &storeRecorder{}, + sender: &nopSender{}, + storage: &storageRecorder{}, + SyncTicker: st, } srv.start() // trigger the server to become a leader and accept sync requests @@ -822,17 +835,19 @@ func TestSyncTrigger(t *testing.T) { // snapshot should snapshot the store and cut the persistent // TODO: node.Compact is called... we need to make the node an interface func TestSnapshot(t *testing.T) { - n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1) + s := raft.NewMemoryStorage() + n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s) defer n.Stop() st := &storeRecorder{} p := &storageRecorder{} - s := &EtcdServer{ - store: st, - storage: p, - node: n, + srv := &EtcdServer{ + store: st, + storage: p, + node: n, + raftStorage: s, } - s.snapshot(0, []uint64{1}) + srv.snapshot(0, []uint64{1}) gaction := st.Action() if len(gaction) != 1 { t.Fatalf("len(action) = %d, want 1", len(gaction)) @@ -853,8 +868,10 @@ func TestSnapshot(t *testing.T) { // Applied > SnapCount should trigger a SaveSnap event func TestTriggerSnap(t *testing.T) { ctx := context.Background() - n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1) - <-n.Ready() + s := raft.NewMemoryStorage() + n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s) + rd := <-n.Ready() + s.Append(rd.Entries) n.Advance() n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 0xBAD0}) n.Campaign(ctx) @@ -862,26 +879,27 @@ func TestTriggerSnap(t *testing.T) { p := &storageRecorder{} cl := newCluster("abc") cl.SetStore(store.New()) - s := &EtcdServer{ - store: st, - sender: &nopSender{}, - storage: p, - node: n, - snapCount: 10, - Cluster: cl, + srv := &EtcdServer{ + store: st, + sender: &nopSender{}, + storage: p, + node: n, + raftStorage: s, + snapCount: 10, + Cluster: cl, } - s.start() - for i := 0; uint64(i) < s.snapCount-1; i++ { - s.Do(ctx, pb.Request{Method: "PUT", ID: 1}) + srv.start() + for i := 0; uint64(i) < srv.snapCount-1; i++ { + srv.Do(ctx, pb.Request{Method: "PUT", ID: 1}) } time.Sleep(time.Millisecond) - s.Stop() + srv.Stop() 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(s.snapCount) + wcnt := 2 + int(srv.snapCount) if len(gaction) != wcnt { t.Fatalf("len(action) = %d, want %d", len(gaction), wcnt) } @@ -897,10 +915,11 @@ func TestRecvSnapshot(t *testing.T) { st := &storeRecorder{} p := &storageRecorder{} s := &EtcdServer{ - store: st, - sender: &nopSender{}, - storage: p, - node: n, + store: st, + sender: &nopSender{}, + storage: p, + node: n, + raftStorage: raft.NewMemoryStorage(), } s.start() @@ -925,10 +944,11 @@ func TestRecvSlowSnapshot(t *testing.T) { n := newReadyNode() st := &storeRecorder{} s := &EtcdServer{ - store: st, - sender: &nopSender{}, - storage: &storageRecorder{}, - node: n, + store: st, + sender: &nopSender{}, + storage: &storageRecorder{}, + node: n, + raftStorage: raft.NewMemoryStorage(), } s.start() @@ -959,11 +979,12 @@ func TestAddMember(t *testing.T) { cl := newTestCluster(nil) cl.SetStore(store.New()) s := &EtcdServer{ - node: n, - store: &storeRecorder{}, - sender: &nopSender{}, - storage: &storageRecorder{}, - Cluster: cl, + node: n, + raftStorage: raft.NewMemoryStorage(), + store: &storeRecorder{}, + sender: &nopSender{}, + storage: &storageRecorder{}, + Cluster: cl, } s.start() m := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"foo"}}} @@ -994,11 +1015,12 @@ func TestRemoveMember(t *testing.T) { } cl := newTestCluster([]Member{{ID: 1234}}) s := &EtcdServer{ - node: n, - store: &storeRecorder{}, - sender: &nopSender{}, - storage: &storageRecorder{}, - Cluster: cl, + node: n, + raftStorage: raft.NewMemoryStorage(), + store: &storeRecorder{}, + sender: &nopSender{}, + storage: &storageRecorder{}, + Cluster: cl, } s.start() err := s.RemoveMember(context.TODO(), 1234) diff --git a/raft/diff_test.go b/raft/diff_test.go index 8112f49d2..d8032b509 100644 --- a/raft/diff_test.go +++ b/raft/diff_test.go @@ -60,7 +60,7 @@ func mustTemp(pre, body string) string { func ltoa(l *raftLog) string { s := fmt.Sprintf("committed: %d\n", l.committed) s += fmt.Sprintf("applied: %d\n", l.applied) - for i, e := range l.ents { + for i, e := range l.allEntries() { s += fmt.Sprintf("#%d: %+v\n", i, e) } return s diff --git a/raft/example_test.go b/raft/example_test.go index 4d4e63aba..632bf0b94 100644 --- a/raft/example_test.go +++ b/raft/example_test.go @@ -26,7 +26,7 @@ func saveStateToDisk(st pb.HardState) {} func saveToDisk(ents []pb.Entry) {} func Example_Node() { - n := StartNode(0, nil, 0, 0) + n := StartNode(0, nil, 0, 0, nil) // stuff to n happens in other goroutines diff --git a/raft/log.go b/raft/log.go index 496a1043c..a8ec0ea98 100644 --- a/raft/log.go +++ b/raft/log.go @@ -24,33 +24,54 @@ import ( ) type raftLog struct { - ents []pb.Entry - unstable uint64 + // storage contains all stable entries since the last snapshot. + storage Storage + // unstableEnts contains all entries that have not yet been written + // to storage. + unstableEnts []pb.Entry + // unstableEnts[i] has raft log position i+unstable. Note that + // unstable may be less than the highest log position in storage; + // this means that the next write to storage will truncate the log + // before persisting unstableEnts. + unstable uint64 + // committed is the highest log position that is known to be in + // stable storage on a quorum of nodes. + // Invariant: committed < unstable committed uint64 - applied uint64 - offset uint64 - snapshot pb.Snapshot + // 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 } -func newLog() *raftLog { +func newLog(storage Storage) *raftLog { + if storage == nil { + storage = NewMemoryStorage() + } + lastIndex, err := storage.GetLastIndex() + if err != nil { + panic(err) // TODO(bdarnell) + } return &raftLog{ - ents: make([]pb.Entry, 1), - unstable: 0, - committed: 0, - applied: 0, + storage: storage, + unstable: lastIndex + 1, } } func (l *raftLog) load(ents []pb.Entry) { - if l.offset != ents[0].Index { + // TODO(bdarnell): does this method need to support other Storage impls or does it go away? + ms := l.storage.(*MemoryStorage) + if ms.offset != ents[0].Index { panic("entries loaded don't match offset index") } - l.ents = ents - l.unstable = l.offset + uint64(len(ents)) + ms.ents = ents + l.unstable = ms.offset + uint64(len(ents)) } func (l *raftLog) String() string { - return fmt.Sprintf("offset=%d committed=%d applied=%d len(ents)=%d", l.offset, l.committed, l.applied, len(l.ents)) + return fmt.Sprintf("unstable=%d committed=%d applied=%d", l.unstable, l.committed, l.applied) + } // maybeAppend returns (0, false) if the entries cannot be appended. Otherwise, @@ -78,7 +99,15 @@ func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry } func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 { - l.ents = append(l.slice(l.offset, after+1), ents...) + if after < l.unstable { + // The log is being truncated to before our current unstable + // portion, so discard it and reset unstable. + l.unstableEnts = nil + l.unstable = after + 1 + } + // Truncate any unstable entries that are being replaced, then + // append the new ones. + l.unstableEnts = append(l.unstableEnts[0:1+after-l.unstable], ents...) l.unstable = min(l.unstable, after+1) return l.lastIndex() } @@ -104,13 +133,12 @@ func (l *raftLog) findConflict(from uint64, ents []pb.Entry) uint64 { return 0 } -func (l *raftLog) unstableEnts() []pb.Entry { - ents := l.slice(l.unstable, l.lastIndex()+1) - if ents == nil { +func (l *raftLog) unstableEntries() []pb.Entry { + if len(l.unstableEnts) == 0 { return nil } - cpy := make([]pb.Entry, len(ents)) - copy(cpy, ents) + cpy := make([]pb.Entry, len(l.unstableEnts)) + copy(cpy, l.unstableEnts) return cpy } @@ -123,6 +151,25 @@ func (l *raftLog) nextEnts() (ents []pb.Entry) { return nil } +func (l *raftLog) firstIndex() uint64 { + index, err := l.storage.GetFirstIndex() + if err != nil { + panic(err) // TODO(bdarnell) + } + return index +} + +func (l *raftLog) lastIndex() uint64 { + if len(l.unstableEnts) > 0 { + return l.unstable + uint64(len(l.unstableEnts)) - 1 + } + index, err := l.storage.GetLastIndex() + if err != nil { + panic(err) // TODO(bdarnell) + } + return index +} + func (l *raftLog) appliedTo(i uint64) { if i == 0 { return @@ -137,12 +184,13 @@ func (l *raftLog) stableTo(i uint64) { if i == 0 { return } + l.unstableEnts = l.unstableEnts[i+1-l.unstable:] l.unstable = i + 1 } -func (l *raftLog) lastIndex() uint64 { return uint64(len(l.ents)) - 1 + l.offset } - -func (l *raftLog) lastTerm() uint64 { return l.term(l.lastIndex()) } +func (l *raftLog) lastTerm() uint64 { + return l.term(l.lastIndex()) +} func (l *raftLog) term(i uint64) uint64 { if e := l.at(i); e != nil { @@ -155,12 +203,19 @@ func (l *raftLog) entries(i uint64) []pb.Entry { // never send out the first entry // first entry is only used for matching // prevLogTerm - if i == l.offset { + if i == 0 { panic("cannot return the first entry in log") } return l.slice(i, l.lastIndex()+1) } +// allEntries returns all entries in the log, including the initial +// entry that is only used for prevLogTerm validation. This method +// should only be used for testing. +func (l *raftLog) allEntries() []pb.Entry { + return l.slice(l.firstIndex(), l.lastIndex()+1) +} + // isUpToDate determines if the given (lastIndex,term) log is more up-to-date // by comparing the index and term of the last entries in the existing logs. // If the logs have last entries with different terms, then the log with the @@ -193,12 +248,22 @@ func (l *raftLog) maybeCommit(maxIndex, term uint64) bool { // 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 [%d:%d]", i, l.offset, l.applied)) + 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.ents = l.slice(i, l.lastIndex()+1) l.unstable = max(i+1, l.unstable) - l.offset = i - return uint64(len(l.ents)) + firstIndex, err := l.storage.GetFirstIndex() + if err != nil { + panic(err) // TODO(bdarnell) + } + lastIndex, err := l.storage.GetLastIndex() + if err != nil { + panic(err) // TODO(bdarnell) + } + return lastIndex - firstIndex } func (l *raftLog) snap(d []byte, index, term uint64, nodes []uint64) { @@ -211,19 +276,23 @@ func (l *raftLog) snap(d []byte, index, term uint64, nodes []uint64) { } func (l *raftLog) restore(s pb.Snapshot) { - l.ents = []pb.Entry{{Term: s.Term}} + l.storage = &MemoryStorage{ + ents: []pb.Entry{{Term: s.Term}}, + offset: s.Index, + } l.unstable = s.Index + 1 + l.unstableEnts = nil l.committed = s.Index l.applied = s.Index - l.offset = s.Index l.snapshot = s } func (l *raftLog) at(i uint64) *pb.Entry { - if l.isOutOfBounds(i) { + ents := l.slice(i, i+1) + if len(ents) == 0 { return nil } - return &l.ents[i-l.offset] + return &ents[0] } // slice returns a slice of log entries from lo through hi-1, inclusive. @@ -234,18 +303,35 @@ func (l *raftLog) slice(lo uint64, hi uint64) []pb.Entry { if l.isOutOfBounds(lo) || l.isOutOfBounds(hi-1) { return nil } - return l.ents[lo-l.offset : hi-l.offset] + var ents []pb.Entry + if lo < l.unstable { + storedEnts, err := l.storage.GetEntries(lo, min(hi, l.unstable)) + if err != nil { + panic(err) // TODO(bdarnell) + } + ents = append(ents, storedEnts...) + } + if len(l.unstableEnts) > 0 && hi > l.unstable { + var firstUnstable uint64 + if lo < l.unstable { + firstUnstable = l.unstable + } else { + firstUnstable = lo + } + ents = append(ents, l.unstableEnts[firstUnstable-l.unstable:hi-l.unstable]...) + } + return ents } func (l *raftLog) isOutOfBounds(i uint64) bool { - if i < l.offset || i > l.lastIndex() { + if i < l.firstIndex() || i > l.lastIndex() { return true } return false } func (l *raftLog) isOutOfAppliedBounds(i uint64) bool { - if i < l.offset || i > l.applied { + if i < l.firstIndex() || i > l.applied { return true } return false diff --git a/raft/log_test.go b/raft/log_test.go index 9ea3aba51..d5950315a 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -49,7 +49,7 @@ func TestFindConflict(t *testing.T) { } for i, tt := range tests { - raftLog := newLog() + raftLog := newLog(nil) raftLog.append(raftLog.lastIndex(), previousEnts...) gconflict := raftLog.findConflict(tt.from, tt.ents) @@ -61,7 +61,7 @@ func TestFindConflict(t *testing.T) { func TestIsUpToDate(t *testing.T) { previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}} - raftLog := newLog() + raftLog := newLog(nil) raftLog.append(raftLog.lastIndex(), previousEnts...) tests := []struct { lastIndex uint64 @@ -92,7 +92,6 @@ func TestIsUpToDate(t *testing.T) { func TestAppend(t *testing.T) { previousEnts := []pb.Entry{{Term: 1}, {Term: 2}} - previousUnstable := uint64(3) tests := []struct { after uint64 ents []pb.Entry @@ -133,9 +132,10 @@ func TestAppend(t *testing.T) { } for i, tt := range tests { - raftLog := newLog() - raftLog.append(raftLog.lastIndex(), previousEnts...) - raftLog.unstable = previousUnstable + storage := NewMemoryStorage() + storage.Append(previousEnts) + raftLog := newLog(storage) + index := raftLog.append(tt.after, tt.ents...) if index != tt.windex { t.Errorf("#%d: lastIndex = %d, want %d", i, index, tt.windex) @@ -241,7 +241,7 @@ func TestLogMaybeAppend(t *testing.T) { } for i, tt := range tests { - raftLog := newLog() + raftLog := newLog(nil) raftLog.append(raftLog.lastIndex(), previousEnts...) raftLog.committed = commit func() { @@ -278,40 +278,50 @@ func TestLogMaybeAppend(t *testing.T) { // a compaction. func TestCompactionSideEffects(t *testing.T) { var i uint64 + // Populate the log with 1000 entries; 750 in stable storage and 250 in unstable. lastIndex := uint64(1000) + unstableIndex := uint64(750) lastTerm := lastIndex - raftLog := newLog() - - for i = 0; i < lastIndex; i++ { - raftLog.append(uint64(i), pb.Entry{Term: uint64(i + 1), Index: uint64(i + 1)}) + storage := NewMemoryStorage() + for i = 1; i <= unstableIndex; i++ { + storage.Append([]pb.Entry{{Term: uint64(i), Index: uint64(i)}}) + } + raftLog := newLog(storage) + for i = unstableIndex; i < lastIndex; i++ { + raftLog.append(i, pb.Entry{Term: uint64(i + 1), Index: uint64(i + 1)}) + } + + ok := raftLog.maybeCommit(lastIndex, lastTerm) + if !ok { + t.Fatalf("maybeCommit returned false") } - raftLog.maybeCommit(lastIndex, lastTerm) raftLog.appliedTo(raftLog.committed) - raftLog.compact(500) + offset := uint64(500) + raftLog.compact(offset) if raftLog.lastIndex() != lastIndex { t.Errorf("lastIndex = %d, want %d", raftLog.lastIndex(), lastIndex) } - for i := raftLog.offset; i <= raftLog.lastIndex(); i++ { + for i := offset; i <= raftLog.lastIndex(); i++ { if raftLog.term(i) != i { t.Errorf("term(%d) = %d, want %d", i, raftLog.term(i), i) } } - for i := raftLog.offset; i <= raftLog.lastIndex(); i++ { + for i := offset; i <= raftLog.lastIndex(); i++ { if !raftLog.matchTerm(i, i) { t.Errorf("matchTerm(%d) = false, want true", i) } } - unstableEnts := raftLog.unstableEnts() - if g := len(unstableEnts); g != 500 { - t.Errorf("len(unstableEntries) = %d, want = %d", g, 500) + unstableEnts := raftLog.unstableEntries() + if g := len(unstableEnts); g != 250 { + t.Errorf("len(unstableEntries) = %d, want = %d", g, 250) } - if unstableEnts[0].Index != 501 { - t.Errorf("Index = %d, want = %d", unstableEnts[0].Index, 501) + if unstableEnts[0].Index != 751 { + t.Errorf("Index = %d, want = %d", unstableEnts[0].Index, 751) } prev := raftLog.lastIndex() @@ -338,10 +348,11 @@ func TestUnstableEnts(t *testing.T) { } for i, tt := range tests { - raftLog := newLog() - raftLog.append(0, previousEnts...) - raftLog.unstable = tt.unstable - ents := raftLog.unstableEnts() + storage := NewMemoryStorage() + storage.Append(previousEnts[:tt.unstable-1]) + raftLog := newLog(storage) + raftLog.append(raftLog.lastIndex(), previousEnts[tt.unstable-1:]...) + ents := raftLog.unstableEntries() raftLog.stableTo(raftLog.lastIndex()) if !reflect.DeepEqual(ents, tt.wents) { t.Errorf("#%d: unstableEnts = %+v, want %+v", i, ents, tt.wents) @@ -374,22 +385,23 @@ func TestCompaction(t *testing.T) { defer func() { if r := recover(); r != nil { if tt.wallow == true { - t.Errorf("%d: allow = %v, want %v", i, false, true) + t.Errorf("%d: allow = %v, want %v: %v", i, false, true, r) } } }() - raftLog := newLog() + storage := NewMemoryStorage() for i := uint64(0); i < tt.lastIndex; i++ { - raftLog.append(uint64(i), pb.Entry{}) + storage.Append([]pb.Entry{{}}) } + raftLog := newLog(storage) raftLog.maybeCommit(tt.applied, 0) raftLog.appliedTo(raftLog.committed) for j := 0; j < len(tt.compact); j++ { raftLog.compact(tt.compact[j]) - if len(raftLog.ents) != tt.wleft[j] { - t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.ents), tt.wleft[j]) + if len(raftLog.allEntries()) != tt.wleft[j] { + t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.allEntries()), tt.wleft[j]) } } }() @@ -398,7 +410,7 @@ func TestCompaction(t *testing.T) { func TestLogRestore(t *testing.T) { var i uint64 - raftLog := newLog() + raftLog := newLog(nil) for i = 0; i < 100; i++ { raftLog.append(i, pb.Entry{Term: i + 1}) } @@ -408,11 +420,11 @@ func TestLogRestore(t *testing.T) { raftLog.restore(pb.Snapshot{Index: index, Term: term}) // only has the guard entry - if len(raftLog.ents) != 1 { - t.Errorf("len = %d, want 0", len(raftLog.ents)) + if len(raftLog.allEntries()) != 1 { + t.Errorf("len = %d, want 1", len(raftLog.allEntries())) } - if raftLog.offset != index { - t.Errorf("offset = %d, want %d", raftLog.offset, index) + if raftLog.firstIndex() != index { + t.Errorf("firstIndex = %d, want %d", raftLog.firstIndex(), index) } if raftLog.applied != index { t.Errorf("applied = %d, want %d", raftLog.applied, index) @@ -431,7 +443,9 @@ func TestLogRestore(t *testing.T) { func TestIsOutOfBounds(t *testing.T) { offset := uint64(100) num := uint64(100) - l := &raftLog{offset: offset, ents: make([]pb.Entry, num)} + l := newLog(nil) + l.restore(pb.Snapshot{Index: offset}) + l.append(offset, make([]pb.Entry, num)...) tests := []struct { index uint64 @@ -440,8 +454,8 @@ func TestIsOutOfBounds(t *testing.T) { {offset - 1, true}, {offset, false}, {offset + num/2, false}, - {offset + num - 1, false}, - {offset + num, true}, + {offset + num, false}, + {offset + num + 1, true}, } for i, tt := range tests { @@ -457,9 +471,10 @@ func TestAt(t *testing.T) { offset := uint64(100) num := uint64(100) - l := &raftLog{offset: offset} + l := newLog(nil) + l.restore(pb.Snapshot{Index: offset}) for i = 0; i < num; i++ { - l.ents = append(l.ents, pb.Entry{Term: i}) + l.append(offset+i-1, pb.Entry{Term: i}) } tests := []struct { @@ -486,9 +501,10 @@ func TestSlice(t *testing.T) { offset := uint64(100) num := uint64(100) - l := &raftLog{offset: offset} + l := newLog(nil) + l.restore(pb.Snapshot{Index: offset}) for i = 0; i < num; i++ { - l.ents = append(l.ents, pb.Entry{Term: i}) + l.append(offset+i-1, pb.Entry{Term: i}) } tests := []struct { diff --git a/raft/node.go b/raft/node.go index df299c56a..c607dd21c 100644 --- a/raft/node.go +++ b/raft/node.go @@ -144,9 +144,9 @@ type Peer struct { // StartNode returns a new Node given a unique raft id, a list of raft peers, and // the election and heartbeat timeouts in units of ticks. // It also builds ConfChangeAddNode entry for each peer and puts them at the head of the log. -func StartNode(id uint64, peers []Peer, election, heartbeat int) Node { +func StartNode(id uint64, peers []Peer, election, heartbeat int, storage Storage) Node { n := newNode() - r := newRaft(id, nil, election, heartbeat) + r := newRaft(id, nil, election, heartbeat, storage) for _, peer := range peers { cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context} @@ -166,9 +166,11 @@ func StartNode(id uint64, peers []Peer, election, heartbeat int) Node { // 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. -func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, ents []pb.Entry) Node { +// 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, ents []pb.Entry, storage Storage) Node { n := newNode() - r := newRaft(id, nil, election, heartbeat) + r := newRaft(id, nil, election, heartbeat, storage) if snapshot != nil { r.restore(*snapshot) } @@ -387,7 +389,7 @@ func (n *node) Compact(index uint64, nodes []uint64, d []byte) { func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState, prevSnapi uint64) Ready { rd := Ready{ - Entries: r.raftLog.unstableEnts(), + Entries: r.raftLog.unstableEntries(), CommittedEntries: r.raftLog.nextEnts(), Messages: r.msgs, } diff --git a/raft/node_bench_test.go b/raft/node_bench_test.go index 78891e4df..dfabdb6fe 100644 --- a/raft/node_bench_test.go +++ b/raft/node_bench_test.go @@ -27,7 +27,7 @@ func BenchmarkOneNode(b *testing.B) { defer cancel() n := newNode() - r := newRaft(1, []uint64{1}, 10, 1) + r := newRaft(1, []uint64{1}, 10, 1, nil) go n.run(r) defer n.Stop() diff --git a/raft/node_test.go b/raft/node_test.go index e25513f67..d6be8ed29 100644 --- a/raft/node_test.go +++ b/raft/node_test.go @@ -112,7 +112,7 @@ func TestNodeStepUnblock(t *testing.T) { // who is the current leader. func TestBlockProposal(t *testing.T) { n := newNode() - r := newRaft(1, []uint64{1}, 10, 1) + r := newRaft(1, []uint64{1}, 10, 1, nil) go n.run(r) defer n.Stop() @@ -175,7 +175,6 @@ func TestNode(t *testing.T) { SoftState: &SoftState{Lead: 1, Nodes: []uint64{1}, RaftState: StateLeader}, HardState: raftpb.HardState{Term: 1, Commit: 2}, Entries: []raftpb.Entry{ - {}, {Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata}, {Term: 1, Index: 2}, }, @@ -190,12 +189,15 @@ func TestNode(t *testing.T) { CommittedEntries: []raftpb.Entry{{Term: 1, Index: 3, Data: []byte("foo")}}, }, } - n := StartNode(1, []Peer{{ID: 1}}, 10, 1) + storage := NewMemoryStorage() + n := StartNode(1, []Peer{{ID: 1}}, 10, 1, storage) n.ApplyConfChange(cc) n.Campaign(ctx) - if g := <-n.Ready(); !reflect.DeepEqual(g, wants[0]) { + g := <-n.Ready() + if !reflect.DeepEqual(g, wants[0]) { t.Errorf("#%d: g = %+v,\n w %+v", 1, g, wants[0]) } else { + storage.Append(g.Entries) n.Advance() } @@ -203,6 +205,7 @@ func TestNode(t *testing.T) { if g := <-n.Ready(); !reflect.DeepEqual(g, wants[1]) { t.Errorf("#%d: g = %+v,\n w %+v", 2, g, wants[1]) } else { + storage.Append(g.Entries) n.Advance() } @@ -227,7 +230,7 @@ func TestNodeRestart(t *testing.T) { CommittedEntries: entries[1 : st.Commit+1], } - n := RestartNode(1, 10, 1, nil, st, entries) + n := RestartNode(1, 10, 1, nil, st, entries, nil) if g := <-n.Ready(); !reflect.DeepEqual(g, want) { t.Errorf("g = %+v,\n w %+v", g, want) } else { @@ -246,7 +249,8 @@ func TestNodeRestart(t *testing.T) { func TestNodeCompact(t *testing.T) { ctx := context.Background() n := newNode() - r := newRaft(1, []uint64{1}, 10, 1) + storage := NewMemoryStorage() + r := newRaft(1, []uint64{1}, 10, 1, storage) go n.run(r) n.Campaign(ctx) @@ -261,7 +265,8 @@ func TestNodeCompact(t *testing.T) { testutil.ForceGosched() select { - case <-n.Ready(): + case rd := <-n.Ready(): + storage.Append(rd.Entries) n.Advance() default: t.Fatalf("unexpected proposal failure: unable to commit entry") @@ -274,6 +279,7 @@ func TestNodeCompact(t *testing.T) { 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") @@ -288,8 +294,8 @@ func TestNodeCompact(t *testing.T) { } n.Stop() - if r.raftLog.offset != w.Index { - t.Errorf("log.offset = %d, want %d", r.raftLog.offset, w.Index) + if r.raftLog.firstIndex() != w.Index { + t.Errorf("log.offset = %d, want %d", r.raftLog.firstIndex(), w.Index) } } @@ -297,7 +303,7 @@ func TestNodeAdvance(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - n := StartNode(1, []Peer{{ID: 1}}, 10, 1) + n := StartNode(1, []Peer{{ID: 1}}, 10, 1, nil) n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1}) n.Campaign(ctx) <-n.Ready() diff --git a/raft/raft.go b/raft/raft.go index 3babd1f26..81214fc5a 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -119,14 +119,14 @@ type raft struct { step stepFunc } -func newRaft(id uint64, peers []uint64, election, heartbeat int) *raft { +func newRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage) *raft { if id == None { panic("cannot use none id") } r := &raft{ id: id, lead: None, - raftLog: newLog(), + raftLog: newLog(storage), prs: make(map[uint64]*progress), electionTimeout: election, heartbeatTimeout: heartbeat, @@ -517,7 +517,7 @@ func (r *raft) restore(s pb.Snapshot) bool { } func (r *raft) needSnapshot(i uint64) bool { - if i < r.raftLog.offset { + if i < r.raftLog.firstIndex() { if r.raftLog.snapshot.Term == 0 { panic("need non-empty snapshot") } diff --git a/raft/raft_paper_test.go b/raft/raft_paper_test.go index 71e8a5cf5..c03a88cab 100644 --- a/raft/raft_paper_test.go +++ b/raft/raft_paper_test.go @@ -52,7 +52,7 @@ func TestLeaderUpdateTermFromMessage(t *testing.T) { // it immediately reverts to follower state. // Reference: section 5.1 func testUpdateTermFromMessage(t *testing.T, state StateType) { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) switch state { case StateFollower: r.becomeFollower(1, 2) @@ -82,7 +82,7 @@ func TestRejectStaleTermMessage(t *testing.T) { fakeStep := func(r *raft, m pb.Message) { called = true } - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.step = fakeStep r.loadState(pb.HardState{Term: 2}) @@ -96,7 +96,7 @@ func TestRejectStaleTermMessage(t *testing.T) { // TestStartAsFollower tests that when servers start up, they begin as followers. // Reference: section 5.2 func TestStartAsFollower(t *testing.T) { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) if r.state != StateFollower { t.Errorf("state = %s, want %s", r.state, StateFollower) } @@ -109,7 +109,7 @@ func TestStartAsFollower(t *testing.T) { func TestLeaderBcastBeat(t *testing.T) { // heartbeat interval hi := 1 - r := newRaft(1, []uint64{1, 2, 3}, 10, hi) + r := newRaft(1, []uint64{1, 2, 3}, 10, hi, nil) r.becomeCandidate() r.becomeLeader() for i := 0; i < 10; i++ { @@ -151,7 +151,7 @@ func TestCandidateStartNewElection(t *testing.T) { func testNonleaderStartElection(t *testing.T, state StateType) { // election timeout et := 10 - r := newRaft(1, []uint64{1, 2, 3}, et, 1) + r := newRaft(1, []uint64{1, 2, 3}, et, 1, nil) switch state { case StateFollower: r.becomeFollower(1, 2) @@ -215,7 +215,7 @@ func TestLeaderElectionInOneRoundRPC(t *testing.T) { {5, map[uint64]bool{}, StateCandidate}, } for i, tt := range tests { - r := newRaft(1, idsBySize(tt.size), 10, 1) + r := newRaft(1, idsBySize(tt.size), 10, 1, nil) r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) for id, vote := range tt.votes { @@ -248,7 +248,7 @@ func TestFollowerVote(t *testing.T) { {2, 1, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.loadState(pb.HardState{Term: 1, Vote: tt.vote}) r.Step(pb.Message{From: tt.nvote, To: 1, Term: 1, Type: pb.MsgVote}) @@ -274,7 +274,7 @@ func TestCandidateFallback(t *testing.T) { {From: 2, To: 1, Term: 2, Type: pb.MsgApp}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) if r.state != StateCandidate { t.Fatalf("unexpected state = %s, want %s", r.state, StateCandidate) @@ -303,7 +303,7 @@ func TestCandidateElectionTimeoutRandomized(t *testing.T) { // Reference: section 5.2 func testNonleaderElectionTimeoutRandomized(t *testing.T, state StateType) { et := 10 - r := newRaft(1, []uint64{1, 2, 3}, et, 1) + r := newRaft(1, []uint64{1, 2, 3}, et, 1, nil) timeouts := make(map[int]bool) for round := 0; round < 50*et; round++ { switch state { @@ -345,7 +345,7 @@ func testNonleadersElectionTimeoutNonconflict(t *testing.T, state StateType) { rs := make([]*raft, size) ids := idsBySize(size) for k := range rs { - rs[k] = newRaft(ids[k], ids, et, 1) + rs[k] = newRaft(ids[k], ids, et, 1, nil) } conflicts := 0 for round := 0; round < 1000; round++ { @@ -387,7 +387,7 @@ func testNonleadersElectionTimeoutNonconflict(t *testing.T, state StateType) { // Also, it writes the new entry into stable storage. // Reference: section 5.3 func TestLeaderStartReplication(t *testing.T) { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.becomeCandidate() r.becomeLeader() commitNoopEntry(r) @@ -412,7 +412,7 @@ func TestLeaderStartReplication(t *testing.T) { if !reflect.DeepEqual(msgs, wmsgs) { t.Errorf("msgs = %+v, want %+v", msgs, wmsgs) } - if g := r.raftLog.unstableEnts(); !reflect.DeepEqual(g, wents) { + if g := r.raftLog.unstableEntries(); !reflect.DeepEqual(g, wents) { t.Errorf("ents = %+v, want %+v", g, wents) } } @@ -425,7 +425,7 @@ func TestLeaderStartReplication(t *testing.T) { // servers eventually find out. // Reference: section 5.3 func TestLeaderCommitEntry(t *testing.T) { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.becomeCandidate() r.becomeLeader() commitNoopEntry(r) @@ -478,7 +478,7 @@ func TestLeaderAcknowledgeCommit(t *testing.T) { {5, map[uint64]bool{2: true, 3: true, 4: true, 5: true}, true}, } for i, tt := range tests { - r := newRaft(1, idsBySize(tt.size), 10, 1) + r := newRaft(1, idsBySize(tt.size), 10, 1, nil) r.becomeCandidate() r.becomeLeader() commitNoopEntry(r) @@ -510,7 +510,7 @@ func TestLeaderCommitPrecedingEntries(t *testing.T) { {{Term: 1, Index: 1}}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.loadEnts(append([]pb.Entry{{}}, tt...)) r.loadState(pb.HardState{Term: 2}) r.becomeCandidate() @@ -566,7 +566,7 @@ func TestFollowerCommitEntry(t *testing.T) { }, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.becomeFollower(1, 2) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 1, Entries: tt.ents, Commit: tt.commit}) @@ -601,7 +601,7 @@ func TestFollowerCheckMsgApp(t *testing.T) { {3, 3, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.loadEnts(ents) r.loadState(pb.HardState{Commit: 2}) r.becomeFollower(2, 2) @@ -656,16 +656,16 @@ func TestFollowerAppendEntries(t *testing.T) { }, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.loadEnts([]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}}) r.becomeFollower(2, 2) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 2, LogTerm: tt.term, Index: tt.index, Entries: tt.ents}) - if g := r.raftLog.ents; !reflect.DeepEqual(g, tt.wents) { + if g := r.raftLog.allEntries(); !reflect.DeepEqual(g, tt.wents) { t.Errorf("#%d: ents = %+v, want %+v", i, g, tt.wents) } - if g := r.raftLog.unstableEnts(); !reflect.DeepEqual(g, tt.wunstable) { + if g := r.raftLog.unstableEntries(); !reflect.DeepEqual(g, tt.wunstable) { t.Errorf("#%d: unstableEnts = %+v, want %+v", i, g, tt.wunstable) } } @@ -724,10 +724,10 @@ func TestLeaderSyncFollowerLog(t *testing.T) { }, } for i, tt := range tests { - lead := newRaft(1, []uint64{1, 2, 3}, 10, 1) + lead := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) lead.loadEnts(ents) lead.loadState(pb.HardState{Commit: lead.raftLog.lastIndex(), Term: term}) - follower := newRaft(2, []uint64{1, 2, 3}, 10, 1) + follower := newRaft(2, []uint64{1, 2, 3}, 10, 1, nil) follower.loadEnts(tt) follower.loadState(pb.HardState{Term: term - 1}) // It is necessary to have a three-node cluster. @@ -757,7 +757,7 @@ func TestVoteRequest(t *testing.T) { {[]pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}, 3}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) r.Step(pb.Message{ From: 2, To: 1, Type: pb.MsgApp, Term: tt.wterm - 1, LogTerm: 0, Index: 0, Entries: tt.ents, }) @@ -818,7 +818,7 @@ func TestVoter(t *testing.T) { {[]pb.Entry{{}, {Term: 2, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2}, 10, 1) + r := newRaft(1, []uint64{1, 2}, 10, 1, nil) r.loadEnts(tt.ents) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgVote, Term: 3, LogTerm: tt.logterm, Index: tt.index}) @@ -853,7 +853,7 @@ func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) { {3, 3}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2}, 10, 1) + r := newRaft(1, []uint64{1, 2}, 10, 1, nil) r.loadEnts(ents) r.loadState(pb.HardState{Term: 2}) // become leader at term 3 @@ -891,6 +891,7 @@ func commitNoopEntry(r *raft) { } // ignore further messages to refresh followers' commmit index r.readMessages() + r.raftLog.storage.(*MemoryStorage).Append(r.raftLog.unstableEntries()) r.raftLog.appliedTo(r.raftLog.committed) r.raftLog.stableTo(r.raftLog.lastIndex()) } diff --git a/raft/raft_test.go b/raft/raft_test.go index 2acf2e636..5544fda35 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -30,6 +30,11 @@ import ( // nextEnts returns the appliable entries and updates the applied index func nextEnts(r *raft) (ents []pb.Entry) { + // Transfer all unstable entries to "stable" storage. + memStorage := r.raftLog.storage.(*MemoryStorage) + memStorage.Append(r.raftLog.unstableEntries()) + r.raftLog.stableTo(r.raftLog.lastIndex()) + ents = r.raftLog.nextEnts() r.raftLog.appliedTo(r.raftLog.committed) return ents @@ -280,9 +285,9 @@ func TestCommitWithoutNewTermEntry(t *testing.T) { } func TestDuelingCandidates(t *testing.T) { - a := newRaft(1, []uint64{1, 2, 3}, 10, 1) - b := newRaft(2, []uint64{1, 2, 3}, 10, 1) - c := newRaft(3, []uint64{1, 2, 3}, 10, 1) + a := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + b := newRaft(2, []uint64{1, 2, 3}, 10, 1, nil) + c := newRaft(3, []uint64{1, 2, 3}, 10, 1, nil) nt := newNetwork(a, b, c) nt.cut(1, 3) @@ -293,7 +298,11 @@ func TestDuelingCandidates(t *testing.T) { nt.recover() nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup}) - wlog := &raftLog{ents: []pb.Entry{{}, pb.Entry{Data: nil, Term: 1, Index: 1}}, committed: 1} + wlog := &raftLog{ + storage: &MemoryStorage{ents: []pb.Entry{{}, pb.Entry{Data: nil, Term: 1, Index: 1}}}, + committed: 1, + unstable: 2, + } tests := []struct { sm *raft state StateType @@ -302,7 +311,7 @@ func TestDuelingCandidates(t *testing.T) { }{ {a, StateFollower, 2, wlog}, {b, StateFollower, 2, wlog}, - {c, StateFollower, 2, newLog()}, + {c, StateFollower, 2, newLog(nil)}, } for i, tt := range tests { @@ -345,7 +354,13 @@ func TestCandidateConcede(t *testing.T) { if g := a.Term; g != 1 { t.Errorf("term = %d, want %d", g, 1) } - wantLog := ltoa(&raftLog{ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}}, committed: 2}) + wantLog := ltoa(&raftLog{ + storage: &MemoryStorage{ + ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}}, + }, + unstable: 3, + committed: 2, + }) for i, p := range tt.peers { if sm, ok := p.(*raft); ok { l := ltoa(sm.raftLog) @@ -378,10 +393,13 @@ func TestOldMessages(t *testing.T) { tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgApp, Term: 1, Entries: []pb.Entry{{Term: 1}}}) l := &raftLog{ - ents: []pb.Entry{ - {}, {Data: nil, Term: 1, Index: 1}, - {Data: nil, Term: 2, Index: 2}, {Data: nil, Term: 3, Index: 3}, + storage: &MemoryStorage{ + ents: []pb.Entry{ + {}, {Data: nil, Term: 1, Index: 1}, + {Data: nil, Term: 2, Index: 2}, {Data: nil, Term: 3, Index: 3}, + }, }, + unstable: 4, committed: 3, } base := ltoa(l) @@ -432,9 +450,14 @@ func TestProposal(t *testing.T) { send(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}}) - wantLog := newLog() + wantLog := newLog(nil) if tt.success { - wantLog = &raftLog{ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}}, committed: 2} + wantLog = &raftLog{ + storage: &MemoryStorage{ + ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}}, + }, + unstable: 3, + committed: 2} } base := ltoa(wantLog) for i, p := range tt.peers { @@ -468,7 +491,12 @@ func TestProposalByProxy(t *testing.T) { // propose via follower tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}}) - wantLog := &raftLog{ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Data: data, Index: 2}}, committed: 2} + wantLog := &raftLog{ + storage: &MemoryStorage{ + ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Data: data, Index: 2}}, + }, + unstable: 3, + committed: 2} base := ltoa(wantLog) for i, p := range tt.peers { if sm, ok := p.(*raft); ok { @@ -513,13 +541,15 @@ func TestCompact(t *testing.T) { raftLog: &raftLog{ committed: 2, applied: 2, - ents: []pb.Entry{{}, {Term: 1}, {Term: 1}, {Term: 1}}, + 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.offset != tt.compacti { - t.Errorf("%d: log.offset = %d, want %d", i, sm.raftLog.offset, tt.compacti) + if sm.raftLog.firstIndex() != tt.compacti { + t.Errorf("%d: log.firstIndex = %d, want %d", i, sm.raftLog.firstIndex(), tt.compacti) } if !reflect.DeepEqual(sm.raftLog.snapshot.Nodes, tt.nodes) { t.Errorf("%d: snap.nodes = %v, want %v", i, sm.raftLog.snapshot.Nodes, tt.nodes) @@ -564,7 +594,11 @@ func TestCommit(t *testing.T) { for j := 0; j < len(tt.matches); j++ { prs[uint64(j)] = &progress{tt.matches[j], tt.matches[j] + 1} } - sm := &raft{raftLog: &raftLog{ents: tt.logs}, prs: prs, HardState: pb.HardState{Term: tt.smTerm}} + sm := &raft{ + raftLog: &raftLog{storage: &MemoryStorage{ents: tt.logs}, unstable: uint64(len(tt.logs))}, + prs: prs, + HardState: pb.HardState{Term: tt.smTerm}, + } sm.maybeCommit() if g := sm.raftLog.committed; g != tt.w { t.Errorf("#%d: committed = %d, want %d", i, g, tt.w) @@ -586,7 +620,7 @@ func TestIsElectionTimeout(t *testing.T) { } for i, tt := range tests { - sm := newRaft(1, []uint64{1}, 10, 1) + sm := newRaft(1, []uint64{1}, 10, 1, nil) sm.elapsed = tt.elapse c := 0 for j := 0; j < 10000; j++ { @@ -611,7 +645,7 @@ func TestStepIgnoreOldTermMsg(t *testing.T) { fakeStep := func(r *raft, m pb.Message) { called = true } - sm := newRaft(1, []uint64{1}, 10, 1) + sm := newRaft(1, []uint64{1}, 10, 1, nil) sm.step = fakeStep sm.Term = 2 sm.Step(pb.Message{Type: pb.MsgApp, Term: sm.Term - 1}) @@ -654,7 +688,11 @@ func TestHandleMsgApp(t *testing.T) { sm := &raft{ state: StateFollower, HardState: pb.HardState{Term: 2}, - raftLog: &raftLog{committed: 0, ents: []pb.Entry{{}, {Term: 1}, {Term: 2}}}, + raftLog: &raftLog{ + committed: 0, + storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 1}, {Term: 2}}}, + unstable: 3, + }, } sm.handleAppendEntries(tt.m) @@ -709,7 +747,7 @@ func TestRecvMsgVote(t *testing.T) { } for i, tt := range tests { - sm := newRaft(1, []uint64{1}, 10, 1) + sm := newRaft(1, []uint64{1}, 10, 1, nil) sm.state = tt.state switch tt.state { case StateFollower: @@ -720,7 +758,10 @@ func TestRecvMsgVote(t *testing.T) { sm.step = stepLeader } sm.HardState = pb.HardState{Vote: tt.voteFor} - sm.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 2}, {Term: 2}}} + sm.raftLog = &raftLog{ + storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 2}, {Term: 2}}}, + unstable: 3, + } sm.Step(pb.Message{Type: pb.MsgVote, From: 2, Index: tt.i, LogTerm: tt.term}) @@ -766,7 +807,7 @@ func TestStateTransition(t *testing.T) { } }() - sm := newRaft(1, []uint64{1}, 10, 1) + sm := newRaft(1, []uint64{1}, 10, 1, nil) sm.state = tt.from switch tt.to { @@ -805,7 +846,7 @@ func TestAllServerStepdown(t *testing.T) { tterm := uint64(3) for i, tt := range tests { - sm := newRaft(1, []uint64{1, 2, 3}, 10, 1) + sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) switch tt.state { case StateFollower: sm.becomeFollower(1, None) @@ -825,8 +866,8 @@ func TestAllServerStepdown(t *testing.T) { if sm.Term != tt.wterm { t.Errorf("#%d.%d term = %v , want %v", i, j, sm.Term, tt.wterm) } - if uint64(len(sm.raftLog.ents)) != tt.windex { - t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.ents), tt.windex) + if uint64(len(sm.raftLog.allEntries())) != tt.windex { + t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.allEntries()), tt.windex) } wlead := uint64(2) if msgType == pb.MsgVote { @@ -861,8 +902,11 @@ func TestLeaderAppResp(t *testing.T) { for i, tt := range tests { // sm term is 1 after it becomes the leader. // thus the last log term must be 1 to be committed. - sm := newRaft(1, []uint64{1, 2, 3}, 10, 1) - sm.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}} + sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + sm.raftLog = &raftLog{ + storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}, + unstable: 3, + } sm.becomeCandidate() sm.becomeLeader() sm.readMessages() @@ -902,7 +946,7 @@ func TestBcastBeat(t *testing.T) { Term: 1, Nodes: []uint64{1, 2, 3}, } - sm := newRaft(1, []uint64{1, 2, 3}, 10, 1) + sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) sm.Term = 1 sm.restore(s) @@ -952,8 +996,8 @@ func TestRecvMsgBeat(t *testing.T) { } for i, tt := range tests { - sm := newRaft(1, []uint64{1, 2, 3}, 10, 1) - sm.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}} + sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}} sm.Term = 1 sm.state = tt.state switch tt.state { @@ -985,7 +1029,7 @@ func TestRestore(t *testing.T) { Nodes: []uint64{1, 2, 3}, } - sm := newRaft(1, []uint64{1, 2}, 10, 1) + sm := newRaft(1, []uint64{1, 2}, 10, 1, nil) if ok := sm.restore(s); !ok { t.Fatal("restore fail, want succeed") } @@ -1016,7 +1060,7 @@ func TestProvideSnap(t *testing.T) { Term: 11, // magic number Nodes: []uint64{1, 2}, } - sm := newRaft(1, []uint64{1}, 10, 1) + sm := newRaft(1, []uint64{1}, 10, 1, nil) // restore the statemachin from a snapshot // so it has a compacted log and a snapshot sm.restore(s) @@ -1026,7 +1070,7 @@ func TestProvideSnap(t *testing.T) { // force set the next of node 1, so that // node 1 needs a snapshot - sm.prs[2].next = sm.raftLog.offset + sm.prs[2].next = sm.raftLog.firstIndex() sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs[2].next - 1, Reject: true}) msgs := sm.readMessages() @@ -1047,7 +1091,7 @@ func TestRestoreFromSnapMsg(t *testing.T) { } m := pb.Message{Type: pb.MsgSnap, From: 1, Term: 2, Snapshot: s} - sm := newRaft(2, []uint64{1, 2}, 10, 1) + sm := newRaft(2, []uint64{1, 2}, 10, 1, nil) sm.Step(m) if !reflect.DeepEqual(sm.raftLog.snapshot, s) { @@ -1086,7 +1130,7 @@ func TestSlowNodeRestore(t *testing.T) { // it appends the entry to log and sets pendingConf to be true. func TestStepConfig(t *testing.T) { // a raft that cannot make progress - r := newRaft(1, []uint64{1, 2}, 10, 1) + r := newRaft(1, []uint64{1, 2}, 10, 1, nil) r.becomeCandidate() r.becomeLeader() index := r.raftLog.lastIndex() @@ -1104,7 +1148,7 @@ func TestStepConfig(t *testing.T) { // the proposal and keep its original state. func TestStepIgnoreConfig(t *testing.T) { // a raft that cannot make progress - r := newRaft(1, []uint64{1, 2}, 10, 1) + r := newRaft(1, []uint64{1, 2}, 10, 1, nil) r.becomeCandidate() r.becomeLeader() r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange}}}) @@ -1130,7 +1174,7 @@ func TestRecoverPendingConfig(t *testing.T) { {pb.EntryConfChange, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2}, 10, 1) + r := newRaft(1, []uint64{1, 2}, 10, 1, nil) r.appendEntry(pb.Entry{Type: tt.entType}) r.becomeCandidate() r.becomeLeader() @@ -1149,7 +1193,7 @@ func TestRecoverDoublePendingConfig(t *testing.T) { t.Errorf("expect panic, but nothing happens") } }() - r := newRaft(1, []uint64{1, 2}, 10, 1) + r := newRaft(1, []uint64{1, 2}, 10, 1, nil) r.appendEntry(pb.Entry{Type: pb.EntryConfChange}) r.appendEntry(pb.Entry{Type: pb.EntryConfChange}) r.becomeCandidate() @@ -1159,7 +1203,7 @@ func TestRecoverDoublePendingConfig(t *testing.T) { // TestAddNode tests that addNode could update pendingConf and nodes correctly. func TestAddNode(t *testing.T) { - r := newRaft(1, []uint64{1}, 10, 1) + r := newRaft(1, []uint64{1}, 10, 1, nil) r.pendingConf = true r.addNode(2) if r.pendingConf != false { @@ -1176,7 +1220,7 @@ func TestAddNode(t *testing.T) { // TestRemoveNode tests that removeNode could update pendingConf, nodes and // and removed list correctly. func TestRemoveNode(t *testing.T) { - r := newRaft(1, []uint64{1, 2}, 10, 1) + r := newRaft(1, []uint64{1, 2}, 10, 1, nil) r.pendingConf = true r.removeNode(2) if r.pendingConf != false { @@ -1216,7 +1260,12 @@ func ents(terms ...uint64) *raft { ents = append(ents, pb.Entry{Term: term}) } - sm := &raft{raftLog: &raftLog{ents: ents}} + sm := &raft{ + raftLog: &raftLog{ + storage: &MemoryStorage{ents: ents}, + unstable: uint64(len(ents)), + }, + } sm.reset(0) return sm } @@ -1241,7 +1290,7 @@ func newNetwork(peers ...Interface) *network { id := peerAddrs[i] switch v := p.(type) { case nil: - sm := newRaft(id, peerAddrs, 10, 1) + sm := newRaft(id, peerAddrs, 10, 1, nil) npeers[id] = sm case *raft: v.id = id diff --git a/raft/storage.go b/raft/storage.go new file mode 100644 index 000000000..3cb388447 --- /dev/null +++ b/raft/storage.go @@ -0,0 +1,108 @@ +/* + Copyright 2014 CoreOS, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +package raft + +import ( + "sync" + + pb "github.com/coreos/etcd/raft/raftpb" +) + +// Storage is an interface that may be implemented by the application +// to retrieve log entries from storage. If no storage implementation +// is supplied by the application, a MemoryStorage will be used, which +// retains all log entries in memory. +// +// If any Storage method returns an error, the raft instance will +// become inoperable and refuse to participate in elections; the +// application is responsible for cleanup and recovery in this case. +type Storage interface { + // GetEntries returns a slice of log entries in the range [lo,hi). + GetEntries(lo, hi uint64) ([]pb.Entry, error) + // GetLastIndex returns the index of the last entry in the log. + GetLastIndex() (uint64, error) + // GetFirstIndex returns the index of the first log entry that is + // available via GetEntries (older entries have been incorporated + // into the latest Snapshot). + GetFirstIndex() (uint64, error) + // Compact discards all log entries prior to i, creating a snapshot + // which can be used to reconstruct the state at that point. + Compact(i uint64) error +} + +// MemoryStorage implements the Storage interface backed by an +// in-memory array. +type MemoryStorage struct { + // Protects access to all fields. Most methods of MemoryStorage are + // run on the raft goroutine, but Append() is run on an application + // goroutine. + sync.Mutex + + 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. +func NewMemoryStorage() *MemoryStorage { + return &MemoryStorage{ + // Populate the list with a dummy entry at term zero. + ents: make([]pb.Entry, 1), + } +} + +// GetEntries implements the Storage interface. +func (ms *MemoryStorage) GetEntries(lo, hi uint64) ([]pb.Entry, error) { + ms.Lock() + defer ms.Unlock() + return ms.ents[lo-ms.offset : hi-ms.offset], nil +} + +// GetLastIndex implements the Storage interface. +func (ms *MemoryStorage) GetLastIndex() (uint64, error) { + ms.Lock() + defer ms.Unlock() + return ms.offset + uint64(len(ms.ents)) - 1, nil +} + +// GetFirstIndex implements the Storage interface. +func (ms *MemoryStorage) GetFirstIndex() (uint64, error) { + ms.Lock() + defer ms.Unlock() + return ms.offset, nil +} + +// Compact implements the Storage interface. +func (ms *MemoryStorage) Compact(i uint64) error { + ms.Lock() + defer ms.Unlock() + i -= ms.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 + return nil +} + +// Append the new entries to storage. +func (ms *MemoryStorage) Append(entries []pb.Entry) { + ms.Lock() + defer ms.Unlock() + ms.ents = append(ms.ents, entries...) +} diff --git a/raft/util.go b/raft/util.go new file mode 100644 index 000000000..d97707796 --- /dev/null +++ b/raft/util.go @@ -0,0 +1,54 @@ +/* + Copyright 2014 CoreOS, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +package raft + +import ( + "bytes" + "fmt" + + pb "github.com/coreos/etcd/raft/raftpb" +) + +// DescribeMessage returns a concise human-readable description of a +// Message for debugging. +func DescribeMessage(m pb.Message) string { + var buf bytes.Buffer + fmt.Fprintf(&buf, "%d->%d %s Term:%d Log:%d/%d", m.From, m.To, m.Type, m.Term, m.LogTerm, m.Index) + if m.Reject { + fmt.Fprintf(&buf, " Rejected") + } + if m.Commit != 0 { + fmt.Fprintf(&buf, " Commit:%d", m.Commit) + } + if len(m.Entries) > 0 { + fmt.Fprintf(&buf, " Entries:[") + for _, e := range m.Entries { + buf.WriteString(DescribeEntry(e)) + } + fmt.Fprintf(&buf, "]") + } + if !IsEmptySnap(m.Snapshot) { + fmt.Fprintf(&buf, " Snapshot:%v", m.Snapshot) + } + return buf.String() +} + +// DescribeEntry returns a concise human-readable description of an +// Entry for debugging. +func DescribeEntry(e pb.Entry) string { + return fmt.Sprintf("%d/%d %s %q", e.Term, e.Index, e.Type, string(e.Data)) +} diff --git a/test b/test index 9b99c2a68..893284f58 100755 --- a/test +++ b/test @@ -39,7 +39,7 @@ split=(${TEST// / }) TEST=${split[@]/#/${REPO_PATH}/} echo "Running tests..." -go test ${COVER} $@ ${TEST} --race +go test -timeout 10s ${COVER} $@ ${TEST} --race echo "Checking gofmt..." fmtRes=$(gofmt -l $FMT) From 76a3de9a338168267f8160558f3a5341d7411bb3 Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Wed, 12 Nov 2014 16:23:42 -0500 Subject: [PATCH 02/52] Require a non-nil Storage parameter in newLog. Callers must in general have a reference to their Storage objects to transfer entries from Ready to Storage, so it doesn't make sense to create a hidden Storage for them. By explicitly creating Storage objects in tests we can remove a few casts of raftLog's storage field. --- raft/log.go | 2 +- raft/log_test.go | 14 +++++----- raft/node_test.go | 11 +++++--- raft/raft_paper_test.go | 57 ++++++++++++++++++++------------------- raft/raft_test.go | 59 ++++++++++++++++++++++------------------- raft/storage.go | 4 +-- 6 files changed, 77 insertions(+), 70 deletions(-) diff --git a/raft/log.go b/raft/log.go index a8ec0ea98..c2b950430 100644 --- a/raft/log.go +++ b/raft/log.go @@ -47,7 +47,7 @@ type raftLog struct { func newLog(storage Storage) *raftLog { if storage == nil { - storage = NewMemoryStorage() + panic("storage must not be nil") } lastIndex, err := storage.GetLastIndex() if err != nil { diff --git a/raft/log_test.go b/raft/log_test.go index d5950315a..5987cfb88 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -49,7 +49,7 @@ func TestFindConflict(t *testing.T) { } for i, tt := range tests { - raftLog := newLog(nil) + raftLog := newLog(NewMemoryStorage()) raftLog.append(raftLog.lastIndex(), previousEnts...) gconflict := raftLog.findConflict(tt.from, tt.ents) @@ -61,7 +61,7 @@ func TestFindConflict(t *testing.T) { func TestIsUpToDate(t *testing.T) { previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}} - raftLog := newLog(nil) + raftLog := newLog(NewMemoryStorage()) raftLog.append(raftLog.lastIndex(), previousEnts...) tests := []struct { lastIndex uint64 @@ -241,7 +241,7 @@ func TestLogMaybeAppend(t *testing.T) { } for i, tt := range tests { - raftLog := newLog(nil) + raftLog := newLog(NewMemoryStorage()) raftLog.append(raftLog.lastIndex(), previousEnts...) raftLog.committed = commit func() { @@ -410,7 +410,7 @@ func TestCompaction(t *testing.T) { func TestLogRestore(t *testing.T) { var i uint64 - raftLog := newLog(nil) + raftLog := newLog(NewMemoryStorage()) for i = 0; i < 100; i++ { raftLog.append(i, pb.Entry{Term: i + 1}) } @@ -443,7 +443,7 @@ func TestLogRestore(t *testing.T) { func TestIsOutOfBounds(t *testing.T) { offset := uint64(100) num := uint64(100) - l := newLog(nil) + l := newLog(NewMemoryStorage()) l.restore(pb.Snapshot{Index: offset}) l.append(offset, make([]pb.Entry, num)...) @@ -471,7 +471,7 @@ func TestAt(t *testing.T) { offset := uint64(100) num := uint64(100) - l := newLog(nil) + l := newLog(NewMemoryStorage()) l.restore(pb.Snapshot{Index: offset}) for i = 0; i < num; i++ { l.append(offset+i-1, pb.Entry{Term: i}) @@ -501,7 +501,7 @@ func TestSlice(t *testing.T) { offset := uint64(100) num := uint64(100) - l := newLog(nil) + l := newLog(NewMemoryStorage()) l.restore(pb.Snapshot{Index: offset}) for i = 0; i < num; i++ { l.append(offset+i-1, pb.Entry{Term: i}) diff --git a/raft/node_test.go b/raft/node_test.go index d6be8ed29..1d10f2d1c 100644 --- a/raft/node_test.go +++ b/raft/node_test.go @@ -112,7 +112,7 @@ func TestNodeStepUnblock(t *testing.T) { // who is the current leader. func TestBlockProposal(t *testing.T) { n := newNode() - r := newRaft(1, []uint64{1}, 10, 1, nil) + r := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) go n.run(r) defer n.Stop() @@ -230,7 +230,7 @@ func TestNodeRestart(t *testing.T) { CommittedEntries: entries[1 : st.Commit+1], } - n := RestartNode(1, 10, 1, nil, st, entries, nil) + n := RestartNode(1, 10, 1, nil, st, entries, NewMemoryStorage()) if g := <-n.Ready(); !reflect.DeepEqual(g, want) { t.Errorf("g = %+v,\n w %+v", g, want) } else { @@ -303,16 +303,19 @@ func TestNodeAdvance(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - n := StartNode(1, []Peer{{ID: 1}}, 10, 1, nil) + storage := NewMemoryStorage() + n := StartNode(1, []Peer{{ID: 1}}, 10, 1, storage) n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1}) n.Campaign(ctx) <-n.Ready() n.Propose(ctx, []byte("foo")) + var rd Ready select { - case rd := <-n.Ready(): + case rd = <-n.Ready(): t.Fatalf("unexpected Ready before Advance: %+v", rd) default: } + storage.Append(rd.Entries) n.Advance() select { case <-n.Ready(): diff --git a/raft/raft_paper_test.go b/raft/raft_paper_test.go index c03a88cab..efbdac192 100644 --- a/raft/raft_paper_test.go +++ b/raft/raft_paper_test.go @@ -52,7 +52,7 @@ func TestLeaderUpdateTermFromMessage(t *testing.T) { // it immediately reverts to follower state. // Reference: section 5.1 func testUpdateTermFromMessage(t *testing.T, state StateType) { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) switch state { case StateFollower: r.becomeFollower(1, 2) @@ -82,7 +82,7 @@ func TestRejectStaleTermMessage(t *testing.T) { fakeStep := func(r *raft, m pb.Message) { called = true } - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) r.step = fakeStep r.loadState(pb.HardState{Term: 2}) @@ -96,7 +96,7 @@ func TestRejectStaleTermMessage(t *testing.T) { // TestStartAsFollower tests that when servers start up, they begin as followers. // Reference: section 5.2 func TestStartAsFollower(t *testing.T) { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) if r.state != StateFollower { t.Errorf("state = %s, want %s", r.state, StateFollower) } @@ -109,7 +109,7 @@ func TestStartAsFollower(t *testing.T) { func TestLeaderBcastBeat(t *testing.T) { // heartbeat interval hi := 1 - r := newRaft(1, []uint64{1, 2, 3}, 10, hi, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, hi, NewMemoryStorage()) r.becomeCandidate() r.becomeLeader() for i := 0; i < 10; i++ { @@ -151,7 +151,7 @@ func TestCandidateStartNewElection(t *testing.T) { func testNonleaderStartElection(t *testing.T, state StateType) { // election timeout et := 10 - r := newRaft(1, []uint64{1, 2, 3}, et, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, et, 1, NewMemoryStorage()) switch state { case StateFollower: r.becomeFollower(1, 2) @@ -215,7 +215,7 @@ func TestLeaderElectionInOneRoundRPC(t *testing.T) { {5, map[uint64]bool{}, StateCandidate}, } for i, tt := range tests { - r := newRaft(1, idsBySize(tt.size), 10, 1, nil) + r := newRaft(1, idsBySize(tt.size), 10, 1, NewMemoryStorage()) r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) for id, vote := range tt.votes { @@ -248,7 +248,7 @@ func TestFollowerVote(t *testing.T) { {2, 1, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) r.loadState(pb.HardState{Term: 1, Vote: tt.vote}) r.Step(pb.Message{From: tt.nvote, To: 1, Term: 1, Type: pb.MsgVote}) @@ -274,7 +274,7 @@ func TestCandidateFallback(t *testing.T) { {From: 2, To: 1, Term: 2, Type: pb.MsgApp}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) if r.state != StateCandidate { t.Fatalf("unexpected state = %s, want %s", r.state, StateCandidate) @@ -303,7 +303,7 @@ func TestCandidateElectionTimeoutRandomized(t *testing.T) { // Reference: section 5.2 func testNonleaderElectionTimeoutRandomized(t *testing.T, state StateType) { et := 10 - r := newRaft(1, []uint64{1, 2, 3}, et, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, et, 1, NewMemoryStorage()) timeouts := make(map[int]bool) for round := 0; round < 50*et; round++ { switch state { @@ -345,7 +345,7 @@ func testNonleadersElectionTimeoutNonconflict(t *testing.T, state StateType) { rs := make([]*raft, size) ids := idsBySize(size) for k := range rs { - rs[k] = newRaft(ids[k], ids, et, 1, nil) + rs[k] = newRaft(ids[k], ids, et, 1, NewMemoryStorage()) } conflicts := 0 for round := 0; round < 1000; round++ { @@ -387,10 +387,11 @@ func testNonleadersElectionTimeoutNonconflict(t *testing.T, state StateType) { // Also, it writes the new entry into stable storage. // Reference: section 5.3 func TestLeaderStartReplication(t *testing.T) { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + s := NewMemoryStorage() + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, s) r.becomeCandidate() r.becomeLeader() - commitNoopEntry(r) + commitNoopEntry(r, s) li := r.raftLog.lastIndex() ents := []pb.Entry{{Data: []byte("some data")}} @@ -425,10 +426,11 @@ func TestLeaderStartReplication(t *testing.T) { // servers eventually find out. // Reference: section 5.3 func TestLeaderCommitEntry(t *testing.T) { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + s := NewMemoryStorage() + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, s) r.becomeCandidate() r.becomeLeader() - commitNoopEntry(r) + commitNoopEntry(r, s) li := r.raftLog.lastIndex() r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) @@ -478,10 +480,11 @@ func TestLeaderAcknowledgeCommit(t *testing.T) { {5, map[uint64]bool{2: true, 3: true, 4: true, 5: true}, true}, } for i, tt := range tests { - r := newRaft(1, idsBySize(tt.size), 10, 1, nil) + s := NewMemoryStorage() + r := newRaft(1, idsBySize(tt.size), 10, 1, s) r.becomeCandidate() r.becomeLeader() - commitNoopEntry(r) + commitNoopEntry(r, s) li := r.raftLog.lastIndex() r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) @@ -510,7 +513,7 @@ func TestLeaderCommitPrecedingEntries(t *testing.T) { {{Term: 1, Index: 1}}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) r.loadEnts(append([]pb.Entry{{}}, tt...)) r.loadState(pb.HardState{Term: 2}) r.becomeCandidate() @@ -566,7 +569,7 @@ func TestFollowerCommitEntry(t *testing.T) { }, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) r.becomeFollower(1, 2) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 1, Entries: tt.ents, Commit: tt.commit}) @@ -601,7 +604,7 @@ func TestFollowerCheckMsgApp(t *testing.T) { {3, 3, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) r.loadEnts(ents) r.loadState(pb.HardState{Commit: 2}) r.becomeFollower(2, 2) @@ -656,7 +659,7 @@ func TestFollowerAppendEntries(t *testing.T) { }, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) r.loadEnts([]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}}) r.becomeFollower(2, 2) @@ -724,10 +727,10 @@ func TestLeaderSyncFollowerLog(t *testing.T) { }, } for i, tt := range tests { - lead := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + lead := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) lead.loadEnts(ents) lead.loadState(pb.HardState{Commit: lead.raftLog.lastIndex(), Term: term}) - follower := newRaft(2, []uint64{1, 2, 3}, 10, 1, nil) + follower := newRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) follower.loadEnts(tt) follower.loadState(pb.HardState{Term: term - 1}) // It is necessary to have a three-node cluster. @@ -757,7 +760,7 @@ func TestVoteRequest(t *testing.T) { {[]pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}, 3}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) r.Step(pb.Message{ From: 2, To: 1, Type: pb.MsgApp, Term: tt.wterm - 1, LogTerm: 0, Index: 0, Entries: tt.ents, }) @@ -818,7 +821,7 @@ func TestVoter(t *testing.T) { {[]pb.Entry{{}, {Term: 2, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) r.loadEnts(tt.ents) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgVote, Term: 3, LogTerm: tt.logterm, Index: tt.index}) @@ -853,7 +856,7 @@ func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) { {3, 3}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) r.loadEnts(ents) r.loadState(pb.HardState{Term: 2}) // become leader at term 3 @@ -876,7 +879,7 @@ func (s messageSlice) Len() int { return len(s) } func (s messageSlice) Less(i, j int) bool { return fmt.Sprint(s[i]) < fmt.Sprint(s[j]) } func (s messageSlice) Swap(i, j int) { s[i], s[j] = s[j], s[i] } -func commitNoopEntry(r *raft) { +func commitNoopEntry(r *raft, s *MemoryStorage) { if r.state != StateLeader { panic("it should only be used when it is the leader") } @@ -891,7 +894,7 @@ func commitNoopEntry(r *raft) { } // ignore further messages to refresh followers' commmit index r.readMessages() - r.raftLog.storage.(*MemoryStorage).Append(r.raftLog.unstableEntries()) + s.Append(r.raftLog.unstableEntries()) r.raftLog.appliedTo(r.raftLog.committed) r.raftLog.stableTo(r.raftLog.lastIndex()) } diff --git a/raft/raft_test.go b/raft/raft_test.go index 5544fda35..5f899e154 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -29,10 +29,9 @@ import ( ) // nextEnts returns the appliable entries and updates the applied index -func nextEnts(r *raft) (ents []pb.Entry) { +func nextEnts(r *raft, s *MemoryStorage) (ents []pb.Entry) { // Transfer all unstable entries to "stable" storage. - memStorage := r.raftLog.storage.(*MemoryStorage) - memStorage.Append(r.raftLog.unstableEntries()) + s.Append(r.raftLog.unstableEntries()) r.raftLog.stableTo(r.raftLog.lastIndex()) ents = r.raftLog.nextEnts() @@ -176,7 +175,7 @@ func TestLogReplication(t *testing.T) { } ents := []pb.Entry{} - for _, e := range nextEnts(sm) { + for _, e := range nextEnts(sm, tt.network.storage[j]) { if e.Data != nil { ents = append(ents, e) } @@ -285,9 +284,9 @@ func TestCommitWithoutNewTermEntry(t *testing.T) { } func TestDuelingCandidates(t *testing.T) { - a := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) - b := newRaft(2, []uint64{1, 2, 3}, 10, 1, nil) - c := newRaft(3, []uint64{1, 2, 3}, 10, 1, nil) + a := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) + b := newRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) + c := newRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) nt := newNetwork(a, b, c) nt.cut(1, 3) @@ -311,7 +310,7 @@ func TestDuelingCandidates(t *testing.T) { }{ {a, StateFollower, 2, wlog}, {b, StateFollower, 2, wlog}, - {c, StateFollower, 2, newLog(nil)}, + {c, StateFollower, 2, newLog(NewMemoryStorage())}, } for i, tt := range tests { @@ -450,7 +449,7 @@ func TestProposal(t *testing.T) { send(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}}) - wantLog := newLog(nil) + wantLog := newLog(NewMemoryStorage()) if tt.success { wantLog = &raftLog{ storage: &MemoryStorage{ @@ -620,7 +619,7 @@ func TestIsElectionTimeout(t *testing.T) { } for i, tt := range tests { - sm := newRaft(1, []uint64{1}, 10, 1, nil) + sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) sm.elapsed = tt.elapse c := 0 for j := 0; j < 10000; j++ { @@ -645,7 +644,7 @@ func TestStepIgnoreOldTermMsg(t *testing.T) { fakeStep := func(r *raft, m pb.Message) { called = true } - sm := newRaft(1, []uint64{1}, 10, 1, nil) + sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) sm.step = fakeStep sm.Term = 2 sm.Step(pb.Message{Type: pb.MsgApp, Term: sm.Term - 1}) @@ -747,7 +746,7 @@ func TestRecvMsgVote(t *testing.T) { } for i, tt := range tests { - sm := newRaft(1, []uint64{1}, 10, 1, nil) + sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) sm.state = tt.state switch tt.state { case StateFollower: @@ -807,7 +806,7 @@ func TestStateTransition(t *testing.T) { } }() - sm := newRaft(1, []uint64{1}, 10, 1, nil) + sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) sm.state = tt.from switch tt.to { @@ -846,7 +845,7 @@ func TestAllServerStepdown(t *testing.T) { tterm := uint64(3) for i, tt := range tests { - sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) switch tt.state { case StateFollower: sm.becomeFollower(1, None) @@ -902,7 +901,7 @@ func TestLeaderAppResp(t *testing.T) { for i, tt := range tests { // sm term is 1 after it becomes the leader. // thus the last log term must be 1 to be committed. - sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) sm.raftLog = &raftLog{ storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}, unstable: 3, @@ -946,7 +945,7 @@ func TestBcastBeat(t *testing.T) { Term: 1, Nodes: []uint64{1, 2, 3}, } - sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) sm.Term = 1 sm.restore(s) @@ -996,7 +995,7 @@ func TestRecvMsgBeat(t *testing.T) { } for i, tt := range tests { - sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil) + sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}} sm.Term = 1 sm.state = tt.state @@ -1029,7 +1028,7 @@ func TestRestore(t *testing.T) { Nodes: []uint64{1, 2, 3}, } - sm := newRaft(1, []uint64{1, 2}, 10, 1, nil) + sm := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) if ok := sm.restore(s); !ok { t.Fatal("restore fail, want succeed") } @@ -1060,7 +1059,7 @@ func TestProvideSnap(t *testing.T) { Term: 11, // magic number Nodes: []uint64{1, 2}, } - sm := newRaft(1, []uint64{1}, 10, 1, nil) + sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) // restore the statemachin from a snapshot // so it has a compacted log and a snapshot sm.restore(s) @@ -1091,7 +1090,7 @@ func TestRestoreFromSnapMsg(t *testing.T) { } m := pb.Message{Type: pb.MsgSnap, From: 1, Term: 2, Snapshot: s} - sm := newRaft(2, []uint64{1, 2}, 10, 1, nil) + sm := newRaft(2, []uint64{1, 2}, 10, 1, NewMemoryStorage()) sm.Step(m) if !reflect.DeepEqual(sm.raftLog.snapshot, s) { @@ -1108,7 +1107,7 @@ func TestSlowNodeRestore(t *testing.T) { nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}}) } lead := nt.peers[1].(*raft) - nextEnts(lead) + nextEnts(lead, nt.storage[1]) lead.compact(lead.raftLog.applied, lead.nodes(), nil) nt.recover() @@ -1130,7 +1129,7 @@ func TestSlowNodeRestore(t *testing.T) { // it appends the entry to log and sets pendingConf to be true. func TestStepConfig(t *testing.T) { // a raft that cannot make progress - r := newRaft(1, []uint64{1, 2}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) r.becomeCandidate() r.becomeLeader() index := r.raftLog.lastIndex() @@ -1148,7 +1147,7 @@ func TestStepConfig(t *testing.T) { // the proposal and keep its original state. func TestStepIgnoreConfig(t *testing.T) { // a raft that cannot make progress - r := newRaft(1, []uint64{1, 2}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) r.becomeCandidate() r.becomeLeader() r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange}}}) @@ -1174,7 +1173,7 @@ func TestRecoverPendingConfig(t *testing.T) { {pb.EntryConfChange, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) r.appendEntry(pb.Entry{Type: tt.entType}) r.becomeCandidate() r.becomeLeader() @@ -1193,7 +1192,7 @@ func TestRecoverDoublePendingConfig(t *testing.T) { t.Errorf("expect panic, but nothing happens") } }() - r := newRaft(1, []uint64{1, 2}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) r.appendEntry(pb.Entry{Type: pb.EntryConfChange}) r.appendEntry(pb.Entry{Type: pb.EntryConfChange}) r.becomeCandidate() @@ -1203,7 +1202,7 @@ func TestRecoverDoublePendingConfig(t *testing.T) { // TestAddNode tests that addNode could update pendingConf and nodes correctly. func TestAddNode(t *testing.T) { - r := newRaft(1, []uint64{1}, 10, 1, nil) + r := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) r.pendingConf = true r.addNode(2) if r.pendingConf != false { @@ -1220,7 +1219,7 @@ func TestAddNode(t *testing.T) { // TestRemoveNode tests that removeNode could update pendingConf, nodes and // and removed list correctly. func TestRemoveNode(t *testing.T) { - r := newRaft(1, []uint64{1, 2}, 10, 1, nil) + r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) r.pendingConf = true r.removeNode(2) if r.pendingConf != false { @@ -1272,6 +1271,7 @@ func ents(terms ...uint64) *raft { type network struct { peers map[uint64]Interface + storage map[uint64]*MemoryStorage dropm map[connem]float64 ignorem map[pb.MessageType]bool } @@ -1285,12 +1285,14 @@ func newNetwork(peers ...Interface) *network { peerAddrs := idsBySize(size) npeers := make(map[uint64]Interface, size) + nstorage := make(map[uint64]*MemoryStorage, size) for i, p := range peers { id := peerAddrs[i] switch v := p.(type) { case nil: - sm := newRaft(id, peerAddrs, 10, 1, nil) + nstorage[id] = NewMemoryStorage() + sm := newRaft(id, peerAddrs, 10, 1, nstorage[id]) npeers[id] = sm case *raft: v.id = id @@ -1308,6 +1310,7 @@ func newNetwork(peers ...Interface) *network { } return &network{ peers: npeers, + storage: nstorage, dropm: make(map[connem]float64), ignorem: make(map[pb.MessageType]bool), } diff --git a/raft/storage.go b/raft/storage.go index 3cb388447..d559d95ac 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -23,9 +23,7 @@ import ( ) // Storage is an interface that may be implemented by the application -// to retrieve log entries from storage. If no storage implementation -// is supplied by the application, a MemoryStorage will be used, which -// retains all log entries in memory. +// to retrieve log entries from storage. // // If any Storage method returns an error, the raft instance will // become inoperable and refuse to participate in elections; the From 147fd614ced3e496c331524cb461ec97f52eb5ad Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Wed, 12 Nov 2014 18:21:30 -0500 Subject: [PATCH 03/52] The initial term=0 log entry is now initially unstable. This entry is now persisted through the normal flow instead of appearing in the stored log at creation time. This is how things worked before the Storage interface was introduced. (see coreos/etcd#1689) --- etcdserver/server_test.go | 6 ++++++ raft/log.go | 16 +++++++++++----- raft/log_test.go | 16 ++++++++-------- raft/node_test.go | 7 +++++-- raft/raft_paper_test.go | 10 ++++++---- raft/storage.go | 14 ++++++++++---- 6 files changed, 46 insertions(+), 23 deletions(-) diff --git a/etcdserver/server_test.go b/etcdserver/server_test.go index 8b505a056..772ba2079 100644 --- a/etcdserver/server_test.go +++ b/etcdserver/server_test.go @@ -838,6 +838,12 @@ func TestSnapshot(t *testing.T) { s := raft.NewMemoryStorage() n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s) defer n.Stop() + + // Save the initial state to storage so we have something to snapshot. + rd := <-n.Ready() + s.Append(rd.Entries) + n.Advance() + st := &storeRecorder{} p := &storageRecorder{} srv := &EtcdServer{ diff --git a/raft/log.go b/raft/log.go index c2b950430..c1dcdbbc2 100644 --- a/raft/log.go +++ b/raft/log.go @@ -49,14 +49,19 @@ func newLog(storage Storage) *raftLog { if storage == nil { panic("storage must not be nil") } + log := &raftLog{ + storage: storage, + } lastIndex, err := storage.GetLastIndex() - if err != nil { + if err == ErrStorageEmpty { + // When starting from scratch populate the list with a dummy entry at term zero. + log.unstableEnts = make([]pb.Entry, 1) + } else if err == nil { + log.unstable = lastIndex + 1 + } else { panic(err) // TODO(bdarnell) } - return &raftLog{ - storage: storage, - unstable: lastIndex + 1, - } + return log } func (l *raftLog) load(ents []pb.Entry) { @@ -67,6 +72,7 @@ func (l *raftLog) load(ents []pb.Entry) { } ms.ents = ents l.unstable = ms.offset + uint64(len(ents)) + l.unstableEnts = nil } func (l *raftLog) String() string { diff --git a/raft/log_test.go b/raft/log_test.go index 5987cfb88..4965c8054 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -91,7 +91,7 @@ func TestIsUpToDate(t *testing.T) { } func TestAppend(t *testing.T) { - previousEnts := []pb.Entry{{Term: 1}, {Term: 2}} + previousEnts := []pb.Entry{{}, {Term: 1}, {Term: 2}} tests := []struct { after uint64 ents []pb.Entry @@ -283,7 +283,7 @@ func TestCompactionSideEffects(t *testing.T) { unstableIndex := uint64(750) lastTerm := lastIndex storage := NewMemoryStorage() - for i = 1; i <= unstableIndex; i++ { + for i = 0; i <= unstableIndex; i++ { storage.Append([]pb.Entry{{Term: uint64(i), Index: uint64(i)}}) } raftLog := newLog(storage) @@ -337,21 +337,21 @@ func TestCompactionSideEffects(t *testing.T) { } func TestUnstableEnts(t *testing.T) { - previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}} + previousEnts := []pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}} tests := []struct { unstable uint64 wents []pb.Entry wunstable uint64 }{ {3, nil, 3}, - {1, previousEnts, 3}, + {1, previousEnts[1:], 3}, } for i, tt := range tests { storage := NewMemoryStorage() - storage.Append(previousEnts[:tt.unstable-1]) + storage.Append(previousEnts[:tt.unstable]) raftLog := newLog(storage) - raftLog.append(raftLog.lastIndex(), previousEnts[tt.unstable-1:]...) + raftLog.append(raftLog.lastIndex(), previousEnts[tt.unstable:]...) ents := raftLog.unstableEntries() raftLog.stableTo(raftLog.lastIndex()) if !reflect.DeepEqual(ents, tt.wents) { @@ -363,7 +363,7 @@ func TestUnstableEnts(t *testing.T) { } } -//TestCompaction ensures that the number of log entreis is correct after compactions. +//TestCompaction ensures that the number of log entries is correct after compactions. func TestCompaction(t *testing.T) { tests := []struct { applied uint64 @@ -391,7 +391,7 @@ func TestCompaction(t *testing.T) { }() storage := NewMemoryStorage() - for i := uint64(0); i < tt.lastIndex; i++ { + for i := uint64(0); i <= tt.lastIndex; i++ { storage.Append([]pb.Entry{{}}) } raftLog := newLog(storage) diff --git a/raft/node_test.go b/raft/node_test.go index 1d10f2d1c..ea6827716 100644 --- a/raft/node_test.go +++ b/raft/node_test.go @@ -175,6 +175,7 @@ func TestNode(t *testing.T) { SoftState: &SoftState{Lead: 1, Nodes: []uint64{1}, RaftState: StateLeader}, HardState: raftpb.HardState{Term: 1, Commit: 2}, Entries: []raftpb.Entry{ + {}, {Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata}, {Term: 1, Index: 2}, }, @@ -195,7 +196,7 @@ func TestNode(t *testing.T) { n.Campaign(ctx) g := <-n.Ready() if !reflect.DeepEqual(g, wants[0]) { - t.Errorf("#%d: g = %+v,\n w %+v", 1, g, wants[0]) + t.Fatalf("#%d: g = %+v,\n w %+v", 1, g, wants[0]) } else { storage.Append(g.Entries) n.Advance() @@ -230,7 +231,9 @@ func TestNodeRestart(t *testing.T) { CommittedEntries: entries[1 : st.Commit+1], } - n := RestartNode(1, 10, 1, nil, st, entries, NewMemoryStorage()) + storage := NewMemoryStorage() + storage.Append(entries) + n := RestartNode(1, 10, 1, nil, st, nil, storage) if g := <-n.Ready(); !reflect.DeepEqual(g, want) { t.Errorf("g = %+v,\n w %+v", g, want) } else { diff --git a/raft/raft_paper_test.go b/raft/raft_paper_test.go index efbdac192..3a54d1584 100644 --- a/raft/raft_paper_test.go +++ b/raft/raft_paper_test.go @@ -513,8 +513,9 @@ func TestLeaderCommitPrecedingEntries(t *testing.T) { {{Term: 1, Index: 1}}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - r.loadEnts(append([]pb.Entry{{}}, tt...)) + storage := NewMemoryStorage() + storage.Append(append([]pb.Entry{{}}, tt...)) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage) r.loadState(pb.HardState{Term: 2}) r.becomeCandidate() r.becomeLeader() @@ -659,8 +660,9 @@ func TestFollowerAppendEntries(t *testing.T) { }, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - r.loadEnts([]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}}) + storage := NewMemoryStorage() + storage.Append([]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}}) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage) r.becomeFollower(2, 2) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 2, LogTerm: tt.term, Index: tt.index, Entries: tt.ents}) diff --git a/raft/storage.go b/raft/storage.go index d559d95ac..7733b2ba0 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -17,11 +17,16 @@ package raft import ( + "errors" "sync" pb "github.com/coreos/etcd/raft/raftpb" ) +// ErrStorageEmpty is returned by Storage.GetLastIndex when there is +// no data. +var ErrStorageEmpty = errors.New("storage is empty") + // Storage is an interface that may be implemented by the application // to retrieve log entries from storage. // @@ -32,6 +37,7 @@ type Storage interface { // GetEntries returns a slice of log entries in the range [lo,hi). GetEntries(lo, hi uint64) ([]pb.Entry, error) // GetLastIndex returns the index of the last entry in the log. + // If the log is empty it returns ErrStorageEmpty. GetLastIndex() (uint64, error) // GetFirstIndex returns the index of the first log entry that is // available via GetEntries (older entries have been incorporated @@ -58,10 +64,7 @@ type MemoryStorage struct { // NewMemoryStorage creates an empty MemoryStorage. func NewMemoryStorage() *MemoryStorage { - return &MemoryStorage{ - // Populate the list with a dummy entry at term zero. - ents: make([]pb.Entry, 1), - } + return &MemoryStorage{} } // GetEntries implements the Storage interface. @@ -75,6 +78,9 @@ func (ms *MemoryStorage) GetEntries(lo, hi uint64) ([]pb.Entry, error) { func (ms *MemoryStorage) GetLastIndex() (uint64, error) { ms.Lock() defer ms.Unlock() + if len(ms.ents) == 0 { + return 0, ErrStorageEmpty + } return ms.offset + uint64(len(ms.ents)) - 1, nil } From 54b07d7974d020c0d202a8cd419a816076ddbb9a Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Wed, 12 Nov 2014 18:31:19 -0500 Subject: [PATCH 04/52] Remove raft.loadEnts and the ents parameter to raft.RestartNode. The initial entries are now provided via the Storage interface. --- etcdserver/force_cluster.go | 3 ++- etcdserver/server.go | 3 ++- raft/log.go | 11 ----------- raft/node.go | 5 +---- raft/node_test.go | 2 +- raft/raft.go | 4 ---- raft/raft_paper_test.go | 25 +++++++++++++++---------- 7 files changed, 21 insertions(+), 32 deletions(-) diff --git a/etcdserver/force_cluster.go b/etcdserver/force_cluster.go index 233648539..c409ff02c 100644 --- a/etcdserver/force_cluster.go +++ b/etcdserver/force_cluster.go @@ -54,7 +54,8 @@ 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() - n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents, s) + s.Append(ents) + n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, s) return id, n, s, w } diff --git a/etcdserver/server.go b/etcdserver/server.go index 24115944b..6bf0a4e77 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -773,7 +773,8 @@ 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() - n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents, s) + s.Append(ents) + n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, s) return id, n, s, w } diff --git a/raft/log.go b/raft/log.go index c1dcdbbc2..1203bcb28 100644 --- a/raft/log.go +++ b/raft/log.go @@ -64,17 +64,6 @@ func newLog(storage Storage) *raftLog { return log } -func (l *raftLog) load(ents []pb.Entry) { - // TODO(bdarnell): does this method need to support other Storage impls or does it go away? - ms := l.storage.(*MemoryStorage) - if ms.offset != ents[0].Index { - panic("entries loaded don't match offset index") - } - ms.ents = ents - l.unstable = ms.offset + uint64(len(ents)) - l.unstableEnts = nil -} - func (l *raftLog) String() string { return fmt.Sprintf("unstable=%d committed=%d applied=%d", l.unstable, l.committed, l.applied) diff --git a/raft/node.go b/raft/node.go index c607dd21c..411085ae1 100644 --- a/raft/node.go +++ b/raft/node.go @@ -168,7 +168,7 @@ func StartNode(id uint64, peers []Peer, election, heartbeat int, storage 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, ents []pb.Entry, storage Storage) Node { +func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, storage Storage) Node { n := newNode() r := newRaft(id, nil, election, heartbeat, storage) if snapshot != nil { @@ -177,9 +177,6 @@ func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st p if !isHardStateEqual(st, emptyState) { r.loadState(st) } - if len(ents) != 0 { - r.loadEnts(ents) - } go n.run(r) return &n } diff --git a/raft/node_test.go b/raft/node_test.go index ea6827716..9639581b5 100644 --- a/raft/node_test.go +++ b/raft/node_test.go @@ -233,7 +233,7 @@ func TestNodeRestart(t *testing.T) { storage := NewMemoryStorage() storage.Append(entries) - n := RestartNode(1, 10, 1, nil, st, nil, storage) + n := RestartNode(1, 10, 1, nil, st, storage) if g := <-n.Ready(); !reflect.DeepEqual(g, want) { t.Errorf("g = %+v,\n w %+v", g, want) } else { diff --git a/raft/raft.go b/raft/raft.go index 81214fc5a..b98b0af00 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -549,10 +549,6 @@ func (r *raft) promotable() bool { return ok } -func (r *raft) loadEnts(ents []pb.Entry) { - r.raftLog.load(ents) -} - func (r *raft) loadState(state pb.HardState) { r.raftLog.committed = state.Commit r.Term = state.Term diff --git a/raft/raft_paper_test.go b/raft/raft_paper_test.go index 3a54d1584..8433010c7 100644 --- a/raft/raft_paper_test.go +++ b/raft/raft_paper_test.go @@ -605,8 +605,9 @@ func TestFollowerCheckMsgApp(t *testing.T) { {3, 3, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - r.loadEnts(ents) + storage := NewMemoryStorage() + storage.Append(ents) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage) r.loadState(pb.HardState{Commit: 2}) r.becomeFollower(2, 2) @@ -729,11 +730,13 @@ func TestLeaderSyncFollowerLog(t *testing.T) { }, } for i, tt := range tests { - lead := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - lead.loadEnts(ents) + leadStorage := NewMemoryStorage() + leadStorage.Append(ents) + lead := newRaft(1, []uint64{1, 2, 3}, 10, 1, leadStorage) lead.loadState(pb.HardState{Commit: lead.raftLog.lastIndex(), Term: term}) - follower := newRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - follower.loadEnts(tt) + followerStorage := NewMemoryStorage() + followerStorage.Append(tt) + follower := newRaft(2, []uint64{1, 2, 3}, 10, 1, followerStorage) follower.loadState(pb.HardState{Term: term - 1}) // It is necessary to have a three-node cluster. // The second may have more up-to-date log than the first one, so the @@ -823,8 +826,9 @@ func TestVoter(t *testing.T) { {[]pb.Entry{{}, {Term: 2, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) - r.loadEnts(tt.ents) + storage := NewMemoryStorage() + storage.Append(tt.ents) + r := newRaft(1, []uint64{1, 2}, 10, 1, storage) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgVote, Term: 3, LogTerm: tt.logterm, Index: tt.index}) @@ -858,8 +862,9 @@ func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) { {3, 3}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) - r.loadEnts(ents) + storage := NewMemoryStorage() + storage.Append(ents) + r := newRaft(1, []uint64{1, 2}, 10, 1, storage) r.loadState(pb.HardState{Term: 2}) // become leader at term 3 r.becomeCandidate() From 0e8ffe9128f7d4af0af2a8a96043529004001ea1 Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Thu, 13 Nov 2014 15:51:36 -0500 Subject: [PATCH 05/52] raft: remove a guard that is no longer necessary --- raft/log.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/raft/log.go b/raft/log.go index 63edccbb1..1a16bb268 100644 --- a/raft/log.go +++ b/raft/log.go @@ -176,9 +176,7 @@ func (l *raftLog) appliedTo(i uint64) { } func (l *raftLog) stableTo(i uint64) { - if len(l.unstableEnts) > 0 { - l.unstableEnts = l.unstableEnts[i+1-l.unstable:] - } + l.unstableEnts = l.unstableEnts[i+1-l.unstable:] l.unstable = i + 1 } From 45e96be605c2b7f1d5777b85fa6fe20930c19397 Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Fri, 14 Nov 2014 13:53:42 -0500 Subject: [PATCH 06/52] raft: PR feedback. Removed Get prefix in method names, added assertions and fixed comments. --- raft/log.go | 23 ++++++++++++++--------- raft/log_test.go | 4 ++-- raft/storage.go | 25 +++++++++++++------------ 3 files changed, 29 insertions(+), 23 deletions(-) diff --git a/raft/log.go b/raft/log.go index 1a16bb268..428b0396d 100644 --- a/raft/log.go +++ b/raft/log.go @@ -52,7 +52,7 @@ func newLog(storage Storage) *raftLog { log := &raftLog{ storage: storage, } - lastIndex, err := storage.GetLastIndex() + lastIndex, err := storage.LastIndex() if err == ErrStorageEmpty { // When starting from scratch populate the list with a dummy entry at term zero. log.unstableEnts = make([]pb.Entry, 1) @@ -94,6 +94,9 @@ func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry } func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 { + if after < l.committed { + log.Panicf("appending after %d, but already committed through %d", after, l.committed) + } if after < l.unstable { // The log is being truncated to before our current unstable // portion, so discard it and reset unstable. @@ -132,9 +135,7 @@ func (l *raftLog) unstableEntries() []pb.Entry { if len(l.unstableEnts) == 0 { return nil } - cpy := make([]pb.Entry, len(l.unstableEnts)) - copy(cpy, l.unstableEnts) - return cpy + return append([]pb.Entry(nil), l.unstableEnts...) } // nextEnts returns all the available entries for execution. @@ -147,7 +148,7 @@ func (l *raftLog) nextEnts() (ents []pb.Entry) { } func (l *raftLog) firstIndex() uint64 { - index, err := l.storage.GetFirstIndex() + index, err := l.storage.FirstIndex() if err != nil { panic(err) // TODO(bdarnell) } @@ -158,7 +159,7 @@ func (l *raftLog) lastIndex() uint64 { if len(l.unstableEnts) > 0 { return l.unstable + uint64(len(l.unstableEnts)) - 1 } - index, err := l.storage.GetLastIndex() + index, err := l.storage.LastIndex() if err != nil { panic(err) // TODO(bdarnell) } @@ -176,6 +177,10 @@ func (l *raftLog) appliedTo(i uint64) { } func (l *raftLog) stableTo(i uint64) { + if i < l.unstable || i+1-l.unstable > uint64(len(l.unstableEnts)) { + log.Panicf("stableTo(%d) is out of range (unstable=%d, len(unstableEnts)=%d)", + i, l.unstable, len(l.unstableEnts)) + } l.unstableEnts = l.unstableEnts[i+1-l.unstable:] l.unstable = i + 1 } @@ -247,11 +252,11 @@ func (l *raftLog) compact(i uint64) uint64 { panic(err) // TODO(bdarnell) } l.unstable = max(i+1, l.unstable) - firstIndex, err := l.storage.GetFirstIndex() + firstIndex, err := l.storage.FirstIndex() if err != nil { panic(err) // TODO(bdarnell) } - lastIndex, err := l.storage.GetLastIndex() + lastIndex, err := l.storage.LastIndex() if err != nil { panic(err) // TODO(bdarnell) } @@ -297,7 +302,7 @@ func (l *raftLog) slice(lo uint64, hi uint64) []pb.Entry { } var ents []pb.Entry if lo < l.unstable { - storedEnts, err := l.storage.GetEntries(lo, min(hi, l.unstable)) + storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable)) if err != nil { panic(err) // TODO(bdarnell) } diff --git a/raft/log_test.go b/raft/log_test.go index 21ff06fad..27089d930 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -495,7 +495,7 @@ func TestAt(t *testing.T) { l := newLog(NewMemoryStorage()) l.restore(pb.Snapshot{Index: offset}) - for i = 0; i < num; i++ { + for i = 1; i < num; i++ { l.append(offset+i-1, pb.Entry{Term: i}) } @@ -525,7 +525,7 @@ func TestSlice(t *testing.T) { l := newLog(NewMemoryStorage()) l.restore(pb.Snapshot{Index: offset}) - for i = 0; i < num; i++ { + for i = 1; i < num; i++ { l.append(offset+i-1, pb.Entry{Term: i}) } diff --git a/raft/storage.go b/raft/storage.go index 7733b2ba0..049b9ea17 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -34,17 +34,18 @@ var ErrStorageEmpty = errors.New("storage is empty") // become inoperable and refuse to participate in elections; the // application is responsible for cleanup and recovery in this case. type Storage interface { - // GetEntries returns a slice of log entries in the range [lo,hi). - GetEntries(lo, hi uint64) ([]pb.Entry, error) + // Entries returns a slice of log entries in the range [lo,hi). + Entries(lo, hi uint64) ([]pb.Entry, error) // GetLastIndex returns the index of the last entry in the log. // If the log is empty it returns ErrStorageEmpty. - GetLastIndex() (uint64, error) + LastIndex() (uint64, error) // GetFirstIndex returns the index of the first log entry that is // available via GetEntries (older entries have been incorporated // into the latest Snapshot). - GetFirstIndex() (uint64, error) - // Compact discards all log entries prior to i, creating a snapshot - // which can be used to reconstruct the state at that point. + 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 } @@ -67,15 +68,15 @@ func NewMemoryStorage() *MemoryStorage { return &MemoryStorage{} } -// GetEntries implements the Storage interface. -func (ms *MemoryStorage) GetEntries(lo, hi uint64) ([]pb.Entry, error) { +// Entries implements the Storage interface. +func (ms *MemoryStorage) Entries(lo, hi uint64) ([]pb.Entry, error) { ms.Lock() defer ms.Unlock() return ms.ents[lo-ms.offset : hi-ms.offset], nil } -// GetLastIndex implements the Storage interface. -func (ms *MemoryStorage) GetLastIndex() (uint64, error) { +// LastIndex implements the Storage interface. +func (ms *MemoryStorage) LastIndex() (uint64, error) { ms.Lock() defer ms.Unlock() if len(ms.ents) == 0 { @@ -84,8 +85,8 @@ func (ms *MemoryStorage) GetLastIndex() (uint64, error) { return ms.offset + uint64(len(ms.ents)) - 1, nil } -// GetFirstIndex implements the Storage interface. -func (ms *MemoryStorage) GetFirstIndex() (uint64, error) { +// FirstIndex implements the Storage interface. +func (ms *MemoryStorage) FirstIndex() (uint64, error) { ms.Lock() defer ms.Unlock() return ms.offset, nil From 64d9bcabf1bed5b0edbbe67bf06b3de3a74a3ae4 Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Mon, 17 Nov 2014 16:37:46 -0500 Subject: [PATCH 07/52] Add Storage.Term() method and hide the first entry from other methods. The first entry in the log is a dummy which is used for matchTerm but may not have an actual payload. This change permits Storage implementations to treat this term value specially instead of storing it as a dummy Entry. Storage.FirstIndex() no longer includes the term-only entry. This reverses a recent decision to create entry zero as initially unstable; Storage implementations are now required to make Term(0) == 0 and the first unstable entry is now index 1. stableTo(0) is no longer allowed. --- etcdserver/server_test.go | 24 +++++++++++++-- raft/log.go | 45 +++++++++++++-------------- raft/log_test.go | 64 ++++++++++++++++++++++++++++----------- raft/node_test.go | 6 ++-- raft/raft.go | 6 ++-- raft/raft_paper_test.go | 40 ++++++++++++------------ raft/raft_test.go | 15 +++++---- raft/storage.go | 41 +++++++++++++++++-------- 8 files changed, 153 insertions(+), 88 deletions(-) diff --git a/etcdserver/server_test.go b/etcdserver/server_test.go index 6eef6e245..e5d24e65d 100644 --- a/etcdserver/server_test.go +++ b/etcdserver/server_test.go @@ -840,9 +840,29 @@ func TestSnapshot(t *testing.T) { n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s) defer n.Stop() - // Save the initial state to storage so we have something to snapshot. + // Progress the node to the point where it has something to snapshot. + // TODO(bdarnell): this could be improved with changes in the raft internals. + // First, we must apply the initial conf changes so we can have an election. rd := <-n.Ready() s.Append(rd.Entries) + for _, e := range rd.CommittedEntries { + if e.Type == raftpb.EntryConfChange { + var cc raftpb.ConfChange + err := cc.Unmarshal(e.Data) + if err != nil { + t.Fatal(err) + } + n.ApplyConfChange(cc) + } + } + n.Advance() + + // Now we can have an election and persist the rest of the log. + // This causes HardState.Commit to advance. HardState.Commit must + // be > 0 to snapshot. + n.Campaign(context.Background()) + rd = <-n.Ready() + s.Append(rd.Entries) n.Advance() st := &storeRecorder{} @@ -854,7 +874,7 @@ func TestSnapshot(t *testing.T) { raftStorage: s, } - srv.snapshot(0, []uint64{1}) + srv.snapshot(1, []uint64{1}) gaction := st.Action() if len(gaction) != 1 { t.Fatalf("len(action) = %d, want 1", len(gaction)) diff --git a/raft/log.go b/raft/log.go index 428b0396d..7e5831f0e 100644 --- a/raft/log.go +++ b/raft/log.go @@ -53,14 +53,11 @@ func newLog(storage Storage) *raftLog { storage: storage, } lastIndex, err := storage.LastIndex() - if err == ErrStorageEmpty { - // When starting from scratch populate the list with a dummy entry at term zero. - log.unstableEnts = make([]pb.Entry, 1) - } else if err == nil { - log.unstable = lastIndex + 1 - } else { + if err != nil { panic(err) // TODO(bdarnell) } + log.unstable = lastIndex + 1 + return log } @@ -190,27 +187,28 @@ func (l *raftLog) lastTerm() uint64 { } func (l *raftLog) term(i uint64) uint64 { - if e := l.at(i); e != nil { - return e.Term + if i < l.unstable { + t, err := l.storage.Term(i) + if err == ErrSnapshotRequired { + return 0 + } else if err != nil { + panic(err) // TODO(bdarnell) + } + return t } - return 0 + if i >= l.unstable+uint64(len(l.unstableEnts)) { + return 0 + } + return l.unstableEnts[i-l.unstable].Term } func (l *raftLog) entries(i uint64) []pb.Entry { - // never send out the first entry - // first entry is only used for matching - // prevLogTerm - if i == 0 { - panic("cannot return the first entry in log") - } return l.slice(i, l.lastIndex()+1) } -// allEntries returns all entries in the log, including the initial -// entry that is only used for prevLogTerm validation. This method -// should only be used for testing. +// allEntries returns all entries in the log. func (l *raftLog) allEntries() []pb.Entry { - return l.slice(l.firstIndex(), l.lastIndex()+1) + return l.entries(l.firstIndex()) } // isUpToDate determines if the given (lastIndex,term) log is more up-to-date @@ -224,10 +222,7 @@ func (l *raftLog) isUpToDate(lasti, term uint64) bool { } func (l *raftLog) matchTerm(i, term uint64) bool { - if e := l.at(i); e != nil { - return e.Term == term - } - return false + return l.term(i) == term } func (l *raftLog) maybeCommit(maxIndex, term uint64) bool { @@ -303,7 +298,9 @@ func (l *raftLog) slice(lo uint64, hi uint64) []pb.Entry { var ents []pb.Entry if lo < l.unstable { storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable)) - if err != nil { + if err == ErrSnapshotRequired { + return nil + } else if err != nil { panic(err) // TODO(bdarnell) } ents = append(ents, storedEnts...) diff --git a/raft/log_test.go b/raft/log_test.go index 27089d930..1d601c027 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -91,7 +91,7 @@ func TestIsUpToDate(t *testing.T) { } func TestAppend(t *testing.T) { - previousEnts := []pb.Entry{{}, {Term: 1}, {Term: 2}} + previousEnts := []pb.Entry{{Term: 1}, {Term: 2}} tests := []struct { after uint64 ents []pb.Entry @@ -283,7 +283,7 @@ func TestCompactionSideEffects(t *testing.T) { unstableIndex := uint64(750) lastTerm := lastIndex storage := NewMemoryStorage() - for i = 0; i <= unstableIndex; i++ { + for i = 1; i <= unstableIndex; i++ { storage.Append([]pb.Entry{{Term: uint64(i), Index: uint64(i)}}) } raftLog := newLog(storage) @@ -337,22 +337,23 @@ func TestCompactionSideEffects(t *testing.T) { } func TestUnstableEnts(t *testing.T) { - previousEnts := []pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}} + previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}} tests := []struct { unstable uint64 wents []pb.Entry wunstable uint64 }{ {3, nil, 3}, - {1, previousEnts[1:], 3}, - {0, append([]pb.Entry{{}}, previousEnts...), 3}, + {1, previousEnts, 3}, } for i, tt := range tests { storage := NewMemoryStorage() - storage.Append(previousEnts[:tt.unstable]) + if tt.unstable > 0 { + storage.Append(previousEnts[:tt.unstable-1]) + } raftLog := newLog(storage) - raftLog.append(raftLog.lastIndex(), previousEnts[tt.unstable:]...) + raftLog.append(raftLog.lastIndex(), previousEnts[tt.unstable-1:]...) ents := raftLog.unstableEntries() if l := len(ents); l > 0 { raftLog.stableTo(ents[l-1].Index) @@ -371,7 +372,6 @@ func TestStableTo(t *testing.T) { stable uint64 wunstable uint64 }{ - {0, 1}, {1, 2}, {2, 3}, } @@ -396,9 +396,9 @@ func TestCompaction(t *testing.T) { }{ // out of upper bound {1000, 1000, []uint64{1001}, []int{-1}, false}, - {1000, 1000, []uint64{300, 500, 800, 900}, []int{701, 501, 201, 101}, true}, + {1000, 1000, []uint64{300, 500, 800, 900}, []int{700, 500, 200, 100}, true}, // out of lower bound - {1000, 1000, []uint64{300, 299}, []int{701, -1}, false}, + {1000, 1000, []uint64{300, 299}, []int{700, -1}, false}, {0, 1000, []uint64{1}, []int{-1}, false}, } @@ -413,7 +413,7 @@ func TestCompaction(t *testing.T) { }() storage := NewMemoryStorage() - for i := uint64(0); i <= tt.lastIndex; i++ { + for i := uint64(1); i <= tt.lastIndex; i++ { storage.Append([]pb.Entry{{}}) } raftLog := newLog(storage) @@ -442,11 +442,11 @@ func TestLogRestore(t *testing.T) { raftLog.restore(pb.Snapshot{Index: index, Term: term}) // only has the guard entry - if len(raftLog.allEntries()) != 1 { + if len(raftLog.allEntries()) != 0 { t.Errorf("len = %d, want 1", len(raftLog.allEntries())) } - if raftLog.firstIndex() != index { - t.Errorf("firstIndex = %d, want %d", raftLog.firstIndex(), index) + if raftLog.firstIndex() != index+1 { + t.Errorf("firstIndex = %d, want %d", raftLog.firstIndex(), index+1) } if raftLog.applied != index { t.Errorf("applied = %d, want %d", raftLog.applied, index) @@ -474,7 +474,7 @@ func TestIsOutOfBounds(t *testing.T) { w bool }{ {offset - 1, true}, - {offset, false}, + {offset, true}, {offset + num/2, false}, {offset + num, false}, {offset + num + 1, true}, @@ -504,7 +504,7 @@ func TestAt(t *testing.T) { w *pb.Entry }{ {offset - 1, nil}, - {offset, &pb.Entry{Term: 0}}, + {offset, nil}, {offset + num/2, &pb.Entry{Term: num / 2}}, {offset + num - 1, &pb.Entry{Term: num - 1}}, {offset + num, nil}, @@ -518,6 +518,36 @@ func TestAt(t *testing.T) { } } +func TestTerm(t *testing.T) { + var i uint64 + offset := uint64(100) + num := uint64(100) + + l := newLog(NewMemoryStorage()) + l.restore(pb.Snapshot{Index: offset}) + for i = 1; i < num; i++ { + l.append(offset+i-1, pb.Entry{Term: i}) + } + + tests := []struct { + index uint64 + w uint64 + }{ + {offset - 1, 0}, + {offset, 0}, + {offset + num/2, num / 2}, + {offset + num - 1, num - 1}, + {offset + num, 0}, + } + + for i, tt := range tests { + term := l.term(tt.index) + if !reflect.DeepEqual(term, tt.w) { + t.Errorf("#%d: at = %d, want %d", i, term, tt.w) + } + } +} + func TestSlice(t *testing.T) { var i uint64 offset := uint64(100) @@ -535,7 +565,7 @@ func TestSlice(t *testing.T) { w []pb.Entry }{ {offset - 1, offset + 1, nil}, - {offset, offset + 1, []pb.Entry{{Term: 0}}}, + {offset, offset + 1, nil}, {offset + num/2, offset + num/2 + 1, []pb.Entry{{Term: num / 2}}}, {offset + num - 1, offset + num, []pb.Entry{{Term: num - 1}}}, {offset + num, offset + num + 1, nil}, diff --git a/raft/node_test.go b/raft/node_test.go index 1ebbb9285..0d1199524 100644 --- a/raft/node_test.go +++ b/raft/node_test.go @@ -308,7 +308,6 @@ func TestNodeStart(t *testing.T) { SoftState: &SoftState{Lead: 1, Nodes: []uint64{1}, RaftState: StateLeader}, HardState: raftpb.HardState{Term: 1, Commit: 2}, Entries: []raftpb.Entry{ - {}, {Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata}, {Term: 1, Index: 2}, }, @@ -352,7 +351,6 @@ func TestNodeStart(t *testing.T) { func TestNodeRestart(t *testing.T) { entries := []raftpb.Entry{ - {}, {Term: 1, Index: 1}, {Term: 1, Index: 2, Data: []byte("foo")}, } @@ -361,7 +359,7 @@ func TestNodeRestart(t *testing.T) { want := Ready{ HardState: emptyState, // commit upto index commit index in st - CommittedEntries: entries[1 : st.Commit+1], + CommittedEntries: entries[:st.Commit], } storage := NewMemoryStorage() @@ -429,7 +427,7 @@ func TestNodeCompact(t *testing.T) { } n.Stop() - if r.raftLog.firstIndex() != w.Index { + if r.raftLog.firstIndex() != w.Index+1 { t.Errorf("log.offset = %d, want %d", r.raftLog.firstIndex(), w.Index) } } diff --git a/raft/raft.go b/raft/raft.go index 9ec1c6145..20bde1dac 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -133,7 +133,7 @@ func newRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage } r.rand = rand.New(rand.NewSource(int64(id))) for _, p := range peers { - r.prs[p] = &progress{} + r.prs[p] = &progress{next: 1} } r.becomeFollower(0, None) return r @@ -187,12 +187,12 @@ func (r *raft) sendAppend(to uint64) { pr := r.prs[to] m := pb.Message{} m.To = to - m.Index = pr.next - 1 - if r.needSnapshot(m.Index) { + if r.needSnapshot(pr.next) { m.Type = pb.MsgSnap m.Snapshot = r.raftLog.snapshot } else { m.Type = pb.MsgApp + m.Index = pr.next - 1 m.LogTerm = r.raftLog.term(pr.next - 1) m.Entries = r.raftLog.entries(pr.next) m.Commit = r.raftLog.committed diff --git a/raft/raft_paper_test.go b/raft/raft_paper_test.go index 8433010c7..690e73836 100644 --- a/raft/raft_paper_test.go +++ b/raft/raft_paper_test.go @@ -514,7 +514,7 @@ func TestLeaderCommitPrecedingEntries(t *testing.T) { } for i, tt := range tests { storage := NewMemoryStorage() - storage.Append(append([]pb.Entry{{}}, tt...)) + storage.Append(tt) r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage) r.loadState(pb.HardState{Term: 2}) r.becomeCandidate() @@ -591,17 +591,17 @@ func TestFollowerCommitEntry(t *testing.T) { // append entries. // Reference: section 5.3 func TestFollowerCheckMsgApp(t *testing.T) { - ents := []pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}} + ents := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}} tests := []struct { term uint64 index uint64 wreject bool }{ + {0, 0, false}, {ents[0].Term, ents[0].Index, false}, {ents[1].Term, ents[1].Index, false}, - {ents[2].Term, ents[2].Index, false}, - {ents[1].Term, ents[1].Index + 1, true}, - {ents[1].Term + 1, ents[1].Index, true}, + {ents[0].Term, ents[0].Index + 1, true}, + {ents[0].Term + 1, ents[0].Index, true}, {3, 3, true}, } for i, tt := range tests { @@ -638,31 +638,31 @@ func TestFollowerAppendEntries(t *testing.T) { { 2, 2, []pb.Entry{{Term: 3, Index: 3}}, - []pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}, {Term: 3, Index: 3}}, + []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}, {Term: 3, Index: 3}}, []pb.Entry{{Term: 3, Index: 3}}, }, { 1, 1, []pb.Entry{{Term: 3, Index: 3}, {Term: 4, Index: 4}}, - []pb.Entry{{}, {Term: 1, Index: 1}, {Term: 3, Index: 3}, {Term: 4, Index: 4}}, + []pb.Entry{{Term: 1, Index: 1}, {Term: 3, Index: 3}, {Term: 4, Index: 4}}, []pb.Entry{{Term: 3, Index: 3}, {Term: 4, Index: 4}}, }, { 0, 0, []pb.Entry{{Term: 1, Index: 1}}, - []pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}}, + []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}, nil, }, { 0, 0, []pb.Entry{{Term: 3, Index: 3}}, - []pb.Entry{{}, {Term: 3, Index: 3}}, + []pb.Entry{{Term: 3, Index: 3}}, []pb.Entry{{Term: 3, Index: 3}}, }, } for i, tt := range tests { storage := NewMemoryStorage() - storage.Append([]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}}) + storage.Append([]pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}) r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage) r.becomeFollower(2, 2) @@ -813,17 +813,17 @@ func TestVoter(t *testing.T) { wreject bool }{ // same logterm - {[]pb.Entry{{}, {Term: 1, Index: 1}}, 1, 1, false}, - {[]pb.Entry{{}, {Term: 1, Index: 1}}, 1, 2, false}, - {[]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true}, + {[]pb.Entry{{Term: 1, Index: 1}}, 1, 1, false}, + {[]pb.Entry{{Term: 1, Index: 1}}, 1, 2, false}, + {[]pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true}, // candidate higher logterm - {[]pb.Entry{{}, {Term: 1, Index: 1}}, 2, 1, false}, - {[]pb.Entry{{}, {Term: 1, Index: 1}}, 2, 2, false}, - {[]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 1, Index: 2}}, 2, 1, false}, + {[]pb.Entry{{Term: 1, Index: 1}}, 2, 1, false}, + {[]pb.Entry{{Term: 1, Index: 1}}, 2, 2, false}, + {[]pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}}, 2, 1, false}, // voter higher logterm - {[]pb.Entry{{}, {Term: 2, Index: 1}}, 1, 1, true}, - {[]pb.Entry{{}, {Term: 2, Index: 1}}, 1, 2, true}, - {[]pb.Entry{{}, {Term: 2, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true}, + {[]pb.Entry{{Term: 2, Index: 1}}, 1, 1, true}, + {[]pb.Entry{{Term: 2, Index: 1}}, 1, 2, true}, + {[]pb.Entry{{Term: 2, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true}, } for i, tt := range tests { storage := NewMemoryStorage() @@ -850,7 +850,7 @@ func TestVoter(t *testing.T) { // current term are committed by counting replicas. // Reference: section 5.4.2 func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) { - ents := []pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}} + ents := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}} tests := []struct { index uint64 wcommit uint64 diff --git a/raft/raft_test.go b/raft/raft_test.go index 21c8dfbef..6798b4932 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -547,8 +547,8 @@ func TestCompact(t *testing.T) { } sm.compact(tt.compacti, tt.nodes, tt.snapd) sort.Sort(uint64Slice(sm.raftLog.snapshot.Nodes)) - if sm.raftLog.firstIndex() != tt.compacti { - t.Errorf("%d: log.firstIndex = %d, want %d", i, sm.raftLog.firstIndex(), tt.compacti) + 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) @@ -836,9 +836,9 @@ func TestAllServerStepdown(t *testing.T) { wterm uint64 windex uint64 }{ - {StateFollower, StateFollower, 3, 1}, - {StateCandidate, StateFollower, 3, 1}, - {StateLeader, StateFollower, 3, 2}, + {StateFollower, StateFollower, 3, 0}, + {StateCandidate, StateFollower, 3, 0}, + {StateLeader, StateFollower, 3, 1}, } tmsgTypes := [...]pb.MessageType{pb.MsgVote, pb.MsgApp} @@ -865,8 +865,11 @@ func TestAllServerStepdown(t *testing.T) { if sm.Term != tt.wterm { t.Errorf("#%d.%d term = %v , want %v", i, j, sm.Term, tt.wterm) } + if uint64(sm.raftLog.lastIndex()) != tt.windex { + t.Errorf("#%d.%d index = %v , want %v", i, j, sm.raftLog.lastIndex(), tt.windex) + } if uint64(len(sm.raftLog.allEntries())) != tt.windex { - t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.allEntries()), tt.windex) + t.Errorf("#%d.%d len(ents) = %v , want %v", i, j, len(sm.raftLog.allEntries()), tt.windex) } wlead := uint64(2) if msgType == pb.MsgVote { diff --git a/raft/storage.go b/raft/storage.go index 049b9ea17..072195c5b 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -23,9 +23,9 @@ import ( pb "github.com/coreos/etcd/raft/raftpb" ) -// ErrStorageEmpty is returned by Storage.GetLastIndex when there is -// no data. -var ErrStorageEmpty = errors.New("storage is empty") +// ErrSnapshotRequired is returned by Storage.Entries when a requested +// index is unavailable because it predates the last snapshot. +var ErrSnapshotRequired = errors.New("snapshot required; requested index is too old") // Storage is an interface that may be implemented by the application // to retrieve log entries from storage. @@ -36,11 +36,15 @@ var ErrStorageEmpty = errors.New("storage is empty") type Storage interface { // Entries returns a slice of log entries in the range [lo,hi). Entries(lo, hi uint64) ([]pb.Entry, error) - // GetLastIndex returns the index of the last entry in the log. - // If the log is empty it returns ErrStorageEmpty. + // Term returns the term of entry i, which must be in the range + // [FirstIndex()-1, LastIndex()]. The term of the entry before + // FirstIndex is retained for matching purposes even though the + // rest of that entry may not be available. + Term(i uint64) (uint64, error) + // LastIndex returns the index of the last entry in the log. LastIndex() (uint64, error) - // GetFirstIndex returns the index of the first log entry that is - // available via GetEntries (older entries have been incorporated + // FirstIndex returns the index of the first log entry that is + // available via Entries (older entries have been incorporated // into the latest Snapshot). FirstIndex() (uint64, error) // Compact discards all log entries prior to i. @@ -65,23 +69,36 @@ type MemoryStorage struct { // NewMemoryStorage creates an empty MemoryStorage. func NewMemoryStorage() *MemoryStorage { - return &MemoryStorage{} + return &MemoryStorage{ + // When starting from scratch populate the list with a dummy entry at term zero. + ents: make([]pb.Entry, 1), + } } // Entries implements the Storage interface. func (ms *MemoryStorage) Entries(lo, hi uint64) ([]pb.Entry, error) { ms.Lock() defer ms.Unlock() + if lo <= ms.offset { + return nil, ErrSnapshotRequired + } return ms.ents[lo-ms.offset : hi-ms.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)) { + return 0, ErrSnapshotRequired + } + return ms.ents[i-ms.offset].Term, nil +} + // LastIndex implements the Storage interface. func (ms *MemoryStorage) LastIndex() (uint64, error) { ms.Lock() defer ms.Unlock() - if len(ms.ents) == 0 { - return 0, ErrStorageEmpty - } return ms.offset + uint64(len(ms.ents)) - 1, nil } @@ -89,7 +106,7 @@ func (ms *MemoryStorage) LastIndex() (uint64, error) { func (ms *MemoryStorage) FirstIndex() (uint64, error) { ms.Lock() defer ms.Unlock() - return ms.offset, nil + return ms.offset + 1, nil } // Compact implements the Storage interface. From 46ee58c6f0604ddc03c1c4643e82a58a80256429 Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Tue, 18 Nov 2014 13:15:10 -0500 Subject: [PATCH 08/52] raft: Rename ErrSnapshotRequired To ErrCompacted. --- raft/log.go | 4 ++-- raft/storage.go | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/raft/log.go b/raft/log.go index e3a00f70b..50a44b53b 100644 --- a/raft/log.go +++ b/raft/log.go @@ -191,7 +191,7 @@ func (l *raftLog) lastTerm() uint64 { func (l *raftLog) term(i uint64) uint64 { if i < l.unstable { t, err := l.storage.Term(i) - if err == ErrSnapshotRequired { + if err == ErrCompacted { return 0 } else if err != nil { panic(err) // TODO(bdarnell) @@ -299,7 +299,7 @@ func (l *raftLog) slice(lo uint64, hi uint64) []pb.Entry { var ents []pb.Entry if lo < l.unstable { storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable)) - if err == ErrSnapshotRequired { + if err == ErrCompacted { return nil } else if err != nil { panic(err) // TODO(bdarnell) diff --git a/raft/storage.go b/raft/storage.go index 072195c5b..d00c24fcc 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -23,9 +23,9 @@ import ( pb "github.com/coreos/etcd/raft/raftpb" ) -// ErrSnapshotRequired is returned by Storage.Entries when a requested +// ErrCompacted is returned by Storage.Entries when a requested // index is unavailable because it predates the last snapshot. -var ErrSnapshotRequired = errors.New("snapshot required; requested index is too old") +var ErrCompacted = errors.New("requested index is unavailable due to compaction") // Storage is an interface that may be implemented by the application // to retrieve log entries from storage. @@ -80,7 +80,7 @@ func (ms *MemoryStorage) Entries(lo, hi uint64) ([]pb.Entry, error) { ms.Lock() defer ms.Unlock() if lo <= ms.offset { - return nil, ErrSnapshotRequired + return nil, ErrCompacted } return ms.ents[lo-ms.offset : hi-ms.offset], nil } @@ -90,7 +90,7 @@ func (ms *MemoryStorage) Term(i uint64) (uint64, error) { ms.Lock() defer ms.Unlock() if i < ms.offset || i > ms.offset+uint64(len(ms.ents)) { - return 0, ErrSnapshotRequired + return 0, ErrCompacted } return ms.ents[i-ms.offset].Term, nil } From 355ee4f393b4677f14d354dbc4fa0e351264cc37 Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Wed, 19 Nov 2014 16:17:50 -0500 Subject: [PATCH 09/52] 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. --- etcdctl/command/backup_command.go | 2 +- etcdserver/force_cluster.go | 8 +- etcdserver/force_cluster_test.go | 25 ++- etcdserver/server.go | 44 ++-- etcdserver/server_test.go | 48 ++-- migrate/etcd4.go | 2 +- migrate/snapshot.go | 14 +- raft/log.go | 62 ++---- raft/log_test.go | 56 ++--- raft/node.go | 114 +++++----- raft/node_test.go | 79 ++----- raft/raft.go | 43 ++-- raft/raft_test.go | 118 ++++------ raft/raftpb/raft.pb.go | 356 ++++++++++++++++++++++++------ raft/raftpb/raft.proto | 16 +- raft/storage.go | 84 +++++-- snap/snapshotter.go | 2 +- snap/snapshotter_test.go | 14 +- 18 files changed, 642 insertions(+), 445 deletions(-) diff --git a/etcdctl/command/backup_command.go b/etcdctl/command/backup_command.go index 4245d957c..309e13c36 100644 --- a/etcdctl/command/backup_command.go +++ b/etcdctl/command/backup_command.go @@ -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) } diff --git a/etcdserver/force_cluster.go b/etcdserver/force_cluster.go index c409ff02c..090e71f3b 100644 --- a/etcdserver/force_cluster.go +++ b/etcdserver/force_cluster.go @@ -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 } } diff --git a/etcdserver/force_cluster_test.go b/etcdserver/force_cluster_test.go index 368ef889b..2131c3f37 100644 --- a/etcdserver/force_cluster_test.go +++ b/etcdserver/force_cluster_test.go @@ -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) } diff --git a/etcdserver/server.go b/etcdserver/server.go index 4090f4edb..8067a929f 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -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 } diff --git a/etcdserver/server_test.go b/etcdserver/server_test.go index 069c9bb70..31a6f1e64 100644 --- a/etcdserver/server_test.go +++ b/etcdserver/server_test.go @@ -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 { diff --git a/migrate/etcd4.go b/migrate/etcd4.go index cf0a2ee1e..ea92a4f8e 100644 --- a/migrate/etcd4.go +++ b/migrate/etcd4.go @@ -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) diff --git a/migrate/snapshot.go b/migrate/snapshot.go index 57dcf501f..f568ac2bb 100644 --- a/migrate/snapshot.go +++ b/migrate/snapshot.go @@ -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 diff --git a/raft/log.go b/raft/log.go index 50a44b53b..17d17a252 100644 --- a/raft/log.go +++ b/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 { diff --git a/raft/log_test.go b/raft/log_test.go index 551054de1..ee3f387f0 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -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}) } diff --git a/raft/node.go b/raft/node.go index b62c083e0..ef0c82e83 100644 --- a/raft/node.go +++ b/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 } diff --git a/raft/node_test.go b/raft/node_test.go index cc6530195..219dfe66d 100644 --- a/raft/node_test.go +++ b/raft/node_test.go @@ -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() diff --git a/raft/raft.go b/raft/raft.go index 20bde1dac..4dd9c2a1a 100644 --- a/raft/raft.go +++ b/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 { diff --git a/raft/raft_test.go b/raft/raft_test.go index 6798b4932..7cf667103 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -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{{}}}) diff --git a/raft/raftpb/raft.pb.go b/raft/raftpb/raft.pb.go index 03bdb8c83..4c3eaafda 100644 --- a/raft/raftpb/raft.pb.go +++ b/raft/raftpb/raft.pb.go @@ -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) diff --git a/raft/raftpb/raft.proto b/raft/raftpb/raft.proto index 7b60393b8..247b5a0db 100644 --- a/raft/raftpb/raft.proto +++ b/raft/raftpb/raft.proto @@ -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; diff --git a/raft/storage.go b/raft/storage.go index d00c24fcc..403fca014 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -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 } diff --git a/snap/snapshotter.go b/snap/snapshotter.go index 1cdd6e7e5..8d9eeadc3 100644 --- a/snap/snapshotter.go +++ b/snap/snapshotter.go @@ -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} diff --git a/snap/snapshotter_test.go b/snap/snapshotter_test.go index 6b6f1e89e..6998d62e9 100644 --- a/snap/snapshotter_test.go +++ b/snap/snapshotter_test.go @@ -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) From 30690d15d92dae33f0a90f5429bab89ce924bf9b Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Fri, 21 Nov 2014 15:27:17 -0500 Subject: [PATCH 10/52] Re-enable a few tests I had missed. Fix integration test for the change to log entry zero. Increase test timeouts since integration tests often take longer than 10s for me. --- etcdserver/server.go | 2 ++ integration/member_test.go | 1 - integration/v2_http_kv_test.go | 2 -- test | 2 +- 4 files changed, 3 insertions(+), 4 deletions(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index c86970c3f..52d816a01 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -249,6 +249,8 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { log.Printf("etcdserver: recovering from snapshot at index %d", snapshot.Metadata.Index) st.Recovery(snapshot.Data) index = snapshot.Metadata.Index + } else { + index = 1 } cfg.Cluster = NewClusterFromStore(cfg.Cluster.token, st) cfg.Print() diff --git a/integration/member_test.go b/integration/member_test.go index dcd71ebdc..7a8457fe7 100644 --- a/integration/member_test.go +++ b/integration/member_test.go @@ -7,7 +7,6 @@ import ( ) func TestRestartMember(t *testing.T) { - t.Skip("TODO(bdarnell): re-enable this test") defer afterTest(t) c := NewCluster(t, 3) c.Launch(t) diff --git a/integration/v2_http_kv_test.go b/integration/v2_http_kv_test.go index 140894bf3..55addb7cf 100644 --- a/integration/v2_http_kv_test.go +++ b/integration/v2_http_kv_test.go @@ -865,7 +865,6 @@ func TestV2WatchWithIndex(t *testing.T) { } func TestV2WatchKeyInDir(t *testing.T) { - t.Skip("TODO(bdarnell): re-enable this test") cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -914,7 +913,6 @@ func TestV2WatchKeyInDir(t *testing.T) { } func TestV2Head(t *testing.T) { - t.Skip("TODO(bdarnell): re-enable this test") cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) diff --git a/test b/test index 41a366388..aa8a12076 100755 --- a/test +++ b/test @@ -39,7 +39,7 @@ split=(${TEST// / }) TEST=${split[@]/#/${REPO_PATH}/} echo "Running tests..." -go test -timeout 10s ${COVER} $@ ${TEST} --race +go test -timeout 20s ${COVER} $@ ${TEST} --race echo "Checking gofmt..." fmtRes=$(gofmt -l $FMT) From 03c8881e35ed649f4bdd8197829edd562270568c Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Fri, 21 Nov 2014 16:37:55 -0500 Subject: [PATCH 11/52] Fix TestSlowNodeRestore --- raft/raft_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft/raft_test.go b/raft/raft_test.go index 82373a334..c5d07387b 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -1179,7 +1179,7 @@ func TestSlowNodeRestore(t *testing.T) { } lead := nt.peers[1].(*raft) nextEnts(lead, nt.storage[1]) - //lead.compact(lead.raftLog.applied, lead.nodes(), nil) + nt.storage[1].Compact(lead.raftLog.applied, &pb.ConfState{Nodes: lead.nodes()}, nil) nt.recover() // trigger a snapshot From 9ddd8ee539081d1d11493b04f828401944664891 Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Fri, 21 Nov 2014 17:22:20 -0500 Subject: [PATCH 12/52] Rename Storage.HardState back to InitialState and include ConfState. This fixes integration/migration_test.go (and highlights the fact that we need some more raft-level testing of restoring from snapshots). --- raft/raft.go | 15 ++++++++++++--- raft/raft_test.go | 2 +- raft/storage.go | 10 +++++----- 3 files changed, 18 insertions(+), 9 deletions(-) diff --git a/raft/raft.go b/raft/raft.go index d5bbfba7e..fe31c1cd8 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -145,10 +145,19 @@ func newRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage panic("cannot use none id") } log := newLog(storage) - st, err := storage.HardState() + hs, cs, err := storage.InitialState() if err != nil { panic(err) // TODO(bdarnell) } + if len(cs.Nodes) > 0 { + if len(peers) > 0 { + // TODO(bdarnell): the peers argument is always nil except in + // tests; the argument should be removed and these tests should be + // updated to specify their nodes through a snapshot. + panic("cannot specify both newRaft(peers) and ConfState.Nodes)") + } + peers = cs.Nodes + } r := &raft{ id: id, lead: None, @@ -161,8 +170,8 @@ 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) + if !isHardStateEqual(hs, emptyState) { + r.loadState(hs) } r.becomeFollower(0, None) return r diff --git a/raft/raft_test.go b/raft/raft_test.go index c5d07387b..1d9003148 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -974,7 +974,7 @@ func TestBcastBeat(t *testing.T) { } storage := NewMemoryStorage() storage.ApplySnapshot(s) - sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage) + sm := newRaft(1, nil, 10, 1, storage) sm.Term = 1 sm.becomeCandidate() diff --git a/raft/storage.go b/raft/storage.go index 403fca014..94dddc498 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -35,8 +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) + // InitialState returns the saved HardState and ConfState information. + InitialState() (pb.HardState, pb.ConfState, 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 @@ -79,9 +79,9 @@ func NewMemoryStorage() *MemoryStorage { } } -// HardState implements the Storage interface. -func (ms *MemoryStorage) HardState() (pb.HardState, error) { - return ms.hardState, nil +// InitialState implements the Storage interface. +func (ms *MemoryStorage) InitialState() (pb.HardState, pb.ConfState, error) { + return ms.hardState, ms.snapshot.Metadata.ConfState, nil } // SetHardState saves the current HardState. From bdbafe2cf33717a21dbbaca9e7e69d12bb05bd38 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 08:36:15 -0800 Subject: [PATCH 13/52] raft: use max in log.slice --- raft/log.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/raft/log.go b/raft/log.go index 5666ed1c5..fca093b2c 100644 --- a/raft/log.go +++ b/raft/log.go @@ -285,12 +285,7 @@ func (l *raftLog) slice(lo uint64, hi uint64) []pb.Entry { ents = append(ents, storedEnts...) } if len(l.unstableEnts) > 0 && hi > l.unstable { - var firstUnstable uint64 - if lo < l.unstable { - firstUnstable = l.unstable - } else { - firstUnstable = lo - } + firstUnstable := max(lo, l.unstable) ents = append(ents, l.unstableEnts[firstUnstable-l.unstable:hi-l.unstable]...) } return ents From 6c8e294d2043c2a673bc99ee0117051d90fe8876 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 08:37:26 -0800 Subject: [PATCH 14/52] test: longer test timeout --- test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test b/test index aa8a12076..be38a00be 100755 --- a/test +++ b/test @@ -39,7 +39,7 @@ split=(${TEST// / }) TEST=${split[@]/#/${REPO_PATH}/} echo "Running tests..." -go test -timeout 20s ${COVER} $@ ${TEST} --race +go test -timeout 60s ${COVER} $@ ${TEST} --race echo "Checking gofmt..." fmtRes=$(gofmt -l $FMT) From f3cef87c698c0512ba824935732fc67a9c4e26e0 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 08:43:34 -0800 Subject: [PATCH 15/52] raft: remove extra empty line in log.go --- raft/log.go | 1 - 1 file changed, 1 deletion(-) diff --git a/raft/log.go b/raft/log.go index fca093b2c..7b5ac0dd2 100644 --- a/raft/log.go +++ b/raft/log.go @@ -69,7 +69,6 @@ func newLog(storage Storage) *raftLog { func (l *raftLog) String() string { return fmt.Sprintf("unstable=%d committed=%d applied=%d", l.unstable, l.committed, l.applied) - } // maybeAppend returns (0, false) if the entries cannot be appended. Otherwise, From bc0e72acb9444dfdf4416f06d8308afb39c48e59 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 09:01:25 -0800 Subject: [PATCH 16/52] raft: clean up panic in log.go --- raft/log.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/raft/log.go b/raft/log.go index 7b5ac0dd2..05c2b6c25 100644 --- a/raft/log.go +++ b/raft/log.go @@ -46,7 +46,7 @@ type raftLog struct { func newLog(storage Storage) *raftLog { if storage == nil { - panic("storage must not be nil") + log.Panic("storage must not be nil") } log := &raftLog{ storage: storage, @@ -93,7 +93,7 @@ func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 { if after < l.committed { - log.Panicf("appending after %d, but already committed through %d", after, l.committed) + log.Panicf("after(%d) out of range [committed(%d)]", after, l.committed) } if after < l.unstable { // The log is being truncated to before our current unstable @@ -170,7 +170,7 @@ func (l *raftLog) commitTo(tocommit uint64) { // never decrease commit if l.committed < tocommit { if l.lastIndex() < tocommit { - panic("committed out of range") + log.Panicf("tocommit(%d) is out of range [lastIndex(%d)]", tocommit, l.lastIndex()) } l.committed = tocommit } @@ -181,14 +181,14 @@ func (l *raftLog) appliedTo(i uint64) { return } if l.committed < i || i < l.applied { - log.Panicf("applied[%d] is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed) + log.Panicf("applied(%d) is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed) } l.applied = i } func (l *raftLog) stableTo(i uint64) { if i < l.unstable || i+1-l.unstable > uint64(len(l.unstableEnts)) { - log.Panicf("stableTo(%d) is out of range (unstable=%d, len(unstableEnts)=%d)", + log.Panicf("stableTo(%d) is out of range [unstable(%d), len(unstableEnts)(%d)]", i, l.unstable, len(l.unstableEnts)) } l.unstableEnts = l.unstableEnts[i+1-l.unstable:] From 0a46c70f5d8e4e917c4948bfd5e49ca94d2ef744 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 09:04:45 -0800 Subject: [PATCH 17/52] raft: use empty slice in unstableEntries in log.go --- raft/log.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft/log.go b/raft/log.go index 05c2b6c25..1c4690301 100644 --- a/raft/log.go +++ b/raft/log.go @@ -133,7 +133,7 @@ func (l *raftLog) unstableEntries() []pb.Entry { if len(l.unstableEnts) == 0 { return nil } - return append([]pb.Entry(nil), l.unstableEnts...) + return append([]pb.Entry{}, l.unstableEnts...) } // nextEnts returns all the available entries for execution. From 94190286ff3e2eab1f9bb5a0ee7181343ee530ca Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 09:05:40 -0800 Subject: [PATCH 18/52] raft: add comment for append in unstableEntries in log.go --- raft/log.go | 1 + 1 file changed, 1 insertion(+) diff --git a/raft/log.go b/raft/log.go index 1c4690301..9815a2b32 100644 --- a/raft/log.go +++ b/raft/log.go @@ -133,6 +133,7 @@ func (l *raftLog) unstableEntries() []pb.Entry { if len(l.unstableEnts) == 0 { return nil } + // copy unstable entries to an empty slice return append([]pb.Entry{}, l.unstableEnts...) } From 3dd4c458ca6c4d943a6668bd3955aacb39dd0987 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 10:13:56 -0800 Subject: [PATCH 19/52] raft: refactor term in log.go --- raft/log.go | 23 ++++++++++++++--------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/raft/log.go b/raft/log.go index 9815a2b32..857caa948 100644 --- a/raft/log.go +++ b/raft/log.go @@ -201,18 +201,23 @@ func (l *raftLog) lastTerm() uint64 { } func (l *raftLog) term(i uint64) uint64 { - if i < l.unstable { - t, err := l.storage.Term(i) - if err == ErrCompacted { - return 0 - } else if err != nil { - panic(err) // TODO(bdarnell) - } - return t - } if i >= l.unstable+uint64(len(l.unstableEnts)) { return 0 } + + if i < l.unstable { + t, err := l.storage.Term(i) + if err == nil { + return t + } + if err == ErrCompacted { + return 0 + } else { + panic(err) // TODO(bdarnell) + return 0 + } + } + return l.unstableEnts[i-l.unstable].Term } From 08f156a1def3f68ada56f3e15d44500292b1d424 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 10:27:18 -0800 Subject: [PATCH 20/52] etcdserver: remove extra empty line in snapshot func --- etcdserver/server.go | 1 - 1 file changed, 1 deletion(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index f70c46f4d..9a7919702 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -817,7 +817,6 @@ func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) { 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, From e8afdcfe0aca109920a0ea49171ccd1e6e5e9f7d Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 10:40:38 -0800 Subject: [PATCH 21/52] raft: refactor testUnstableEnts --- raft/log_test.go | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/raft/log_test.go b/raft/log_test.go index 8ea5cb8b8..8d7b93b4e 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -369,24 +369,27 @@ func TestNextEnts(t *testing.T) { } } +// TestUnstableEnts ensures unstableEntries returns the unstable part of the +// entries correctly. func TestUnstableEnts(t *testing.T) { previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}} tests := []struct { - unstable uint64 - wents []pb.Entry - wunstable uint64 + unstable uint64 + wents []pb.Entry }{ - {3, nil, 3}, - {1, previousEnts, 3}, + {3, nil}, + {1, previousEnts}, } for i, tt := range tests { + // append stable entries to storage storage := NewMemoryStorage() - if tt.unstable > 0 { - storage.Append(previousEnts[:tt.unstable-1]) - } + storage.Append(previousEnts[:tt.unstable-1]) + + // append unstable entries to raftlog raftLog := newLog(storage) raftLog.append(raftLog.lastIndex(), previousEnts[tt.unstable-1:]...) + ents := raftLog.unstableEntries() if l := len(ents); l > 0 { raftLog.stableTo(ents[l-1].Index) @@ -394,8 +397,9 @@ func TestUnstableEnts(t *testing.T) { if !reflect.DeepEqual(ents, tt.wents) { t.Errorf("#%d: unstableEnts = %+v, want %+v", i, ents, tt.wents) } - if g := raftLog.unstable; g != tt.wunstable { - t.Errorf("#%d: unstable = %d, want %d", i, g, tt.wunstable) + w := previousEnts[len(previousEnts)-1].Index + 1 + if g := raftLog.unstable; g != w { + t.Errorf("#%d: unstable = %d, want %d", i, g, w) } } } From 62a8df304ac127f6e3c756f4ed123a9a3fb9a09a Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 11:10:02 -0800 Subject: [PATCH 22/52] raft: fix error message in TestLogRestore --- raft/log_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/raft/log_test.go b/raft/log_test.go index 8d7b93b4e..9878102aa 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -505,9 +505,8 @@ func TestLogRestore(t *testing.T) { storage.ApplySnapshot(pb.Snapshot{Metadata: snap}) raftLog := newLog(storage) - // only has the guard entry if len(raftLog.allEntries()) != 0 { - t.Errorf("len = %d, want 1", len(raftLog.allEntries())) + t.Errorf("len = %d, want 0", len(raftLog.allEntries())) } if raftLog.firstIndex() != index+1 { t.Errorf("firstIndex = %d, want %d", raftLog.firstIndex(), index+1) From 4b7af29c37e312e46b2b0df93238d439322da86d Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 14:04:57 -0800 Subject: [PATCH 23/52] etcdserver: fix TriggerSnap test. Sleep for millisecond to allow the server to apply the first nop and first put separately. --- etcdserver/server_test.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/etcdserver/server_test.go b/etcdserver/server_test.go index 956e5ee69..a9a79837c 100644 --- a/etcdserver/server_test.go +++ b/etcdserver/server_test.go @@ -963,16 +963,19 @@ func TestTriggerSnap(t *testing.T) { } srv.start() + // wait for saving nop + time.Sleep(time.Millisecond) for i := 0; uint64(i) < srv.snapCount-1; i++ { srv.Do(ctx, pb.Request{Method: "PUT", ID: 1}) } + // wait for saving the last entry time.Sleep(time.Millisecond) srv.Stop() gaction := p.Action() // each operation is recorded as a Save // BootstrapConfig/Nop + (SnapCount - 1) * Puts + Cut + SaveSnap = Save + (SnapCount - 1) * Save + Cut + SaveSnap - wcnt := 1 + int(srv.snapCount) + wcnt := 2 + int(srv.snapCount) if len(gaction) != wcnt { t.Fatalf("len(action) = %d, want %d", len(gaction), wcnt) } From 453133977de542af702e608b2fa894d34720c561 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 14:28:13 -0800 Subject: [PATCH 24/52] etcdserver: save snapshot only if the index is greater than previous snap index --- etcdserver/server.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index 9a7919702..3f731555a 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -395,10 +395,11 @@ 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 !raft.IsEmptySnap(rd.Snapshot) { + if !raft.IsEmptySnap(rd.Snapshot) && rd.Snapshot.Metadata.Index > snapi { if err := s.storage.SaveSnap(rd.Snapshot); err != nil { log.Fatalf("etcdserver: create snapshot error: %v", err) } + snapi = rd.Snapshot.Metadata.Index } s.sendhub.Send(rd.Messages) From d69e4dbe6d83a85796a92e676240ecb1d331d01f Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 14:57:08 -0800 Subject: [PATCH 25/52] etcdserver: initial index to 1 --- etcdserver/server.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index 3f731555a..3b33a30e9 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -270,7 +270,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { if cfg.ShouldDiscover() { log.Printf("etcdserver: warn: ignoring discovery: etcd has already been initialized and has a valid log in %q", cfg.WALDir()) } - var index uint64 + index := uint64(1) snapshot, err := ss.Load() if err != nil && err != snap.ErrNoSnapshot { return nil, err @@ -279,8 +279,6 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { log.Printf("etcdserver: recovering from snapshot at index %d", snapshot.Metadata.Index) st.Recovery(snapshot.Data) index = snapshot.Metadata.Index - } else { - index = 1 } cfg.Cluster = NewClusterFromStore(cfg.Cluster.token, st) cfg.Print() From 2876c652ab03b41742bc4c67522f95331243bcf9 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 15:00:38 -0800 Subject: [PATCH 26/52] raft: fix for go vet --- raft/log.go | 1 - 1 file changed, 1 deletion(-) diff --git a/raft/log.go b/raft/log.go index 857caa948..b13bd8b52 100644 --- a/raft/log.go +++ b/raft/log.go @@ -214,7 +214,6 @@ func (l *raftLog) term(i uint64) uint64 { return 0 } else { panic(err) // TODO(bdarnell) - return 0 } } From 10ebf1a335c0037dd9de2851dfcb9ec59c9d2e9b Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 16:36:59 -0800 Subject: [PATCH 27/52] raft: fix memoryStorage append --- raft/log.go | 2 +- raft/storage.go | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/raft/log.go b/raft/log.go index b13bd8b52..fd9ccdae1 100644 --- a/raft/log.go +++ b/raft/log.go @@ -81,7 +81,7 @@ func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry switch { case ci == 0: case ci <= l.committed: - panic("conflict with committed entry") + log.Panicf("conflict(%d) with committed entry [committed(%d)]", ci, l.committed) default: l.append(ci-1, ents[ci-from:]...) } diff --git a/raft/storage.go b/raft/storage.go index 94dddc498..9d62306e4 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -176,5 +176,12 @@ func (ms *MemoryStorage) Compact(i uint64, cs *pb.ConfState, data []byte) error func (ms *MemoryStorage) Append(entries []pb.Entry) { ms.Lock() defer ms.Unlock() + if len(entries) == 0 { + return + } + offset := entries[0].Index - ms.snapshot.Metadata.Index + if uint64(len(ms.ents)) >= offset { + ms.ents = ms.ents[:offset] + } ms.ents = append(ms.ents, entries...) } From 65ad1f6ffd57f0bbf19f3411917ae50441f61992 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 24 Nov 2014 17:13:47 -0800 Subject: [PATCH 28/52] raft: attach Index to Entry in all tests --- raft/log_test.go | 80 ++++++++++++++++++++--------------------- raft/raft_paper_test.go | 2 +- raft/raft_test.go | 14 ++++---- 3 files changed, 48 insertions(+), 48 deletions(-) diff --git a/raft/log_test.go b/raft/log_test.go index 9878102aa..4bfd37aab 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -24,7 +24,7 @@ import ( ) func TestFindConflict(t *testing.T) { - previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}} + previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}} tests := []struct { from uint64 ents []pb.Entry @@ -34,18 +34,18 @@ func TestFindConflict(t *testing.T) { {1, []pb.Entry{}, 0}, {3, []pb.Entry{}, 0}, // no conflict - {1, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}, 0}, - {2, []pb.Entry{{Term: 2}, {Term: 3}}, 0}, - {3, []pb.Entry{{Term: 3}}, 0}, + {1, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0}, + {2, []pb.Entry{{Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0}, + {3, []pb.Entry{{Index: 3, Term: 3}}, 0}, // no conflict, but has new entries - {1, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}, {Term: 4}, {Term: 4}}, 4}, - {2, []pb.Entry{{Term: 2}, {Term: 3}, {Term: 4}, {Term: 4}}, 4}, - {3, []pb.Entry{{Term: 3}, {Term: 4}, {Term: 4}}, 4}, - {4, []pb.Entry{{Term: 4}, {Term: 4}}, 4}, + {1, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4}, + {2, []pb.Entry{{Index: 2, Term: 2}, {Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4}, + {3, []pb.Entry{{Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4}, + {4, []pb.Entry{{Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4}, // conflicts with existing entries - {1, []pb.Entry{{Term: 4}, {Term: 4}}, 1}, - {2, []pb.Entry{{Term: 1}, {Term: 4}, {Term: 4}}, 2}, - {3, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 4}, {Term: 4}}, 3}, + {1, []pb.Entry{{Index: 1, Term: 4}, {Index: 2, Term: 4}}, 1}, + {2, []pb.Entry{{Index: 2, Term: 1}, {Index: 3, Term: 4}, {Index: 4, Term: 4}}, 2}, + {3, []pb.Entry{{Index: 3, Term: 1}, {Index: 4, Term: 2}, {Index: 5, Term: 4}, {Index: 6, Term: 4}}, 3}, } for i, tt := range tests { @@ -60,7 +60,7 @@ func TestFindConflict(t *testing.T) { } func TestIsUpToDate(t *testing.T) { - previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}} + previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}} raftLog := newLog(NewMemoryStorage()) raftLog.append(raftLog.lastIndex(), previousEnts...) tests := []struct { @@ -91,7 +91,7 @@ func TestIsUpToDate(t *testing.T) { } func TestAppend(t *testing.T) { - previousEnts := []pb.Entry{{Term: 1}, {Term: 2}} + previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}} tests := []struct { after uint64 ents []pb.Entry @@ -103,30 +103,30 @@ func TestAppend(t *testing.T) { 2, []pb.Entry{}, 2, - []pb.Entry{{Term: 1}, {Term: 2}}, + []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}, 3, }, { 2, - []pb.Entry{{Term: 2}}, + []pb.Entry{{Index: 3, Term: 2}}, 3, - []pb.Entry{{Term: 1}, {Term: 2}, {Term: 2}}, + []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 2}}, 3, }, // conflicts with index 1 { 0, - []pb.Entry{{Term: 2}}, + []pb.Entry{{Index: 1, Term: 2}}, 1, - []pb.Entry{{Term: 2}}, + []pb.Entry{{Index: 1, Term: 2}}, 1, }, // conflicts with index 2 { 1, - []pb.Entry{{Term: 3}, {Term: 3}}, + []pb.Entry{{Index: 2, Term: 3}, {Index: 3, Term: 3}}, 3, - []pb.Entry{{Term: 1}, {Term: 3}, {Term: 3}}, + []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 3}, {Index: 3, Term: 3}}, 2, }, } @@ -158,7 +158,7 @@ func TestAppend(t *testing.T) { // If the given (index, term) does not match with the existing log: // return false func TestLogMaybeAppend(t *testing.T) { - previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}} + previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}} lastindex := uint64(3) lastterm := uint64(3) commit := uint64(1) @@ -176,12 +176,12 @@ func TestLogMaybeAppend(t *testing.T) { }{ // not match: term is different { - lastterm - 1, lastindex, lastindex, []pb.Entry{{Term: 4}}, + lastterm - 1, lastindex, lastindex, []pb.Entry{{Index: lastindex + 1, Term: 4}}, 0, false, commit, false, }, // not match: index out of bound { - lastterm, lastindex + 1, lastindex, []pb.Entry{{Term: 4}}, + lastterm, lastindex + 1, lastindex, []pb.Entry{{Index: lastindex + 2, Term: 4}}, 0, false, commit, false, }, // match with the last existing entry @@ -206,36 +206,36 @@ func TestLogMaybeAppend(t *testing.T) { 0, true, commit, false, // commit do not decrease }, { - lastterm, lastindex, lastindex, []pb.Entry{{Term: 4}}, + lastterm, lastindex, lastindex, []pb.Entry{{Index: lastindex + 1, Term: 4}}, lastindex + 1, true, lastindex, false, }, { - lastterm, lastindex, lastindex + 1, []pb.Entry{{Term: 4}}, + lastterm, lastindex, lastindex + 1, []pb.Entry{{Index: lastindex + 1, Term: 4}}, lastindex + 1, true, lastindex + 1, false, }, { - lastterm, lastindex, lastindex + 2, []pb.Entry{{Term: 4}}, + lastterm, lastindex, lastindex + 2, []pb.Entry{{Index: lastindex + 1, Term: 4}}, lastindex + 1, true, lastindex + 1, false, // do not increase commit higher than lastnewi }, { - lastterm, lastindex, lastindex + 2, []pb.Entry{{Term: 4}, {Term: 4}}, + lastterm, lastindex, lastindex + 2, []pb.Entry{{Index: lastindex + 1, Term: 4}, {Index: lastindex + 2, Term: 4}}, lastindex + 2, true, lastindex + 2, false, }, // match with the the entry in the middle { - lastterm - 1, lastindex - 1, lastindex, []pb.Entry{{Term: 4}}, + lastterm - 1, lastindex - 1, lastindex, []pb.Entry{{Index: lastindex, Term: 4}}, lastindex, true, lastindex, false, }, { - lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Term: 4}}, + lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Index: lastindex - 1, Term: 4}}, lastindex - 1, true, lastindex - 1, false, }, { - lastterm - 3, lastindex - 3, lastindex, []pb.Entry{{Term: 4}}, + lastterm - 3, lastindex - 3, lastindex, []pb.Entry{{Index: lastindex - 2, Term: 4}}, lastindex - 2, true, lastindex - 2, true, // conflict with existing committed entry }, { - lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Term: 4}, {Term: 4}}, + lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Index: lastindex - 1, Term: 4}, {Index: lastindex, Term: 4}}, lastindex, true, lastindex, false, }, } @@ -325,7 +325,7 @@ func TestCompactionSideEffects(t *testing.T) { } prev := raftLog.lastIndex() - raftLog.append(raftLog.lastIndex(), pb.Entry{Term: raftLog.lastIndex() + 1}) + raftLog.append(raftLog.lastIndex(), pb.Entry{Index: raftLog.lastIndex() + 1, Term: raftLog.lastIndex() + 1}) if raftLog.lastIndex() != prev+1 { t.Errorf("lastIndex = %d, want = %d", raftLog.lastIndex(), prev+1) } @@ -481,7 +481,7 @@ func TestCompaction(t *testing.T) { storage := NewMemoryStorage() for i := uint64(1); i <= tt.lastIndex; i++ { - storage.Append([]pb.Entry{{}}) + storage.Append([]pb.Entry{{Index: i}}) } raftLog := newLog(storage) raftLog.maybeCommit(tt.lastIndex, 0) @@ -558,7 +558,7 @@ func TestAt(t *testing.T) { 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}) + l.append(offset+i-1, pb.Entry{Index: i, Term: i}) } tests := []struct { @@ -567,8 +567,8 @@ func TestAt(t *testing.T) { }{ {offset - 1, nil}, {offset, nil}, - {offset + num/2, &pb.Entry{Term: num / 2}}, - {offset + num - 1, &pb.Entry{Term: num - 1}}, + {offset + num/2, &pb.Entry{Index: num / 2, Term: num / 2}}, + {offset + num - 1, &pb.Entry{Index: num - 1, Term: num - 1}}, {offset + num, nil}, } @@ -589,7 +589,7 @@ func TestTerm(t *testing.T) { 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}) + l.append(offset+i-1, pb.Entry{Index: i, Term: i}) } tests := []struct { @@ -620,7 +620,7 @@ func TestSlice(t *testing.T) { 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}) + l.append(offset+i-1, pb.Entry{Index: i, Term: i}) } tests := []struct { @@ -630,8 +630,8 @@ func TestSlice(t *testing.T) { }{ {offset - 1, offset + 1, nil}, {offset, offset + 1, nil}, - {offset + num/2, offset + num/2 + 1, []pb.Entry{{Term: num / 2}}}, - {offset + num - 1, offset + num, []pb.Entry{{Term: num - 1}}}, + {offset + num/2, offset + num/2 + 1, []pb.Entry{{Index: num / 2, Term: num / 2}}}, + {offset + num - 1, offset + num, []pb.Entry{{Index: num - 1, Term: num - 1}}}, {offset + num, offset + num + 1, nil}, {offset + num/2, offset + num/2, nil}, diff --git a/raft/raft_paper_test.go b/raft/raft_paper_test.go index 92167386f..d2be61129 100644 --- a/raft/raft_paper_test.go +++ b/raft/raft_paper_test.go @@ -113,7 +113,7 @@ func TestLeaderBcastBeat(t *testing.T) { r.becomeCandidate() r.becomeLeader() for i := 0; i < 10; i++ { - r.appendEntry(pb.Entry{}) + r.appendEntry(pb.Entry{Index: uint64(i) + 1}) } for i := 0; i <= hi; i++ { diff --git a/raft/raft_test.go b/raft/raft_test.go index 1d9003148..4969fbfc8 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -715,7 +715,7 @@ func TestHandleHeartbeat(t *testing.T) { for i, tt := range tests { storage := NewMemoryStorage() - storage.Append([]pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}) + storage.Append([]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}) sm := &raft{ state: StateFollower, HardState: pb.HardState{Term: 2}, @@ -780,7 +780,7 @@ func TestRecvMsgVote(t *testing.T) { } sm.HardState = pb.HardState{Vote: tt.voteFor} sm.raftLog = &raftLog{ - storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 2}, {Term: 2}}}, + storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 2}, {Index: 2, Term: 2}}}, unstable: 3, } @@ -928,7 +928,7 @@ func TestLeaderAppResp(t *testing.T) { // thus the last log term must be 1 to be committed. sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) sm.raftLog = &raftLog{ - storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}, + storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}}, unstable: 3, } sm.becomeCandidate() @@ -980,7 +980,7 @@ func TestBcastBeat(t *testing.T) { sm.becomeCandidate() sm.becomeLeader() for i := 0; i < 10; i++ { - sm.appendEntry(pb.Entry{}) + sm.appendEntry(pb.Entry{Index: uint64(i) + 1}) } // slow follower sm.prs[2].match, sm.prs[2].next = 5, 6 @@ -1034,7 +1034,7 @@ func TestRecvMsgBeat(t *testing.T) { for i, tt := range tests { sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}} + sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}}} sm.Term = 1 sm.state = tt.state switch tt.state { @@ -1344,8 +1344,8 @@ func TestRaftNodes(t *testing.T) { func ents(terms ...uint64) *raft { ents := []pb.Entry{{}} - for _, term := range terms { - ents = append(ents, pb.Entry{Term: term}) + for i, term := range terms { + ents = append(ents, pb.Entry{Index: uint64(i), Term: term}) } sm := &raft{ From 185d37c3339a9bc26767cdf18584b35c1309a8df Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 20:51:04 -0800 Subject: [PATCH 29/52] etcdserver: not load dummy entry from the wal --- etcdserver/server.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index 3b33a30e9..cd18f2e8c 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -270,7 +270,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { if cfg.ShouldDiscover() { log.Printf("etcdserver: warn: ignoring discovery: etcd has already been initialized and has a valid log in %q", cfg.WALDir()) } - index := uint64(1) + var index uint64 snapshot, err := ss.Load() if err != nil && err != snap.ErrNoSnapshot { return nil, err @@ -286,9 +286,9 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { log.Printf("etcdserver: loaded peers from snapshot: %s", cfg.Cluster) } if !cfg.ForceNewCluster { - id, n, s, w = restartNode(cfg, index, snapshot) + id, n, s, w = restartNode(cfg, index+1, snapshot) } else { - id, n, s, w = restartAsStandaloneNode(cfg, index, snapshot) + id, n, s, w = restartAsStandaloneNode(cfg, index+1, snapshot) } default: return nil, fmt.Errorf("unsupported bootstrap config") From bd9e93eeea4f7501004a0a49e7ba3a91ee8fe26c Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 21:10:53 -0800 Subject: [PATCH 30/52] etcdserver: remove finished TODO for raftStorage.Compact --- etcdserver/server.go | 1 - 1 file changed, 1 deletion(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index cd18f2e8c..bed86b91a 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -804,7 +804,6 @@ func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) { if err != nil { log.Panicf("store save should never fail: %v", err) } - // 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) From 1b038da18a108927ebbf104ef996197e23e7f757 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 21:19:30 -0800 Subject: [PATCH 31/52] etcdserver: init snapi when init appliedi --- etcdserver/server.go | 1 + 1 file changed, 1 insertion(+) diff --git a/etcdserver/server.go b/etcdserver/server.go index bed86b91a..7459f86cf 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -418,6 +418,7 @@ func (s *EtcdServer) run() { firsti := rd.CommittedEntries[0].Index if appliedi == 0 { appliedi = firsti - 1 + snapi = appliedi } if firsti > appliedi+1 { log.Panicf("etcdserver: first index of committed entry[%d] should <= appliedi[%d] + 1", firsti, appliedi) From 54e123727127f1b3c34c5eb3ea43601af5f097c4 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 21:22:49 -0800 Subject: [PATCH 32/52] etcdserver: panic when snapshot on raft storage Snapshot on raft storage should always succeed. If there is an error, it must be internal fault and needs stack info to debug. --- etcdserver/server.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index 7459f86cf..37e9822a3 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -811,7 +811,7 @@ func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) { } snap, err := s.raftStorage.Snapshot() if err != nil { - log.Fatalf("etcdserver: snapshot error: %v", err) + log.Panicf("etcdserver: snapshot error: %v", err) } if err := s.storage.SaveSnap(snap); err != nil { log.Fatalf("etcdserver: create snapshot error: %v", err) From 239c8dd479bfb0dc8d74052615ef8d745297f1f8 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 21:47:12 -0800 Subject: [PATCH 33/52] raft: add comment to newLog --- raft/log.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/raft/log.go b/raft/log.go index fd9ccdae1..2591bba77 100644 --- a/raft/log.go +++ b/raft/log.go @@ -44,6 +44,8 @@ type raftLog struct { applied uint64 } +// newLog returns log using the given storage. It recovers the log to the state +// that it just commits and applies the lastest snapshot. func newLog(storage Storage) *raftLog { if storage == nil { log.Panic("storage must not be nil") @@ -68,7 +70,7 @@ func newLog(storage Storage) *raftLog { } func (l *raftLog) String() string { - return fmt.Sprintf("unstable=%d committed=%d applied=%d", l.unstable, l.committed, l.applied) + return fmt.Sprintf("unstable=%d committed=%d applied=%d len(unstableEntries)=%d", l.unstable, l.committed, l.applied, len(l.unstableEnts)) } // maybeAppend returns (0, false) if the entries cannot be appended. Otherwise, From 8670f4012b8c7a139395e4257c83fb1f892f83db Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 22:42:55 -0800 Subject: [PATCH 34/52] raft: remove useless line in raftLog.append --- raft/log.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/raft/log.go b/raft/log.go index 2591bba77..d633a5e67 100644 --- a/raft/log.go +++ b/raft/log.go @@ -105,8 +105,7 @@ func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 { } // Truncate any unstable entries that are being replaced, then // append the new ones. - l.unstableEnts = append(l.unstableEnts[0:1+after-l.unstable], ents...) - l.unstable = min(l.unstable, after+1) + l.unstableEnts = append(l.unstableEnts[:after+1-l.unstable], ents...) return l.lastIndex() } From 7fcaca6d187d4bf6099fa195c1262e95a1bd357c Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 23:08:51 -0800 Subject: [PATCH 35/52] raft: simplify raftLog.lastIndex --- raft/log.go | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/raft/log.go b/raft/log.go index d633a5e67..2e66d48dc 100644 --- a/raft/log.go +++ b/raft/log.go @@ -158,14 +158,7 @@ func (l *raftLog) firstIndex() uint64 { } func (l *raftLog) lastIndex() uint64 { - if len(l.unstableEnts) > 0 { - return l.unstable + uint64(len(l.unstableEnts)) - 1 - } - index, err := l.storage.LastIndex() - if err != nil { - panic(err) // TODO(bdarnell) - } - return index + return l.unstable + uint64(len(l.unstableEnts)) - 1 } func (l *raftLog) commitTo(tocommit uint64) { From 0d200baf72960815c4b7e51dba1fb1adfd011350 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 23:27:57 -0800 Subject: [PATCH 36/52] raft: refine raftLog.term --- raft/log.go | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/raft/log.go b/raft/log.go index 2e66d48dc..bacbbe05b 100644 --- a/raft/log.go +++ b/raft/log.go @@ -195,23 +195,22 @@ func (l *raftLog) lastTerm() uint64 { } func (l *raftLog) term(i uint64) uint64 { - if i >= l.unstable+uint64(len(l.unstableEnts)) { + switch { + case i > l.lastIndex(): return 0 - } - - if i < l.unstable { + case i < l.unstable: t, err := l.storage.Term(i) - if err == nil { + switch err { + case nil: return t - } - if err == ErrCompacted { + case ErrCompacted: return 0 - } else { + default: panic(err) // TODO(bdarnell) } + default: + return l.unstableEnts[i-l.unstable].Term } - - return l.unstableEnts[i-l.unstable].Term } func (l *raftLog) entries(i uint64) []pb.Entry { From 2c06a1d815318b2f2f0daa29c283c11de1fe9673 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 23:37:47 -0800 Subject: [PATCH 37/52] raft: not set applied when restore log from snapshot applied is only updated by application level through Advance. --- raft/log.go | 1 - 1 file changed, 1 deletion(-) diff --git a/raft/log.go b/raft/log.go index bacbbe05b..ab3473a8c 100644 --- a/raft/log.go +++ b/raft/log.go @@ -250,7 +250,6 @@ func (l *raftLog) restore(s pb.Snapshot) { panic(err) // TODO(bdarnell) } l.committed = s.Metadata.Index - l.applied = s.Metadata.Index l.unstable = l.committed + 1 l.unstableEnts = nil } From 1d01c8aa2dbd624844f57380145771dc51ddc55b Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Mon, 24 Nov 2014 23:51:34 -0800 Subject: [PATCH 38/52] raft: remove unused raftLog.at function --- raft/log.go | 10 +--------- raft/log_test.go | 31 ------------------------------- 2 files changed, 1 insertion(+), 40 deletions(-) diff --git a/raft/log.go b/raft/log.go index ab3473a8c..11ea6e417 100644 --- a/raft/log.go +++ b/raft/log.go @@ -123,7 +123,7 @@ func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 { func (l *raftLog) findConflict(from uint64, ents []pb.Entry) uint64 { // TODO(xiangli): validate the index of ents for i, ne := range ents { - if oe := l.at(from + uint64(i)); oe == nil || oe.Term != ne.Term { + if !l.matchTerm(from+uint64(i), ne.Term) { return from + uint64(i) } } @@ -254,14 +254,6 @@ func (l *raftLog) restore(s pb.Snapshot) { l.unstableEnts = nil } -func (l *raftLog) at(i uint64) *pb.Entry { - ents := l.slice(i, i+1) - if len(ents) == 0 { - return nil - } - return &ents[0] -} - // slice returns a slice of log entries from lo through hi-1, inclusive. func (l *raftLog) slice(lo uint64, hi uint64) []pb.Entry { if lo >= hi { diff --git a/raft/log_test.go b/raft/log_test.go index 4bfd37aab..de3046493 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -549,37 +549,6 @@ func TestIsOutOfBounds(t *testing.T) { } } -func TestAt(t *testing.T) { - var i uint64 - offset := uint64(100) - num := uint64(100) - - 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{Index: i, Term: i}) - } - - tests := []struct { - index uint64 - w *pb.Entry - }{ - {offset - 1, nil}, - {offset, nil}, - {offset + num/2, &pb.Entry{Index: num / 2, Term: num / 2}}, - {offset + num - 1, &pb.Entry{Index: num - 1, Term: num - 1}}, - {offset + num, nil}, - } - - for i, tt := range tests { - g := l.at(tt.index) - if !reflect.DeepEqual(g, tt.w) { - t.Errorf("#%d: at = %v, want %v", i, g, tt.w) - } - } -} - func TestTerm(t *testing.T) { var i uint64 offset := uint64(100) From 1e0f87df8c9bac45dcee564ef1bd66aa79ce55e1 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Tue, 25 Nov 2014 00:05:00 -0800 Subject: [PATCH 39/52] raft: stricter checking in raftLog.slice --- raft/log.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/raft/log.go b/raft/log.go index 11ea6e417..fa7bf5deb 100644 --- a/raft/log.go +++ b/raft/log.go @@ -266,13 +266,15 @@ func (l *raftLog) slice(lo uint64, hi uint64) []pb.Entry { if lo < l.unstable { storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable)) if err == ErrCompacted { + // This should never fail because it has been checked before. + log.Panicf("entries[%d:%d) from storage is out of bound", lo, min(hi, l.unstable)) return nil } else if err != nil { panic(err) // TODO(bdarnell) } ents = append(ents, storedEnts...) } - if len(l.unstableEnts) > 0 && hi > l.unstable { + if hi > l.unstable { firstUnstable := max(lo, l.unstable) ents = append(ents, l.unstableEnts[firstUnstable-l.unstable:hi-l.unstable]...) } From 85d0e2f130053b2725fbe77bdfa73c237cd29ee3 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Tue, 25 Nov 2014 00:07:55 -0800 Subject: [PATCH 40/52] raft: remove unused raftLog.isOutOfAppliedBounds --- raft/log.go | 7 ------- 1 file changed, 7 deletions(-) diff --git a/raft/log.go b/raft/log.go index fa7bf5deb..87f7bb6dd 100644 --- a/raft/log.go +++ b/raft/log.go @@ -288,13 +288,6 @@ func (l *raftLog) isOutOfBounds(i uint64) bool { return false } -func (l *raftLog) isOutOfAppliedBounds(i uint64) bool { - if i < l.firstIndex() || i > l.applied { - return true - } - return false -} - func min(a, b uint64) uint64 { if a > b { return b From e17bcd89325d716296686d51106f19370f12afd0 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Tue, 25 Nov 2014 00:10:44 -0800 Subject: [PATCH 41/52] raft: remove wont-fix TODO in ApplyConfChange --- raft/node.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/raft/node.go b/raft/node.go index 605b1cb26..b2bb963cb 100644 --- a/raft/node.go +++ b/raft/node.go @@ -120,8 +120,6 @@ type Node interface { // 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) *pb.ConfState // Stop performs any necessary termination of the Node Stop() From e4661265106ea6afc0923a1185df1548ccc1415c Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Tue, 25 Nov 2014 00:22:26 -0800 Subject: [PATCH 42/52] raft: set snapshot to nil when it is saved --- raft/node.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/raft/node.go b/raft/node.go index b2bb963cb..f934aaf81 100644 --- a/raft/node.go +++ b/raft/node.go @@ -210,6 +210,7 @@ func (n *node) run(r *raft) { var advancec chan struct{} var prevLastUnstablei uint64 var havePrevLastUnstablei bool + var prevSnapi uint64 var rd Ready lead := None @@ -293,6 +294,7 @@ func (n *node) run(r *raft) { prevLastUnstablei = rd.Snapshot.Metadata.Index havePrevLastUnstablei = true } + prevSnapi = rd.Snapshot.Metadata.Index } r.msgs = nil advancec = n.advancec @@ -304,6 +306,9 @@ func (n *node) run(r *raft) { r.raftLog.stableTo(prevLastUnstablei) havePrevLastUnstablei = false } + if r.snapshot != nil && r.snapshot.Metadata.Index == prevSnapi { + r.snapshot = nil + } advancec = nil case <-n.stop: close(n.done) From 8ee1bf31d6868349e2c8cdd56d4b9ee01b1f241a Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Tue, 25 Nov 2014 00:37:21 -0800 Subject: [PATCH 43/52] raft: use IsEmptySnap to check the empty snapshot --- raft/raft.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft/raft.go b/raft/raft.go index fe31c1cd8..4944dd522 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -231,7 +231,7 @@ func (r *raft) sendAppend(to uint64) { if err != nil { panic(err) // TODO(bdarnell) } - if snapshot.Metadata.Term == 0 { + if IsEmptySnap(snapshot) { panic("need non-empty snapshot") } m.Snapshot = snapshot From 8aa89dba3d5e6b2d645c2aa7fcd816404b3767a9 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Tue, 25 Nov 2014 00:52:13 -0800 Subject: [PATCH 44/52] raft: make if checking match the error in storage.Term --- raft/storage.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft/storage.go b/raft/storage.go index 9d62306e4..dcfbb6f5d 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -106,7 +106,7 @@ func (ms *MemoryStorage) Term(i uint64) (uint64, error) { ms.Lock() defer ms.Unlock() offset := ms.snapshot.Metadata.Index - if i < offset || i > offset+uint64(len(ms.ents)) { + if i < offset { return 0, ErrCompacted } return ms.ents[i-offset].Term, nil From a13d5a70ff8c351bd056f9ad34139ab914dbad05 Mon Sep 17 00:00:00 2001 From: Yicheng Qin Date: Tue, 25 Nov 2014 12:37:55 -0800 Subject: [PATCH 45/52] etcdserver: save snapshot before entries --- etcdserver/server.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index 37e9822a3..0253bf329 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -389,16 +389,17 @@ func (s *EtcdServer) run() { } } - s.raftStorage.Append(rd.Entries) - if err := s.storage.Save(rd.HardState, rd.Entries); err != nil { - log.Fatalf("etcdserver: save state and entries error: %v", err) - } if !raft.IsEmptySnap(rd.Snapshot) && rd.Snapshot.Metadata.Index > snapi { if err := s.storage.SaveSnap(rd.Snapshot); err != nil { log.Fatalf("etcdserver: create snapshot error: %v", err) } + s.raftStorage.ApplySnapshot(rd.Snapshot) snapi = rd.Snapshot.Metadata.Index } + if err := s.storage.Save(rd.HardState, rd.Entries); err != nil { + log.Fatalf("etcdserver: save state and entries error: %v", err) + } + s.raftStorage.Append(rd.Entries) s.sendhub.Send(rd.Messages) if !raft.IsEmptySnap(rd.Snapshot) { From 74d8c7f457535bd2f4e592dd39b1118f60c797de Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 25 Nov 2014 14:38:18 -0800 Subject: [PATCH 46/52] etcdserver: cleanup main loop --- etcdserver/server.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index 0253bf329..13ceb5e39 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -389,6 +389,7 @@ func (s *EtcdServer) run() { } } + // apply snapshot to storage if it is more updated than current snapi if !raft.IsEmptySnap(rd.Snapshot) && rd.Snapshot.Metadata.Index > snapi { if err := s.storage.SaveSnap(rd.Snapshot); err != nil { log.Fatalf("etcdserver: create snapshot error: %v", err) @@ -396,15 +397,17 @@ func (s *EtcdServer) run() { s.raftStorage.ApplySnapshot(rd.Snapshot) snapi = rd.Snapshot.Metadata.Index } + if err := s.storage.Save(rd.HardState, rd.Entries); err != nil { log.Fatalf("etcdserver: save state and entries error: %v", err) } s.raftStorage.Append(rd.Entries) + s.sendhub.Send(rd.Messages) - if !raft.IsEmptySnap(rd.Snapshot) { - // recover from snapshot if it is more updated than current applied - if rd.Snapshot.Metadata.Index > appliedi { + // recover from snapshot if it is more updated than current applied + if !raft.IsEmptySnap(rd.Snapshot) && rd.Snapshot.Metadata.Index > appliedi { + { if err := s.store.Recovery(rd.Snapshot.Data); err != nil { log.Panicf("recovery store error: %v", err) } From 01cbcce8ba9c5d5d7b023eb7a25f042c1aba9779 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 25 Nov 2014 14:53:49 -0800 Subject: [PATCH 47/52] etcdserver: do not applySnapshot twice --- etcdserver/server.go | 1 - raft/log.go | 2 ++ 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index 13ceb5e39..03d99ca7d 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -394,7 +394,6 @@ func (s *EtcdServer) run() { if err := s.storage.SaveSnap(rd.Snapshot); err != nil { log.Fatalf("etcdserver: create snapshot error: %v", err) } - s.raftStorage.ApplySnapshot(rd.Snapshot) snapi = rd.Snapshot.Metadata.Index } diff --git a/raft/log.go b/raft/log.go index 87f7bb6dd..a3bf9a3d3 100644 --- a/raft/log.go +++ b/raft/log.go @@ -245,6 +245,8 @@ func (l *raftLog) maybeCommit(maxIndex, term uint64) bool { } func (l *raftLog) restore(s pb.Snapshot) { + // TODO: rethink restore logic. + // This breaks the rule that raft never modifies storage. err := l.storage.ApplySnapshot(s) if err != nil { panic(err) // TODO(bdarnell) From 9df0e7715d4b67e043c4c157b3347228b6e4e386 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 25 Nov 2014 15:13:07 -0800 Subject: [PATCH 48/52] raft: do not panic on out of date compaction --- etcdserver/server.go | 11 ++++++++++- raft/log_test.go | 8 +++++++- raft/storage.go | 12 +++++++----- 3 files changed, 24 insertions(+), 7 deletions(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index 03d99ca7d..d7d8e3fab 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -808,7 +808,16 @@ func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) { if err != nil { log.Panicf("store save should never fail: %v", err) } - s.raftStorage.Compact(snapi, &raftpb.ConfState{Nodes: snapnodes}, d) + err = s.raftStorage.Compact(snapi, &raftpb.ConfState{Nodes: snapnodes}, d) + if err != nil { + // the snapshot was done asynchronously with the progress of raft. + // raft might have already got a newer snapshot and called compact. + if err == raft.ErrCompacted { + return + } + log.Panicf("etcdserver: unexpected compaction error %v", err) + } + if err := s.storage.Cut(); err != nil { log.Panicf("rotate wal file should never fail: %v", err) } diff --git a/raft/log_test.go b/raft/log_test.go index de3046493..80a57428a 100644 --- a/raft/log_test.go +++ b/raft/log_test.go @@ -488,7 +488,13 @@ func TestCompaction(t *testing.T) { raftLog.appliedTo(raftLog.committed) for j := 0; j < len(tt.compact); j++ { - storage.Compact(tt.compact[j], nil, nil) + err := storage.Compact(tt.compact[j], nil, nil) + if err != nil { + if tt.wallow { + t.Errorf("#%d.%d allow = %t, want %t", i, j, false, tt.wallow) + } + continue + } if len(raftLog.allEntries()) != tt.wleft[j] { t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.allEntries()), tt.wleft[j]) } diff --git a/raft/storage.go b/raft/storage.go index dcfbb6f5d..8516bad03 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -18,13 +18,13 @@ package raft import ( "errors" - "fmt" + "log" "sync" pb "github.com/coreos/etcd/raft/raftpb" ) -// ErrCompacted is returned by Storage.Entries when a requested +// ErrCompacted is returned by Storage.Entries/Compact when a requested // index is unavailable because it predates the last snapshot. var ErrCompacted = errors.New("requested index is unavailable due to compaction") @@ -154,9 +154,11 @@ 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)) + if i <= offset { + return ErrCompacted + } + if i > offset+uint64(len(ms.ents))-1 { + log.Panicf("compact %d out of bound lastindex(%d)", i, offset+uint64(len(ms.ents))-1) } i -= offset ents := make([]pb.Entry, 1, 1+uint64(len(ms.ents))-i) From 9bd1786fe4e3d04ffa88c8cc4f4d9a30639c5be4 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 25 Nov 2014 15:18:40 -0800 Subject: [PATCH 49/52] raft: memory storage does not append out of date entries --- raft/storage.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/raft/storage.go b/raft/storage.go index 8516bad03..4d7bd7842 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -182,6 +182,10 @@ func (ms *MemoryStorage) Append(entries []pb.Entry) { return } offset := entries[0].Index - ms.snapshot.Metadata.Index + // do not append out of date entries + if offset < 0 { + return + } if uint64(len(ms.ents)) >= offset { ms.ents = ms.ents[:offset] } From 8de98d4903582585b3d3d3d104df943b438b0424 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 25 Nov 2014 16:21:50 -0800 Subject: [PATCH 50/52] raft: clean up --- raft/log.go | 18 ++---------------- raft/raft.go | 11 +---------- raft/storage.go | 2 +- raft/util.go | 21 +++++++++++++++++++++ 4 files changed, 25 insertions(+), 27 deletions(-) diff --git a/raft/log.go b/raft/log.go index a3bf9a3d3..ccd6122d0 100644 --- a/raft/log.go +++ b/raft/log.go @@ -83,7 +83,7 @@ func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry switch { case ci == 0: case ci <= l.committed: - log.Panicf("conflict(%d) with committed entry [committed(%d)]", ci, l.committed) + log.Panicf("entry %d conflict with committed entry [committed(%d)]", ci, l.committed) default: l.append(ci-1, ents[ci-from:]...) } @@ -95,7 +95,7 @@ func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 { if after < l.committed { - log.Panicf("after(%d) out of range [committed(%d)]", after, l.committed) + log.Panicf("after(%d) is out of range [committed(%d)]", after, l.committed) } if after < l.unstable { // The log is being truncated to before our current unstable @@ -289,17 +289,3 @@ func (l *raftLog) isOutOfBounds(i uint64) bool { } return false } - -func min(a, b uint64) uint64 { - if a > b { - return b - } - return a -} - -func max(a, b uint64) uint64 { - if a > b { - return a - } - return b -} diff --git a/raft/raft.go b/raft/raft.go index 3597b74fe..f64e14b60 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -100,13 +100,6 @@ func (pr *progress) String() string { return fmt.Sprintf("n=%d m=%d", pr.next, pr.match) } -// uint64Slice implements sort interface -type uint64Slice []uint64 - -func (p uint64Slice) Len() int { return len(p) } -func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] } -func (p uint64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } - type raft struct { pb.HardState @@ -452,9 +445,7 @@ func (r *raft) handleSnapshot(m pb.Message) { } } -func (r *raft) resetPendingConf() { - r.pendingConf = false -} +func (r *raft) resetPendingConf() { r.pendingConf = false } func (r *raft) addNode(id uint64) { r.setProgress(id, 0, r.raftLog.lastIndex()+1) diff --git a/raft/storage.go b/raft/storage.go index 4d7bd7842..14d3fa3ec 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -158,7 +158,7 @@ func (ms *MemoryStorage) Compact(i uint64, cs *pb.ConfState, data []byte) error return ErrCompacted } if i > offset+uint64(len(ms.ents))-1 { - log.Panicf("compact %d out of bound lastindex(%d)", i, offset+uint64(len(ms.ents))-1) + log.Panicf("compact %d is out of bound lastindex(%d)", i, offset+uint64(len(ms.ents))-1) } i -= offset ents := make([]pb.Entry, 1, 1+uint64(len(ms.ents))-i) diff --git a/raft/util.go b/raft/util.go index d97707796..95af25f3a 100644 --- a/raft/util.go +++ b/raft/util.go @@ -23,6 +23,27 @@ import ( pb "github.com/coreos/etcd/raft/raftpb" ) +// uint64Slice implements sort interface +type uint64Slice []uint64 + +func (p uint64Slice) Len() int { return len(p) } +func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] } +func (p uint64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } + +func min(a, b uint64) uint64 { + if a > b { + return b + } + return a +} + +func max(a, b uint64) uint64 { + if a > b { + return a + } + return b +} + // DescribeMessage returns a concise human-readable description of a // Message for debugging. func DescribeMessage(m pb.Message) string { From e23f9e76d11c8032173b76a7d064895a2b559b47 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 26 Nov 2014 10:59:13 -0800 Subject: [PATCH 51/52] raft: do not applysnapshot in raft --- etcdserver/server.go | 1 + raft/log.go | 26 ++++++++++++++++++++------ raft/node.go | 8 ++++---- raft/raft.go | 6 +----- 4 files changed, 26 insertions(+), 15 deletions(-) diff --git a/etcdserver/server.go b/etcdserver/server.go index d7d8e3fab..a36ae47ee 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -394,6 +394,7 @@ func (s *EtcdServer) run() { if err := s.storage.SaveSnap(rd.Snapshot); err != nil { log.Fatalf("etcdserver: create snapshot error: %v", err) } + s.raftStorage.ApplySnapshot(rd.Snapshot) snapi = rd.Snapshot.Metadata.Index } diff --git a/raft/log.go b/raft/log.go index ccd6122d0..e6f1ff989 100644 --- a/raft/log.go +++ b/raft/log.go @@ -26,6 +26,9 @@ import ( type raftLog struct { // storage contains all stable entries since the last snapshot. storage Storage + + // the incoming unstable snapshot, if any. + unstableSnapshot *pb.Snapshot // unstableEnts contains all entries that have not yet been written // to storage. unstableEnts []pb.Entry @@ -149,7 +152,17 @@ func (l *raftLog) nextEnts() (ents []pb.Entry) { return nil } +func (l *raftLog) snapshot() (pb.Snapshot, error) { + if l.unstableSnapshot != nil { + return *l.unstableSnapshot, nil + } + return l.storage.Snapshot() +} + func (l *raftLog) firstIndex() uint64 { + if l.unstableSnapshot != nil { + return l.unstableSnapshot.Metadata.Index + 1 + } index, err := l.storage.FirstIndex() if err != nil { panic(err) // TODO(bdarnell) @@ -199,6 +212,12 @@ func (l *raftLog) term(i uint64) uint64 { case i > l.lastIndex(): return 0 case i < l.unstable: + if snap := l.unstableSnapshot; snap != nil { + if i == snap.Metadata.Index { + return snap.Metadata.Term + } + return 0 + } t, err := l.storage.Term(i) switch err { case nil: @@ -245,15 +264,10 @@ func (l *raftLog) maybeCommit(maxIndex, term uint64) bool { } func (l *raftLog) restore(s pb.Snapshot) { - // TODO: rethink restore logic. - // This breaks the rule that raft never modifies storage. - err := l.storage.ApplySnapshot(s) - if err != nil { - panic(err) // TODO(bdarnell) - } l.committed = s.Metadata.Index l.unstable = l.committed + 1 l.unstableEnts = nil + l.unstableSnapshot = &s } // slice returns a slice of log entries from lo through hi-1, inclusive. diff --git a/raft/node.go b/raft/node.go index f22477ae5..827f9a9b4 100644 --- a/raft/node.go +++ b/raft/node.go @@ -306,8 +306,8 @@ func (n *node) run(r *raft) { r.raftLog.stableTo(prevLastUnstablei) havePrevLastUnstablei = false } - if r.snapshot != nil && r.snapshot.Metadata.Index == prevSnapi { - r.snapshot = nil + if r.raftLog.unstableSnapshot != nil && r.raftLog.unstableSnapshot.Metadata.Index == prevSnapi { + r.raftLog.unstableSnapshot = nil } advancec = nil case <-n.stop: @@ -405,8 +405,8 @@ func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { if !isHardStateEqual(r.HardState, prevHardSt) { rd.HardState = r.HardState } - if r.snapshot != nil { - rd.Snapshot = *r.snapshot + if r.raftLog.unstableSnapshot != nil { + rd.Snapshot = *r.raftLog.unstableSnapshot } return rd } diff --git a/raft/raft.go b/raft/raft.go index f64e14b60..92275561b 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -116,9 +116,6 @@ type raft struct { msgs []pb.Message - // the incoming snapshot, if any. - snapshot *pb.Snapshot - // the leader id lead uint64 @@ -222,7 +219,7 @@ func (r *raft) sendAppend(to uint64) { m.To = to if r.needSnapshot(pr.next) { m.Type = pb.MsgSnap - snapshot, err := r.raftLog.storage.Snapshot() + snapshot, err := r.raftLog.snapshot() if err != nil { panic(err) // TODO(bdarnell) } @@ -438,7 +435,6 @@ func (r *raft) handleHeartbeat(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}) From 732cfa1ad60896077ca999fc22a1113d4d0477cd Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 26 Nov 2014 11:28:51 -0800 Subject: [PATCH 52/52] raft: remove the applysnap from Storage interface --- raft/storage.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/raft/storage.go b/raft/storage.go index 14d3fa3ec..7c61d599d 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -52,9 +52,6 @@ type Storage interface { FirstIndex() (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 @@ -133,7 +130,8 @@ func (ms *MemoryStorage) Snapshot() (pb.Snapshot, error) { return ms.snapshot, nil } -// ApplySnapshot implements the Storage interface. +// ApplySnapshot overwrites the contents of this Storage object with +// those of the given snapshot. func (ms *MemoryStorage) ApplySnapshot(snap pb.Snapshot) error { ms.Lock() defer ms.Unlock()