diff --git a/etcdctl/command/backup_command.go b/etcdctl/command/backup_command.go index e0a926a7b..a970f57ab 100644 --- a/etcdctl/command/backup_command.go +++ b/etcdctl/command/backup_command.go @@ -60,7 +60,7 @@ func handleBackup(c *cli.Context) { } var index uint64 if snapshot != nil { - index = snapshot.Index + index = snapshot.Metadata.Index newss := snap.New(destSnap) if err := newss.SaveSnap(*snapshot); err != nil { log.Fatal(err) diff --git a/etcdserver/force_cluster.go b/etcdserver/force_cluster.go index 568f28674..10b03ff5a 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) @@ -57,8 +57,14 @@ 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() + if snapshot != nil { + s.ApplySnapshot(*snapshot) + } + s.SetHardState(st) + s.Append(ents) + n := raft.RestartNode(uint64(id), 10, 1, s) + return id, n, s, w } // getIDs returns an ordered set of IDs included in the given snapshot and @@ -69,7 +75,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 05f440fe7..0253bf329 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -165,8 +165,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 @@ -211,6 +212,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 walVersion := wal.DetectVersion(cfg.DataDir) if walVersion == wal.WALUnknown { @@ -243,7 +245,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { cfg.Cluster.SetID(existingCluster.id) cfg.Cluster.SetStore(st) cfg.Print() - 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 @@ -263,7 +265,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { } cfg.Cluster.SetStore(st) cfg.PrintWithInitial() - 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()) @@ -274,9 +276,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() @@ -284,9 +286,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+1, snapshot) } else { - id, n, w = restartAsStandaloneNode(cfg, index, snapshot) + id, n, s, w = restartAsStandaloneNode(cfg, index+1, snapshot) } default: return nil, fmt.Errorf("unsupported bootstrap config") @@ -298,12 +300,13 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { } lstats := stats.NewLeaderStats(id.String()) - 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 @@ -314,8 +317,8 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) { SyncTicker: time.Tick(500 * time.Millisecond), snapCount: cfg.SnapCount, } - s.sendhub = newSendHub(cfg.Transport, cfg.Cluster, s, sstats, lstats) - return s, nil + srv.sendhub = newSendHub(cfg.Transport, cfg.Cluster, srv, sstats, lstats) + return srv, nil } // Start prepares and starts server in a new goroutine. It is no longer safe to @@ -386,21 +389,28 @@ func (s *EtcdServer) run() { } } + 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) } - if err := s.storage.SaveSnap(rd.Snapshot); err != nil { - log.Fatalf("etcdserver: create snapshot error: %v", err) - } + s.raftStorage.Append(rd.Entries) s.sendhub.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 @@ -409,6 +419,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) @@ -424,9 +435,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 @@ -798,10 +806,17 @@ 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) + 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.Panicf("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, @@ -875,7 +890,7 @@ func getClusterFromPeers(urls []string, logerr bool) (*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( @@ -897,7 +912,8 @@ func startNode(cfg *ServerConfig, ids []types.ID) (id types.ID, n raft.Node, w * } id = member.ID log.Printf("etcdserver: start member %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 } @@ -915,13 +931,19 @@ 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() + if snapshot != nil { + s.ApplySnapshot(*snapshot) + } + s.SetHardState(st) + s.Append(ents) + n := raft.RestartNode(uint64(id), 10, 1, 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 e2daa4672..1c12209d5 100644 --- a/etcdserver/server_test.go +++ b/etcdserver/server_test.go @@ -560,24 +560,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, - sendhub: &fakeSender{ss}, - storage: &storageRecorder{}, - Ticker: tk.C, - Cluster: cl, + node: n, + raftStorage: s, + store: st, + sendhub: &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", @@ -628,7 +634,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 @@ -636,12 +643,13 @@ func TestDoProposal(t *testing.T) { cl := newCluster("abc") cl.SetStore(store.New()) srv := &EtcdServer{ - node: n, - store: st, - sendhub: &nopSender{}, - storage: &storageRecorder{}, - Ticker: tk, - Cluster: cl, + node: n, + raftStorage: s, + store: st, + sendhub: &nopSender{}, + storage: &storageRecorder{}, + Ticker: tk, + Cluster: cl, } srv.start() resp, err := srv.Do(ctx, tt) @@ -664,14 +672,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{}) @@ -712,7 +722,8 @@ 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 @@ -721,12 +732,13 @@ func TestDoProposalStopped(t *testing.T) { cl.SetStore(store.New()) srv := &EtcdServer{ // TODO: use fake node for better testability - node: n, - store: st, - sendhub: &nopSender{}, - storage: &storageRecorder{}, - Ticker: tk, - Cluster: cl, + node: n, + raftStorage: s, + store: st, + sendhub: &nopSender{}, + storage: &storageRecorder{}, + Ticker: tk, + Cluster: cl, } srv.start() @@ -832,11 +844,12 @@ func TestSyncTrigger(t *testing.T) { } st := make(chan time.Time, 1) srv := &EtcdServer{ - node: n, - store: &storeRecorder{}, - sendhub: &nopSender{}, - storage: &storageRecorder{}, - SyncTicker: st, + node: n, + raftStorage: raft.NewMemoryStorage(), + store: &storeRecorder{}, + sendhub: &nopSender{}, + storage: &storageRecorder{}, + SyncTicker: st, } srv.start() // trigger the server to become a leader and accept sync requests @@ -866,17 +879,45 @@ 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() + + // 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{} 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(1, []uint64{1}) gaction := st.Action() if len(gaction) != 1 { t.Fatalf("len(action) = %d, want 1", len(gaction)) @@ -886,19 +927,24 @@ 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 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) @@ -906,26 +952,30 @@ func TestTriggerSnap(t *testing.T) { p := &storageRecorder{} cl := newCluster("abc") cl.SetStore(store.New()) - s := &EtcdServer{ - store: st, - sendhub: &nopSender{}, - storage: p, - node: n, - snapCount: 10, - Cluster: cl, + srv := &EtcdServer{ + store: st, + sendhub: &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() + // wait for saving nop time.Sleep(time.Millisecond) - s.Stop() + 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 := 2 + int(s.snapCount) + wcnt := 2 + int(srv.snapCount) if len(gaction) != wcnt { t.Fatalf("len(action) = %d, want %d", len(gaction), wcnt) } @@ -943,15 +993,16 @@ func TestRecvSnapshot(t *testing.T) { cl := newCluster("abc") cl.SetStore(store.New()) s := &EtcdServer{ - store: st, - sendhub: &nopSender{}, - storage: p, - node: n, - Cluster: cl, + store: st, + sendhub: &nopSender{}, + storage: p, + node: n, + raftStorage: raft.NewMemoryStorage(), + Cluster: cl, } 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() @@ -960,7 +1011,7 @@ func TestRecvSnapshot(t *testing.T) { if g := st.Action(); !reflect.DeepEqual(g, wactions) { t.Errorf("store action = %v, want %v", g, wactions) } - wactions = []action{action{name: "Save"}, action{name: "SaveSnap"}} + wactions = []action{action{name: "SaveSnap"}, action{name: "Save"}} if g := p.Action(); !reflect.DeepEqual(g, wactions) { t.Errorf("storage action = %v, want %v", g, wactions) } @@ -975,20 +1026,21 @@ func TestRecvSlowSnapshot(t *testing.T) { cl := newCluster("abc") cl.SetStore(store.New()) s := &EtcdServer{ - store: st, - sendhub: &nopSender{}, - storage: &storageRecorder{}, - node: n, - Cluster: cl, + store: st, + sendhub: &nopSender{}, + storage: &storageRecorder{}, + node: n, + raftStorage: raft.NewMemoryStorage(), + Cluster: cl, } 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() @@ -1005,18 +1057,20 @@ func TestApplySnapshotAndCommittedEntries(t *testing.T) { st := &storeRecorder{} cl := newCluster("abc") cl.SetStore(store.New()) + storage := raft.NewMemoryStorage() s := &EtcdServer{ - store: st, - sendhub: &nopSender{}, - storage: &storageRecorder{}, - node: n, - Cluster: cl, + store: st, + sendhub: &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)}, }, @@ -1049,11 +1103,12 @@ func TestAddMember(t *testing.T) { cl := newTestCluster(nil) cl.SetStore(store.New()) s := &EtcdServer{ - node: n, - store: &storeRecorder{}, - sendhub: &nopSender{}, - storage: &storageRecorder{}, - Cluster: cl, + node: n, + raftStorage: raft.NewMemoryStorage(), + store: &storeRecorder{}, + sendhub: &nopSender{}, + storage: &storageRecorder{}, + Cluster: cl, } s.start() m := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"foo"}}} @@ -1086,11 +1141,12 @@ func TestRemoveMember(t *testing.T) { cl.SetStore(store.New()) cl.AddMember(&Member{ID: 1234}) s := &EtcdServer{ - node: n, - store: &storeRecorder{}, - sendhub: &nopSender{}, - storage: &storageRecorder{}, - Cluster: cl, + node: n, + raftStorage: raft.NewMemoryStorage(), + store: &storeRecorder{}, + sendhub: &nopSender{}, + storage: &storageRecorder{}, + Cluster: cl, } s.start() err := s.RemoveMember(context.TODO(), 1234) @@ -1122,11 +1178,12 @@ func TestUpdateMember(t *testing.T) { cl.SetStore(store.New()) cl.AddMember(&Member{ID: 1234}) s := &EtcdServer{ - node: n, - store: &storeRecorder{}, - sendhub: &nopSender{}, - storage: &storageRecorder{}, - Cluster: cl, + node: n, + raftStorage: raft.NewMemoryStorage(), + store: &storeRecorder{}, + sendhub: &nopSender{}, + storage: &storageRecorder{}, + Cluster: cl, } s.start() wm := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"http://127.0.0.1:1"}}} @@ -1488,12 +1545,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 @@ -1519,8 +1576,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"}) @@ -1580,8 +1638,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 559191a40..ccc5b0e7a 100644 --- a/migrate/etcd4.go +++ b/migrate/etcd4.go @@ -86,7 +86,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 8a457fac9..9d0be6273 100644 --- a/migrate/snapshot.go +++ b/migrate/snapshot.go @@ -178,10 +178,14 @@ func (s *Snapshot4) Snapshot5() *raftpb.Snapshot { } snap5 := raftpb.Snapshot{ - Data: newState, - Index: s.LastIndex, - Term: s.LastTerm, - Nodes: nodeList, + Data: newState, + Metadata: raftpb.SnapshotMetadata{ + Index: s.LastIndex, + Term: s.LastTerm, + ConfState: raftpb.ConfState{ + Nodes: nodeList, + }, + }, } return &snap5 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 6ece348b7..87f7bb6dd 100644 --- a/raft/log.go +++ b/raft/log.go @@ -24,33 +24,53 @@ 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 } -func newLog() *raftLog { - return &raftLog{ - ents: make([]pb.Entry, 1), - unstable: 0, - committed: 0, - applied: 0, +// 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") } -} + 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 -func (l *raftLog) load(ents []pb.Entry) { - if l.offset != ents[0].Index { - panic("entries loaded don't match offset index") - } - l.ents = ents - l.unstable = l.offset + uint64(len(ents)) + return log } 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 len(unstableEntries)=%d", l.unstable, l.committed, l.applied, len(l.unstableEnts)) } // maybeAppend returns (0, false) if the entries cannot be appended. Otherwise, @@ -63,7 +83,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:]...) } @@ -74,8 +94,18 @@ 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...) - l.unstable = min(l.unstable, after+1) + if 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 + // 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[:after+1-l.unstable], ents...) return l.lastIndex() } @@ -93,39 +123,49 @@ 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) } } 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) - return cpy + // copy unstable entries to an empty slice + return append([]pb.Entry{}, l.unstableEnts...) } // nextEnts returns all the available entries for execution. // 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 } +func (l *raftLog) firstIndex() uint64 { + index, err := l.storage.FirstIndex() + if err != nil { + panic(err) // TODO(bdarnell) + } + return index +} + +func (l *raftLog) lastIndex() uint64 { + return l.unstable + uint64(len(l.unstableEnts)) - 1 +} + 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 } @@ -136,36 +176,52 @@ 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)]", + i, l.unstable, len(l.unstableEnts)) + } + 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 { - return e.Term + switch { + case i > l.lastIndex(): + return 0 + case i < l.unstable: + t, err := l.storage.Term(i) + switch err { + case nil: + return t + case ErrCompacted: + return 0 + default: + panic(err) // TODO(bdarnell) + } + default: + return l.unstableEnts[i-l.unstable].Term } - return 0 } 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 { - panic("cannot return the first entry in log") - } return l.slice(i, l.lastIndex()+1) } +// allEntries returns all entries in the log. +func (l *raftLog) allEntries() []pb.Entry { + return l.entries(l.firstIndex()) +} + // 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 @@ -177,10 +233,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 { @@ -191,43 +244,14 @@ 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 [%d:%d]", i, l.offset, l.applied)) - } - l.ents = l.slice(i, l.lastIndex()+1) - l.unstable = max(i+1, l.unstable) - l.offset = i - return uint64(len(l.ents)) -} - -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.ents = []pb.Entry{{Term: s.Term}} - l.unstable = s.Index + 1 - l.committed = s.Index - l.offset = s.Index - l.snapshot = s -} - -func (l *raftLog) at(i uint64) *pb.Entry { - if l.isOutOfBounds(i) { - return nil + err := l.storage.ApplySnapshot(s) + if err != nil { + panic(err) // TODO(bdarnell) } - return &l.ents[i-l.offset] + l.committed = s.Metadata.Index + l.unstable = l.committed + 1 + l.unstableEnts = nil } // slice returns a slice of log entries from lo through hi-1, inclusive. @@ -238,18 +262,27 @@ 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.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 hi > l.unstable { + firstUnstable := max(lo, l.unstable) + 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() { - return true - } - return false -} - -func (l *raftLog) isOutOfAppliedBounds(i uint64) bool { - if i < l.offset || i > l.applied { + if i < l.firstIndex() || i > l.lastIndex() { return true } return false diff --git a/raft/log_test.go b/raft/log_test.go index 6ce4f5db9..de3046493 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,22 +34,22 @@ 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 { - raftLog := newLog() + raftLog := newLog(NewMemoryStorage()) raftLog.append(raftLog.lastIndex(), previousEnts...) gconflict := raftLog.findConflict(tt.from, tt.ents) @@ -60,8 +60,8 @@ func TestFindConflict(t *testing.T) { } func TestIsUpToDate(t *testing.T) { - previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}} - raftLog := newLog() + 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 { lastIndex uint64 @@ -91,8 +91,7 @@ func TestIsUpToDate(t *testing.T) { } func TestAppend(t *testing.T) { - previousEnts := []pb.Entry{{Term: 1}, {Term: 2}} - previousUnstable := uint64(3) + previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}} tests := []struct { after uint64 ents []pb.Entry @@ -104,38 +103,39 @@ 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, }, } 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) @@ -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,42 +206,42 @@ 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, }, } for i, tt := range tests { - raftLog := newLog() + raftLog := newLog(NewMemoryStorage()) raftLog.append(raftLog.lastIndex(), previousEnts...) raftLog.committed = commit func() { @@ -278,44 +278,54 @@ 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) + storage.Compact(offset, nil, nil) 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() - 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) } @@ -327,9 +337,10 @@ 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: 3}, {Term: 1, Index: 4}, {Term: 1, Index: 5}, {Term: 1, Index: 6}, @@ -338,15 +349,16 @@ func TestNextEnts(t *testing.T) { applied uint64 wents []pb.Entry }{ - {0, ents[1:3]}, - {3, ents[1:3]}, - {4, ents[2:3]}, + {0, ents[:2]}, + {3, ents[:2]}, + {4, ents[1:2]}, {5, nil}, } for i, tt := range tests { - raftLog := newLog() - raftLog.restore(snap) - raftLog.load(ents) + storage := NewMemoryStorage() + storage.ApplySnapshot(snap) + raftLog := newLog(storage) + raftLog.append(snap.Metadata.Index, ents...) raftLog.maybeCommit(5, 1) raftLog.appliedTo(tt.applied) @@ -357,31 +369,37 @@ 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}, - {0, append([]pb.Entry{{}}, previousEnts...), 3}, + {3, nil}, + {1, previousEnts}, } for i, tt := range tests { - raftLog := newLog() - raftLog.append(0, previousEnts...) - raftLog.unstable = tt.unstable - ents := raftLog.unstableEnts() + // append stable entries to storage + storage := NewMemoryStorage() + 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) } 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) } } } @@ -407,7 +425,7 @@ func TestCommitTo(t *testing.T) { } } }() - raftLog := newLog() + raftLog := newLog(NewMemoryStorage()) raftLog.append(0, previousEnts...) raftLog.committed = commit raftLog.commitTo(tt.commit) @@ -423,12 +441,12 @@ func TestStableTo(t *testing.T) { stable uint64 wunstable uint64 }{ - {0, 1}, {1, 2}, {2, 3}, } for i, tt := range tests { - raftLog := newLog() + raftLog := newLog(NewMemoryStorage()) + raftLog.append(0, []pb.Entry{{}, {}}...) raftLog.stableTo(tt.stable) if raftLog.unstable != tt.wunstable { t.Errorf("#%d: unstable = %d, want %d", i, raftLog.unstable, tt.wunstable) @@ -436,21 +454,19 @@ func TestStableTo(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 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{701, 501, 201, 101}, 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{701, -1}, false}, - {0, 1000, []uint64{1}, []int{-1}, false}, + {1000, []uint64{300, 299}, []int{700, -1}, false}, } for i, tt := range tests { @@ -458,22 +474,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() - for i := uint64(0); i < tt.lastIndex; i++ { - raftLog.append(uint64(i), pb.Entry{}) + storage := NewMemoryStorage() + for i := uint64(1); i <= tt.lastIndex; i++ { + storage.Append([]pb.Entry{{Index: i}}) } - raftLog.maybeCommit(tt.applied, 0) + raftLog := newLog(storage) + raftLog.maybeCommit(tt.lastIndex, 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]) + 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]) } } }() @@ -481,22 +498,18 @@ func TestCompaction(t *testing.T) { } func TestLogRestore(t *testing.T) { - var i uint64 - raftLog := newLog() - 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.ents) != 1 { - t.Errorf("len = %d, want 0", len(raftLog.ents)) + if len(raftLog.allEntries()) != 0 { + t.Errorf("len = %d, want 0", len(raftLog.allEntries())) } - if raftLog.offset != index { - t.Errorf("offset = %d, want %d", raftLog.offset, index) + if raftLog.firstIndex() != index+1 { + t.Errorf("firstIndex = %d, want %d", raftLog.firstIndex(), index+1) } if raftLog.committed != index { t.Errorf("comitted = %d, want %d", raftLog.committed, index) @@ -512,17 +525,20 @@ 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)} + storage := NewMemoryStorage() + storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}}) + l := newLog(storage) + l.append(offset, make([]pb.Entry, num)...) tests := []struct { index uint64 w bool }{ {offset - 1, true}, - {offset, false}, + {offset, true}, {offset + num/2, false}, - {offset + num - 1, false}, - {offset + num, true}, + {offset + num, false}, + {offset + num + 1, true}, } for i, tt := range tests { @@ -533,31 +549,33 @@ func TestIsOutOfBounds(t *testing.T) { } } -func TestAt(t *testing.T) { +func TestTerm(t *testing.T) { var i uint64 offset := uint64(100) num := uint64(100) - l := &raftLog{offset: offset} - for i = 0; i < num; i++ { - l.ents = append(l.ents, pb.Entry{Term: i}) + 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 + w uint64 }{ - {offset - 1, nil}, - {offset, &pb.Entry{Term: 0}}, - {offset + num/2, &pb.Entry{Term: num / 2}}, - {offset + num - 1, &pb.Entry{Term: num - 1}}, - {offset + num, nil}, + {offset - 1, 0}, + {offset, 0}, + {offset + num/2, num / 2}, + {offset + num - 1, num - 1}, + {offset + num, 0}, } 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) + term := l.term(tt.index) + if !reflect.DeepEqual(term, tt.w) { + t.Errorf("#%d: at = %d, want %d", i, term, tt.w) } } } @@ -567,9 +585,11 @@ func TestSlice(t *testing.T) { offset := uint64(100) num := uint64(100) - l := &raftLog{offset: offset} - for i = 0; i < num; i++ { - l.ents = append(l.ents, pb.Entry{Term: i}) + 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 { @@ -578,9 +598,9 @@ func TestSlice(t *testing.T) { w []pb.Entry }{ {offset - 1, offset + 1, nil}, - {offset, offset + 1, []pb.Entry{{Term: 0}}}, - {offset + num/2, offset + num/2 + 1, []pb.Entry{{Term: num / 2}}}, - {offset + num - 1, offset + num, []pb.Entry{{Term: num - 1}}}, + {offset, offset + 1, nil}, + {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/node.go b/raft/node.go index e870106aa..f22477ae5 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,12 @@ type Node interface { // It prepares the node to return the next available Ready. Advance() // ApplyConfChange applies config change to the local node. - // TODO: reject existing node when add node - // TODO: reject non-existant node when remove node - ApplyConfChange(cc pb.ConfChange) + // Returns an opaque ConfState protobuf which must be recorded + // in snapshots. Will never return nil; it returns a pointer only + // to match MemoryStorage.Compact. + 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 { @@ -144,9 +133,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 appends a ConfChangeAddNode entry for each given peer to the initial 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} @@ -157,56 +146,49 @@ func StartNode(id uint64, peers []Peer, election, heartbeat int) Node { 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. -func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, ents []pb.Entry) 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) - if snapshot != nil { - r.restore(*snapshot) - r.raftLog.appliedTo(snapshot.Index) - } - if !isHardStateEqual(st, emptyState) { - r.loadState(st) - } - if len(ents) != 0 { - r.loadEnts(ents) - } + r := newRaft(id, nil, election, heartbeat, storage) + 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{}), } } @@ -228,18 +210,18 @@ func (n *node) run(r *raft) { var advancec chan struct{} var prevLastUnstablei uint64 var havePrevLastUnstablei bool + var prevSnapi uint64 var rd Ready 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 { @@ -271,11 +253,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 { @@ -288,6 +272,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: @@ -302,11 +290,11 @@ 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 } + prevSnapi = rd.Snapshot.Metadata.Index } r.msgs = nil advancec = n.advancec @@ -318,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) @@ -389,23 +380,22 @@ 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.unstableEnts(), + Entries: r.raftLog.unstableEntries(), CommittedEntries: r.raftLog.nextEnts(), Messages: r.msgs, } @@ -415,8 +405,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_bench_test.go b/raft/node_bench_test.go index ab7cc103c..a92d883c4 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 74d3b943a..6f4bb0602 100644 --- a/raft/node_test.go +++ b/raft/node_test.go @@ -115,11 +115,13 @@ func TestNodePropose(t *testing.T) { } n := newNode() - r := newRaft(1, []uint64{1}, 10, 1) + s := NewMemoryStorage() + r := newRaft(1, []uint64{1}, 10, 1, s) go n.run(r) n.Campaign(context.TODO()) for { rd := <-n.Ready() + s.Append(rd.Entries) // change the step function to appendStep until this raft becomes leader if rd.SoftState.Lead == r.id { r.step = appendStep @@ -151,11 +153,13 @@ func TestNodeProposeConfig(t *testing.T) { } n := newNode() - r := newRaft(1, []uint64{1}, 10, 1) + s := NewMemoryStorage() + r := newRaft(1, []uint64{1}, 10, 1, s) go n.run(r) n.Campaign(context.TODO()) for { rd := <-n.Ready() + s.Append(rd.Entries) // change the step function to appendStep until this raft becomes leader if rd.SoftState.Lead == r.id { r.step = appendStep @@ -188,7 +192,7 @@ func TestNodeProposeConfig(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, NewMemoryStorage()) go n.run(r) defer n.Stop() @@ -220,7 +224,8 @@ func TestBlockProposal(t *testing.T) { // elapsed of the underlying raft state machine. func TestNodeTick(t *testing.T) { n := newNode() - r := newRaft(1, []uint64{1}, 10, 1) + s := NewMemoryStorage() + r := newRaft(1, []uint64{1}, 10, 1, s) go n.run(r) elapsed := r.elapsed n.Tick() @@ -234,7 +239,8 @@ func TestNodeTick(t *testing.T) { // processing, and that it is idempotent func TestNodeStop(t *testing.T) { n := newNode() - r := newRaft(1, []uint64{1}, 10, 1) + s := NewMemoryStorage() + r := newRaft(1, []uint64{1}, 10, 1, s) donec := make(chan struct{}) go func() { @@ -275,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 { @@ -302,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}, }, @@ -317,12 +322,15 @@ func TestNodeStart(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]) { - t.Errorf("#%d: g = %+v,\n w %+v", 1, g, wants[0]) + g := <-n.Ready() + if !reflect.DeepEqual(g, wants[0]) { + t.Fatalf("#%d: g = %+v,\n w %+v", 1, g, wants[0]) } else { + storage.Append(g.Entries) n.Advance() } @@ -330,6 +338,7 @@ func TestNodeStart(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() } @@ -342,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")}, } @@ -351,15 +359,17 @@ 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], } - n := RestartNode(1, 10, 1, nil, st, entries) + storage := NewMemoryStorage() + storage.SetHardState(st) + storage.Append(entries) + n := RestartNode(1, 10, 1, storage) if g := <-n.Ready(); !reflect.DeepEqual(g, want) { t.Errorf("g = %+v,\n w %+v", g, want) - } else { - n.Advance() } + n.Advance() select { case rd := <-n.Ready(): @@ -369,14 +379,14 @@ func TestNodeRestart(t *testing.T) { } func TestNodeRestartFromSnapshot(t *testing.T) { - 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} @@ -384,10 +394,14 @@ func TestNodeRestartFromSnapshot(t *testing.T) { want := Ready{ HardState: emptyState, // commit upto index commit index in st - CommittedEntries: entries[1:], + CommittedEntries: entries, } - n := RestartNode(1, 10, 1, snap, st, entries) + s := NewMemoryStorage() + s.SetHardState(st) + s.ApplySnapshot(snap) + s.Append(entries) + n := RestartNode(1, 10, 1, s) if g := <-n.Ready(); !reflect.DeepEqual(g, want) { t.Errorf("g = %+v,\n w %+v", g, want) } else { @@ -401,72 +415,23 @@ 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() - r := newRaft(1, []uint64{1}, 10, 1) - 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 <-n.Ready(): - 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) - } - 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.offset != w.Index { - t.Errorf("log.offset = %d, want %d", r.raftLog.offset, w.Index) - } -} - func TestNodeAdvance(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - n := StartNode(1, []Peer{{ID: 1}}, 10, 1) + 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) case <-time.After(time.Millisecond): } + storage.Append(rd.Entries) n.Advance() select { case <-n.Ready(): diff --git a/raft/raft.go b/raft/raft.go index 6d82fc7fe..3597b74fe 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -123,6 +123,9 @@ type raft struct { msgs []pb.Message + // the incoming snapshot, if any. + snapshot *pb.Snapshot + // the leader id lead uint64 @@ -137,21 +140,38 @@ 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") } + log := newLog(storage) + 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, - raftLog: newLog(), + raftLog: log, prs: make(map[uint64]*progress), electionTimeout: election, heartbeatTimeout: heartbeat, } r.rand = rand.New(rand.NewSource(int64(id))) for _, p := range peers { - r.prs[p] = &progress{} + r.prs[p] = &progress{next: 1} + } + if !isHardStateEqual(hs, emptyState) { + r.loadState(hs) } r.becomeFollower(0, None) return r @@ -207,12 +227,19 @@ 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 + snapshot, err := r.raftLog.storage.Snapshot() + if err != nil { + panic(err) // TODO(bdarnell) + } + if IsEmptySnap(snapshot) { + panic("need non-empty snapshot") + } + m.Snapshot = 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 @@ -418,6 +445,7 @@ 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}) @@ -530,28 +558,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)) - } - if index < r.raftLog.offset { - //TODO: return an error? - return - } - 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 { @@ -562,13 +578,7 @@ func (r *raft) restore(s pb.Snapshot) bool { } func (r *raft) needSnapshot(i uint64) bool { - if i < r.raftLog.offset { - 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 { @@ -595,10 +605,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 2ed44fabe..d2be61129 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, 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) + 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) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) if r.state != StateFollower { t.Errorf("state = %s, want %s", r.state, StateFollower) } @@ -109,11 +109,11 @@ 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, NewMemoryStorage()) 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++ { @@ -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, 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) + 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) + 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) + 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) + 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) + 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) + 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")}} @@ -412,7 +413,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,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) + 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) + 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,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) - r.loadEnts(append([]pb.Entry{{}}, tt...)) + storage := NewMemoryStorage() + storage.Append(tt) + r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage) r.loadState(pb.HardState{Term: 2}) r.becomeCandidate() r.becomeLeader() @@ -566,7 +570,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, NewMemoryStorage()) r.becomeFollower(1, 2) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 1, Entries: tt.ents, Commit: tt.commit}) @@ -587,21 +591,22 @@ 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 }{ + {ents[0].Term, ents[0].Index, false}, + {ents[0].Term, ents[0].Index + 1, true}, + {ents[0].Term + 1, ents[0].Index, true}, {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}, {3, 3, true}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) - 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) @@ -632,39 +637,40 @@ 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 { - r := newRaft(1, []uint64{1, 2, 3}, 10, 1) - 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}) - 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) } } @@ -723,11 +729,13 @@ func TestLeaderSyncFollowerLog(t *testing.T) { }, } for i, tt := range tests { - lead := newRaft(1, []uint64{1, 2, 3}, 10, 1) - 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) - 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 @@ -756,7 +764,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, NewMemoryStorage()) r.Step(pb.Message{ From: 2, To: 1, Type: pb.MsgApp, Term: tt.wterm - 1, LogTerm: 0, Index: 0, Entries: tt.ents, }) @@ -804,21 +812,22 @@ 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 { - r := newRaft(1, []uint64{1, 2}, 10, 1) - 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}) @@ -840,7 +849,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 @@ -852,8 +861,9 @@ func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) { {3, 3}, } for i, tt := range tests { - r := newRaft(1, []uint64{1, 2}, 10, 1) - 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() @@ -875,7 +885,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") } @@ -890,6 +900,7 @@ func commitNoopEntry(r *raft) { } // ignore further messages to refresh followers' commmit index r.readMessages() + 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 8f51250f0..4969fbfc8 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -22,14 +22,17 @@ import ( "math" "math/rand" "reflect" - "sort" "testing" pb "github.com/coreos/etcd/raft/raftpb" ) // 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. + s.Append(r.raftLog.unstableEntries()) + r.raftLog.stableTo(r.raftLog.lastIndex()) + ents = r.raftLog.nextEnts() r.raftLog.appliedTo(r.raftLog.committed) return ents @@ -209,7 +212,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) } @@ -318,9 +321,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, 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) @@ -331,7 +334,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 @@ -340,7 +347,7 @@ func TestDuelingCandidates(t *testing.T) { }{ {a, StateFollower, 2, wlog}, {b, StateFollower, 2, wlog}, - {c, StateFollower, 2, newLog()}, + {c, StateFollower, 2, newLog(NewMemoryStorage())}, } for i, tt := range tests { @@ -383,7 +390,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) @@ -416,10 +429,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) @@ -470,9 +486,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(NewMemoryStorage()) 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 { @@ -506,7 +527,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 { @@ -525,50 +551,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, - 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 !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 @@ -602,7 +584,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) @@ -624,7 +610,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, NewMemoryStorage()) sm.elapsed = tt.elapse c := 0 for j := 0; j < 10000; j++ { @@ -649,7 +635,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, NewMemoryStorage()) sm.step = fakeStep sm.Term = 2 sm.Step(pb.Message{Type: pb.MsgApp, Term: sm.Term - 1}) @@ -692,7 +678,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) @@ -724,10 +714,12 @@ func TestHandleHeartbeat(t *testing.T) { } for i, tt := range tests { + storage := NewMemoryStorage() + 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}, - raftLog: &raftLog{committed: 0, ents: []pb.Entry{{}, {Term: 1}, {Term: 2}, {Term: 3}}}, + raftLog: newLog(storage), } sm.raftLog.commitTo(commit) sm.handleHeartbeat(tt.m) @@ -776,7 +768,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, NewMemoryStorage()) sm.state = tt.state switch tt.state { case StateFollower: @@ -787,7 +779,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{{}, {Index: 1, Term: 2}, {Index: 2, Term: 2}}}, + unstable: 3, + } sm.Step(pb.Message{Type: pb.MsgVote, From: 2, Index: tt.i, LogTerm: tt.term}) @@ -833,7 +828,7 @@ func TestStateTransition(t *testing.T) { } }() - sm := newRaft(1, []uint64{1}, 10, 1) + sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) sm.state = tt.from switch tt.to { @@ -863,16 +858,16 @@ 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} 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, NewMemoryStorage()) switch tt.state { case StateFollower: sm.becomeFollower(1, None) @@ -892,8 +887,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(len(sm.raftLog.ents)) != tt.windex { - t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.ents), tt.windex) + 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 len(ents) = %v , want %v", i, j, len(sm.raftLog.allEntries()), tt.windex) } wlead := uint64(2) if msgType == pb.MsgVote { @@ -928,8 +926,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, NewMemoryStorage()) + sm.raftLog = &raftLog{ + storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}}, + unstable: 3, + } sm.becomeCandidate() sm.becomeLeader() sm.readMessages() @@ -965,18 +966,21 @@ 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) + storage := NewMemoryStorage() + storage.ApplySnapshot(s) + sm := newRaft(1, nil, 10, 1, storage) sm.Term = 1 - sm.restore(s) 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 @@ -1029,8 +1033,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, NewMemoryStorage()) + 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 { @@ -1072,7 +1076,7 @@ func TestLeaderIncreaseNext(t *testing.T) { } for i, tt := range tests { - sm := newRaft(1, []uint64{1, 2}, 10, 1) + sm := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) sm.raftLog.append(0, previousEnts...) sm.becomeCandidate() sm.becomeLeader() @@ -1088,28 +1092,28 @@ func TestLeaderIncreaseNext(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) + 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 { @@ -1118,14 +1122,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) // 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() @@ -1133,7 +1140,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() @@ -1148,18 +1155,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) + 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) { @@ -1171,16 +1178,13 @@ 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) - lead.compact(lead.raftLog.applied, lead.nodes(), nil) + nextEnts(lead, nt.storage[1]) + nt.storage[1].Compact(lead.raftLog.applied, &pb.ConfState{Nodes: 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{{}}}) @@ -1193,7 +1197,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, NewMemoryStorage()) r.becomeCandidate() r.becomeLeader() index := r.raftLog.lastIndex() @@ -1211,7 +1215,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, NewMemoryStorage()) r.becomeCandidate() r.becomeLeader() r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange}}}) @@ -1237,7 +1241,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, NewMemoryStorage()) r.appendEntry(pb.Entry{Type: tt.entType}) r.becomeCandidate() r.becomeLeader() @@ -1256,7 +1260,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, NewMemoryStorage()) r.appendEntry(pb.Entry{Type: pb.EntryConfChange}) r.appendEntry(pb.Entry{Type: pb.EntryConfChange}) r.becomeCandidate() @@ -1266,7 +1270,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, NewMemoryStorage()) r.pendingConf = true r.addNode(2) if r.pendingConf != false { @@ -1282,7 +1286,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, NewMemoryStorage()) r.pendingConf = true r.removeNode(2) if r.pendingConf != false { @@ -1331,7 +1335,7 @@ func TestRaftNodes(t *testing.T) { }, } for i, tt := range tests { - r := newRaft(1, tt.ids, 10, 1) + r := newRaft(1, tt.ids, 10, 1, NewMemoryStorage()) if !reflect.DeepEqual(r.nodes(), tt.wids) { t.Errorf("#%d: nodes = %+v, want %+v", i, r.nodes(), tt.wids) } @@ -1340,17 +1344,23 @@ 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{raftLog: &raftLog{ents: ents}} + sm := &raft{ + raftLog: &raftLog{ + storage: &MemoryStorage{ents: ents}, + unstable: uint64(len(ents)), + }, + } sm.reset(0) return sm } type network struct { peers map[uint64]Interface + storage map[uint64]*MemoryStorage dropm map[connem]float64 ignorem map[pb.MessageType]bool } @@ -1364,12 +1374,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) + nstorage[id] = NewMemoryStorage() + sm := newRaft(id, peerAddrs, 10, 1, nstorage[id]) npeers[id] = sm case *raft: v.id = id @@ -1387,6 +1399,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/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 new file mode 100644 index 000000000..dcfbb6f5d --- /dev/null +++ b/raft/storage.go @@ -0,0 +1,187 @@ +/* + 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 ( + "errors" + "fmt" + "sync" + + pb "github.com/coreos/etcd/raft/raftpb" +) + +// ErrCompacted is returned by Storage.Entries when a requested +// index is unavailable because it predates the last snapshot. +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. +// +// 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 { + // 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 + // [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) + // 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) + // 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 +// 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 + + hardState pb.HardState + snapshot pb.Snapshot + // ents[i] has raft log position i+snapshot.Metadata.Index + ents []pb.Entry +} + +// NewMemoryStorage creates an empty MemoryStorage. +func NewMemoryStorage() *MemoryStorage { + return &MemoryStorage{ + // When starting from scratch populate the list with a dummy entry at term zero. + ents: make([]pb.Entry, 1), + } +} + +// 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. +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() + offset := ms.snapshot.Metadata.Index + if lo <= offset { + return nil, ErrCompacted + } + 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() + offset := ms.snapshot.Metadata.Index + if i < offset { + return 0, ErrCompacted + } + 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.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.snapshot.Metadata.Index + 1, nil +} + +// Snapshot implements the Storage interface. +func (ms *MemoryStorage) Snapshot() (pb.Snapshot, error) { + ms.Lock() + defer ms.Unlock() + 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.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 +} + +// Append the new entries to storage. +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...) +} 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/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) diff --git a/test b/test index 2210e6e2a..6f0aaa7be 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 60s ${COVER} $@ ${TEST} --race echo "Checking gofmt..." fmtRes=$(gofmt -l $FMT)