Merge branch 'log_interface'

Conflicts:
	raft/raft.go
release-2.0
Yicheng Qin 2014-11-25 14:22:11 -08:00
commit 7e6e305c4f
24 changed files with 1424 additions and 805 deletions

View File

@ -60,7 +60,7 @@ func handleBackup(c *cli.Context) {
} }
var index uint64 var index uint64
if snapshot != nil { if snapshot != nil {
index = snapshot.Index index = snapshot.Metadata.Index
newss := snap.New(destSnap) newss := snap.New(destSnap)
if err := newss.SaveSnap(*snapshot); err != nil { if err := newss.SaveSnap(*snapshot); err != nil {
log.Fatal(err) log.Fatal(err)

View File

@ -28,7 +28,7 @@ import (
"github.com/coreos/etcd/wal" "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) w, id, cid, st, ents := readWAL(cfg.WALDir(), index)
cfg.Cluster.SetID(cid) 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) 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) s := raft.NewMemoryStorage()
return id, n, w 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 // 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 { func getIDs(snap *raftpb.Snapshot, ents []raftpb.Entry) []uint64 {
ids := make(map[uint64]bool) ids := make(map[uint64]bool)
if snap != nil { if snap != nil {
for _, id := range snap.Nodes { for _, id := range snap.Metadata.ConfState.Nodes {
ids[id] = true ids[id] = true
} }
} }

View File

@ -34,21 +34,30 @@ func TestGetIDs(t *testing.T) {
normalEntry := raftpb.Entry{Type: raftpb.EntryNormal} normalEntry := raftpb.Entry{Type: raftpb.EntryNormal}
tests := []struct { tests := []struct {
snap *raftpb.Snapshot confState *raftpb.ConfState
ents []raftpb.Entry ents []raftpb.Entry
widSet []uint64 widSet []uint64
}{ }{
{nil, []raftpb.Entry{}, []uint64{}}, {nil, []raftpb.Entry{}, []uint64{}},
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{}, []uint64{1}}, {&raftpb.ConfState{Nodes: []uint64{1}},
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry}, []uint64{1, 2}}, []raftpb.Entry{}, []uint64{1}},
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry, removeEntry}, []uint64{1}}, {&raftpb.ConfState{Nodes: []uint64{1}},
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry, normalEntry}, []uint64{1, 2}}, []raftpb.Entry{addEntry}, []uint64{1, 2}},
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry, removeEntry, normalEntry}, []uint64{1}}, {&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 { 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) { if !reflect.DeepEqual(idSet, tt.widSet) {
t.Errorf("#%d: idset = %#v, want %#v", i, idSet, tt.widSet) t.Errorf("#%d: idset = %#v, want %#v", i, idSet, tt.widSet)
} }

View File

@ -165,8 +165,9 @@ type EtcdServer struct {
Cluster *Cluster Cluster *Cluster
node raft.Node node raft.Node
store store.Store raftStorage *raft.MemoryStorage
store store.Store
stats *stats.ServerStats stats *stats.ServerStats
lstats *stats.LeaderStats lstats *stats.LeaderStats
@ -211,6 +212,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
st := store.New() st := store.New()
var w *wal.WAL var w *wal.WAL
var n raft.Node var n raft.Node
var s *raft.MemoryStorage
var id types.ID var id types.ID
walVersion := wal.DetectVersion(cfg.DataDir) walVersion := wal.DetectVersion(cfg.DataDir)
if walVersion == wal.WALUnknown { if walVersion == wal.WALUnknown {
@ -243,7 +245,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
cfg.Cluster.SetID(existingCluster.id) cfg.Cluster.SetID(existingCluster.id)
cfg.Cluster.SetStore(st) cfg.Cluster.SetStore(st)
cfg.Print() cfg.Print()
id, n, w = startNode(cfg, nil) id, n, s, w = startNode(cfg, nil)
case !haveWAL && cfg.NewCluster: case !haveWAL && cfg.NewCluster:
if err := cfg.VerifyBootstrapConfig(); err != nil { if err := cfg.VerifyBootstrapConfig(); err != nil {
return nil, err return nil, err
@ -263,7 +265,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
} }
cfg.Cluster.SetStore(st) cfg.Cluster.SetStore(st)
cfg.PrintWithInitial() cfg.PrintWithInitial()
id, n, w = startNode(cfg, cfg.Cluster.MemberIDs()) id, n, s, w = startNode(cfg, cfg.Cluster.MemberIDs())
case haveWAL: case haveWAL:
if cfg.ShouldDiscover() { if cfg.ShouldDiscover() {
log.Printf("etcdserver: warn: ignoring discovery: etcd has already been initialized and has a valid log in %q", cfg.WALDir()) 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 return nil, err
} }
if snapshot != nil { 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) st.Recovery(snapshot.Data)
index = snapshot.Index index = snapshot.Metadata.Index
} }
cfg.Cluster = NewClusterFromStore(cfg.Cluster.token, st) cfg.Cluster = NewClusterFromStore(cfg.Cluster.token, st)
cfg.Print() cfg.Print()
@ -284,9 +286,9 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
log.Printf("etcdserver: loaded peers from snapshot: %s", cfg.Cluster) log.Printf("etcdserver: loaded peers from snapshot: %s", cfg.Cluster)
} }
if !cfg.ForceNewCluster { if !cfg.ForceNewCluster {
id, n, w = restartNode(cfg, index, snapshot) id, n, s, w = restartNode(cfg, index+1, snapshot)
} else { } else {
id, n, w = restartAsStandaloneNode(cfg, index, snapshot) id, n, s, w = restartAsStandaloneNode(cfg, index+1, snapshot)
} }
default: default:
return nil, fmt.Errorf("unsupported bootstrap config") return nil, fmt.Errorf("unsupported bootstrap config")
@ -298,12 +300,13 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
} }
lstats := stats.NewLeaderStats(id.String()) lstats := stats.NewLeaderStats(id.String())
s := &EtcdServer{ srv := &EtcdServer{
store: st, store: st,
node: n, node: n,
id: id, raftStorage: s,
attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()}, id: id,
Cluster: cfg.Cluster, attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
Cluster: cfg.Cluster,
storage: struct { storage: struct {
*wal.WAL *wal.WAL
*snap.Snapshotter *snap.Snapshotter
@ -314,8 +317,8 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
SyncTicker: time.Tick(500 * time.Millisecond), SyncTicker: time.Tick(500 * time.Millisecond),
snapCount: cfg.SnapCount, snapCount: cfg.SnapCount,
} }
s.sendhub = newSendHub(cfg.Transport, cfg.Cluster, s, sstats, lstats) srv.sendhub = newSendHub(cfg.Transport, cfg.Cluster, srv, sstats, lstats)
return s, nil return srv, nil
} }
// Start prepares and starts server in a new goroutine. It is no longer safe to // 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 { if err := s.storage.Save(rd.HardState, rd.Entries); err != nil {
log.Fatalf("etcdserver: save state and entries error: %v", err) log.Fatalf("etcdserver: save state and entries error: %v", err)
} }
if err := s.storage.SaveSnap(rd.Snapshot); err != nil { s.raftStorage.Append(rd.Entries)
log.Fatalf("etcdserver: create snapshot error: %v", err)
}
s.sendhub.Send(rd.Messages) s.sendhub.Send(rd.Messages)
// recover from snapshot if it is more updated than current applied if !raft.IsEmptySnap(rd.Snapshot) {
if rd.Snapshot.Index > appliedi { // recover from snapshot if it is more updated than current applied
if err := s.store.Recovery(rd.Snapshot.Data); err != nil { if rd.Snapshot.Metadata.Index > appliedi {
log.Panicf("recovery store error: %v", err) 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 // TODO(bmizerany): do this in the background, but take
// care to apply entries in a single goroutine, and not // care to apply entries in a single goroutine, and not
@ -409,6 +419,7 @@ func (s *EtcdServer) run() {
firsti := rd.CommittedEntries[0].Index firsti := rd.CommittedEntries[0].Index
if appliedi == 0 { if appliedi == 0 {
appliedi = firsti - 1 appliedi = firsti - 1
snapi = appliedi
} }
if firsti > appliedi+1 { if firsti > appliedi+1 {
log.Panicf("etcdserver: first index of committed entry[%d] should <= appliedi[%d] + 1", firsti, appliedi) 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() s.node.Advance()
if rd.Snapshot.Index > snapi {
snapi = rd.Snapshot.Index
}
if appliedi-snapi > s.snapCount { if appliedi-snapi > s.snapCount {
s.snapshot(appliedi, nodes) s.snapshot(appliedi, nodes)
snapi = appliedi snapi = appliedi
@ -798,10 +806,17 @@ func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) {
if err != nil { if err != nil {
log.Panicf("store save should never fail: %v", err) 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 { if err := s.storage.Cut(); err != nil {
log.Panicf("rotate wal file should never fail: %v", err) 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, // 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") 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 var err error
member := cfg.Cluster.MemberByName(cfg.Name) member := cfg.Cluster.MemberByName(cfg.Name)
metadata := pbutil.MustMarshal( metadata := pbutil.MustMarshal(
@ -897,7 +912,8 @@ func startNode(cfg *ServerConfig, ids []types.ID) (id types.ID, n raft.Node, w *
} }
id = member.ID id = member.ID
log.Printf("etcdserver: start member %s in cluster %s", id, cfg.Cluster.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 return
} }
@ -915,13 +931,19 @@ func getOtherPeerURLs(cl ClusterInfo, self string) []string {
return us 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) w, id, cid, st, ents := readWAL(cfg.WALDir(), index)
cfg.Cluster.SetID(cid) cfg.Cluster.SetID(cid)
log.Printf("etcdserver: restart member %s in cluster %s at commit index %d", id, cfg.Cluster.ID(), st.Commit) 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) s := raft.NewMemoryStorage()
return id, n, w 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) { func readWAL(waldir string, index uint64) (w *wal.WAL, id, cid types.ID, st raftpb.HardState, ents []raftpb.Entry) {

View File

@ -560,24 +560,30 @@ func testServer(t *testing.T, ns uint64) {
members := mustMakePeerSlice(t, ids...) members := mustMakePeerSlice(t, ids...)
for i := uint64(0); i < ns; i++ { for i := uint64(0); i < ns; i++ {
id := i + 1 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) tk := time.NewTicker(10 * time.Millisecond)
defer tk.Stop() defer tk.Stop()
st := store.New() st := store.New()
cl := newCluster("abc") cl := newCluster("abc")
cl.SetStore(st) cl.SetStore(st)
srv := &EtcdServer{ srv := &EtcdServer{
node: n, node: n,
store: st, raftStorage: s,
sendhub: &fakeSender{ss}, store: st,
storage: &storageRecorder{}, sendhub: &fakeSender{ss},
Ticker: tk.C, storage: &storageRecorder{},
Cluster: cl, Ticker: tk.C,
Cluster: cl,
} }
srv.start()
ss[i] = srv 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++ { for i := 1; i <= 10; i++ {
r := pb.Request{ r := pb.Request{
Method: "PUT", Method: "PUT",
@ -628,7 +634,8 @@ func TestDoProposal(t *testing.T) {
for i, tt := range tests { for i, tt := range tests {
ctx, _ := context.WithCancel(context.Background()) 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{} st := &storeRecorder{}
tk := make(chan time.Time) tk := make(chan time.Time)
// this makes <-tk always successful, which accelerates internal clock // this makes <-tk always successful, which accelerates internal clock
@ -636,12 +643,13 @@ func TestDoProposal(t *testing.T) {
cl := newCluster("abc") cl := newCluster("abc")
cl.SetStore(store.New()) cl.SetStore(store.New())
srv := &EtcdServer{ srv := &EtcdServer{
node: n, node: n,
store: st, raftStorage: s,
sendhub: &nopSender{}, store: st,
storage: &storageRecorder{}, sendhub: &nopSender{},
Ticker: tk, storage: &storageRecorder{},
Cluster: cl, Ticker: tk,
Cluster: cl,
} }
srv.start() srv.start()
resp, err := srv.Do(ctx, tt) resp, err := srv.Do(ctx, tt)
@ -664,14 +672,16 @@ func TestDoProposal(t *testing.T) {
func TestDoProposalCancelled(t *testing.T) { func TestDoProposalCancelled(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
// node cannot make any progress because there are two nodes // 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{} st := &storeRecorder{}
wait := &waitRecorder{} wait := &waitRecorder{}
srv := &EtcdServer{ srv := &EtcdServer{
// TODO: use fake node for better testability // TODO: use fake node for better testability
node: n, node: n,
store: st, raftStorage: s,
w: wait, store: st,
w: wait,
} }
done := make(chan struct{}) done := make(chan struct{})
@ -712,7 +722,8 @@ func TestDoProposalStopped(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
// node cannot make any progress because there are two nodes // 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{} st := &storeRecorder{}
tk := make(chan time.Time) tk := make(chan time.Time)
// this makes <-tk always successful, which accelarates internal clock // this makes <-tk always successful, which accelarates internal clock
@ -721,12 +732,13 @@ func TestDoProposalStopped(t *testing.T) {
cl.SetStore(store.New()) cl.SetStore(store.New())
srv := &EtcdServer{ srv := &EtcdServer{
// TODO: use fake node for better testability // TODO: use fake node for better testability
node: n, node: n,
store: st, raftStorage: s,
sendhub: &nopSender{}, store: st,
storage: &storageRecorder{}, sendhub: &nopSender{},
Ticker: tk, storage: &storageRecorder{},
Cluster: cl, Ticker: tk,
Cluster: cl,
} }
srv.start() srv.start()
@ -832,11 +844,12 @@ func TestSyncTrigger(t *testing.T) {
} }
st := make(chan time.Time, 1) st := make(chan time.Time, 1)
srv := &EtcdServer{ srv := &EtcdServer{
node: n, node: n,
store: &storeRecorder{}, raftStorage: raft.NewMemoryStorage(),
sendhub: &nopSender{}, store: &storeRecorder{},
storage: &storageRecorder{}, sendhub: &nopSender{},
SyncTicker: st, storage: &storageRecorder{},
SyncTicker: st,
} }
srv.start() srv.start()
// trigger the server to become a leader and accept sync requests // 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 // snapshot should snapshot the store and cut the persistent
// TODO: node.Compact is called... we need to make the node an interface // TODO: node.Compact is called... we need to make the node an interface
func TestSnapshot(t *testing.T) { 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() 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{} st := &storeRecorder{}
p := &storageRecorder{} p := &storageRecorder{}
s := &EtcdServer{ srv := &EtcdServer{
store: st, store: st,
storage: p, storage: p,
node: n, node: n,
raftStorage: s,
} }
s.snapshot(0, []uint64{1}) srv.snapshot(1, []uint64{1})
gaction := st.Action() gaction := st.Action()
if len(gaction) != 1 { if len(gaction) != 1 {
t.Fatalf("len(action) = %d, want 1", len(gaction)) t.Fatalf("len(action) = %d, want 1", len(gaction))
@ -886,19 +927,24 @@ func TestSnapshot(t *testing.T) {
} }
gaction = p.Action() gaction = p.Action()
if len(gaction) != 1 { if len(gaction) != 2 {
t.Fatalf("len(action) = %d, want 1", len(gaction)) t.Fatalf("len(action) = %d, want 2", len(gaction))
} }
if !reflect.DeepEqual(gaction[0], action{name: "Cut"}) { if !reflect.DeepEqual(gaction[0], action{name: "Cut"}) {
t.Errorf("action = %s, want Cut", gaction[0]) 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 // Applied > SnapCount should trigger a SaveSnap event
func TestTriggerSnap(t *testing.T) { func TestTriggerSnap(t *testing.T) {
ctx := context.Background() ctx := context.Background()
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1) s := raft.NewMemoryStorage()
<-n.Ready() n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s)
rd := <-n.Ready()
s.Append(rd.Entries)
n.Advance() n.Advance()
n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 0xBAD0}) n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 0xBAD0})
n.Campaign(ctx) n.Campaign(ctx)
@ -906,26 +952,30 @@ func TestTriggerSnap(t *testing.T) {
p := &storageRecorder{} p := &storageRecorder{}
cl := newCluster("abc") cl := newCluster("abc")
cl.SetStore(store.New()) cl.SetStore(store.New())
s := &EtcdServer{ srv := &EtcdServer{
store: st, store: st,
sendhub: &nopSender{}, sendhub: &nopSender{},
storage: p, storage: p,
node: n, node: n,
snapCount: 10, raftStorage: s,
Cluster: cl, snapCount: 10,
Cluster: cl,
} }
s.start() srv.start()
for i := 0; uint64(i) < s.snapCount-1; i++ { // wait for saving nop
s.Do(ctx, pb.Request{Method: "PUT", ID: 1})
}
time.Sleep(time.Millisecond) 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() gaction := p.Action()
// each operation is recorded as a Save // each operation is recorded as a Save
// BootstrapConfig/Nop + (SnapCount - 1) * Puts + Cut + SaveSnap = Save + (SnapCount - 1) * Save + Cut + SaveSnap // 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 { if len(gaction) != wcnt {
t.Fatalf("len(action) = %d, want %d", 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 := newCluster("abc")
cl.SetStore(store.New()) cl.SetStore(store.New())
s := &EtcdServer{ s := &EtcdServer{
store: st, store: st,
sendhub: &nopSender{}, sendhub: &nopSender{},
storage: p, storage: p,
node: n, node: n,
Cluster: cl, raftStorage: raft.NewMemoryStorage(),
Cluster: cl,
} }
s.start() 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 // make goroutines move forward to receive snapshot
testutil.ForceGosched() testutil.ForceGosched()
s.Stop() s.Stop()
@ -960,7 +1011,7 @@ func TestRecvSnapshot(t *testing.T) {
if g := st.Action(); !reflect.DeepEqual(g, wactions) { if g := st.Action(); !reflect.DeepEqual(g, wactions) {
t.Errorf("store action = %v, want %v", 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) { if g := p.Action(); !reflect.DeepEqual(g, wactions) {
t.Errorf("storage action = %v, want %v", g, wactions) t.Errorf("storage action = %v, want %v", g, wactions)
} }
@ -975,20 +1026,21 @@ func TestRecvSlowSnapshot(t *testing.T) {
cl := newCluster("abc") cl := newCluster("abc")
cl.SetStore(store.New()) cl.SetStore(store.New())
s := &EtcdServer{ s := &EtcdServer{
store: st, store: st,
sendhub: &nopSender{}, sendhub: &nopSender{},
storage: &storageRecorder{}, storage: &storageRecorder{},
node: n, node: n,
Cluster: cl, raftStorage: raft.NewMemoryStorage(),
Cluster: cl,
} }
s.start() 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 // make goroutines move forward to receive snapshot
testutil.ForceGosched() testutil.ForceGosched()
action := st.Action() 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 // make goroutines move forward to receive snapshot
testutil.ForceGosched() testutil.ForceGosched()
s.Stop() s.Stop()
@ -1005,18 +1057,20 @@ func TestApplySnapshotAndCommittedEntries(t *testing.T) {
st := &storeRecorder{} st := &storeRecorder{}
cl := newCluster("abc") cl := newCluster("abc")
cl.SetStore(store.New()) cl.SetStore(store.New())
storage := raft.NewMemoryStorage()
s := &EtcdServer{ s := &EtcdServer{
store: st, store: st,
sendhub: &nopSender{}, sendhub: &nopSender{},
storage: &storageRecorder{}, storage: &storageRecorder{},
node: n, node: n,
Cluster: cl, raftStorage: storage,
Cluster: cl,
} }
s.start() s.start()
req := &pb.Request{Method: "QGET"} req := &pb.Request{Method: "QGET"}
n.readyc <- raft.Ready{ n.readyc <- raft.Ready{
Snapshot: raftpb.Snapshot{Index: 1}, Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}},
CommittedEntries: []raftpb.Entry{ CommittedEntries: []raftpb.Entry{
{Index: 2, Data: pbutil.MustMarshal(req)}, {Index: 2, Data: pbutil.MustMarshal(req)},
}, },
@ -1049,11 +1103,12 @@ func TestAddMember(t *testing.T) {
cl := newTestCluster(nil) cl := newTestCluster(nil)
cl.SetStore(store.New()) cl.SetStore(store.New())
s := &EtcdServer{ s := &EtcdServer{
node: n, node: n,
store: &storeRecorder{}, raftStorage: raft.NewMemoryStorage(),
sendhub: &nopSender{}, store: &storeRecorder{},
storage: &storageRecorder{}, sendhub: &nopSender{},
Cluster: cl, storage: &storageRecorder{},
Cluster: cl,
} }
s.start() s.start()
m := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"foo"}}} m := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"foo"}}}
@ -1086,11 +1141,12 @@ func TestRemoveMember(t *testing.T) {
cl.SetStore(store.New()) cl.SetStore(store.New())
cl.AddMember(&Member{ID: 1234}) cl.AddMember(&Member{ID: 1234})
s := &EtcdServer{ s := &EtcdServer{
node: n, node: n,
store: &storeRecorder{}, raftStorage: raft.NewMemoryStorage(),
sendhub: &nopSender{}, store: &storeRecorder{},
storage: &storageRecorder{}, sendhub: &nopSender{},
Cluster: cl, storage: &storageRecorder{},
Cluster: cl,
} }
s.start() s.start()
err := s.RemoveMember(context.TODO(), 1234) err := s.RemoveMember(context.TODO(), 1234)
@ -1122,11 +1178,12 @@ func TestUpdateMember(t *testing.T) {
cl.SetStore(store.New()) cl.SetStore(store.New())
cl.AddMember(&Member{ID: 1234}) cl.AddMember(&Member{ID: 1234})
s := &EtcdServer{ s := &EtcdServer{
node: n, node: n,
store: &storeRecorder{}, raftStorage: raft.NewMemoryStorage(),
sendhub: &nopSender{}, store: &storeRecorder{},
storage: &storageRecorder{}, sendhub: &nopSender{},
Cluster: cl, storage: &storageRecorder{},
Cluster: cl,
} }
s.start() s.start()
wm := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"http://127.0.0.1:1"}}} 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 { func (n *readyNode) ProposeConfChange(ctx context.Context, conf raftpb.ConfChange) error {
return nil return nil
} }
func (n *readyNode) Step(ctx context.Context, msg raftpb.Message) 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) Ready() <-chan raft.Ready { return n.readyc }
func (n *readyNode) Advance() {} func (n *readyNode) Advance() {}
func (n *readyNode) ApplyConfChange(conf raftpb.ConfChange) {} func (n *readyNode) ApplyConfChange(conf raftpb.ConfChange) *raftpb.ConfState { return nil }
func (n *readyNode) Stop() {} func (n *readyNode) Stop() {}
func (n *readyNode) Compact(index uint64, nodes []uint64, d []byte) {} func (n *readyNode) Compact(index uint64, nodes []uint64, d []byte) {}
type nodeRecorder struct { type nodeRecorder struct {
recorder 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) Ready() <-chan raft.Ready { return nil }
func (n *nodeRecorder) Advance() {} 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}}) n.record(action{name: "ApplyConfChange", params: []interface{}{conf}})
return nil
} }
func (n *nodeRecorder) Stop() { func (n *nodeRecorder) Stop() {
n.record(action{name: "Stop"}) n.record(action{name: "Stop"})
@ -1580,8 +1638,9 @@ func (n *nodeConfChangeCommitterRecorder) ProposeConfChange(ctx context.Context,
func (n *nodeConfChangeCommitterRecorder) Ready() <-chan raft.Ready { func (n *nodeConfChangeCommitterRecorder) Ready() <-chan raft.Ready {
return n.readyc 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()}) n.record(action{name: "ApplyConfChange:" + conf.Type.String()})
return nil
} }
type waitWithResponse struct { type waitWithResponse struct {

View File

@ -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 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 { if snap5 != nil {
st5.Commit = snap5.Index st5.Commit = snap5.Metadata.Index
} }
ents5, err := Entries4To5(ents4) ents5, err := Entries4To5(ents4)

View File

@ -178,10 +178,14 @@ func (s *Snapshot4) Snapshot5() *raftpb.Snapshot {
} }
snap5 := raftpb.Snapshot{ snap5 := raftpb.Snapshot{
Data: newState, Data: newState,
Index: s.LastIndex, Metadata: raftpb.SnapshotMetadata{
Term: s.LastTerm, Index: s.LastIndex,
Nodes: nodeList, Term: s.LastTerm,
ConfState: raftpb.ConfState{
Nodes: nodeList,
},
},
} }
return &snap5 return &snap5

View File

@ -60,7 +60,7 @@ func mustTemp(pre, body string) string {
func ltoa(l *raftLog) string { func ltoa(l *raftLog) string {
s := fmt.Sprintf("committed: %d\n", l.committed) s := fmt.Sprintf("committed: %d\n", l.committed)
s += fmt.Sprintf("applied: %d\n", l.applied) 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) s += fmt.Sprintf("#%d: %+v\n", i, e)
} }
return s return s

View File

@ -26,7 +26,7 @@ func saveStateToDisk(st pb.HardState) {}
func saveToDisk(ents []pb.Entry) {} func saveToDisk(ents []pb.Entry) {}
func Example_Node() { func Example_Node() {
n := StartNode(0, nil, 0, 0) n := StartNode(0, nil, 0, 0, nil)
// stuff to n happens in other goroutines // stuff to n happens in other goroutines

View File

@ -24,33 +24,53 @@ import (
) )
type raftLog struct { type raftLog struct {
ents []pb.Entry // storage contains all stable entries since the last snapshot.
unstable uint64 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 committed uint64
applied uint64 // applied is the highest log position that the application has
offset uint64 // been instructed to apply to its state machine.
snapshot pb.Snapshot // Invariant: applied <= committed
applied uint64
} }
func newLog() *raftLog { // newLog returns log using the given storage. It recovers the log to the state
return &raftLog{ // that it just commits and applies the lastest snapshot.
ents: make([]pb.Entry, 1), func newLog(storage Storage) *raftLog {
unstable: 0, if storage == nil {
committed: 0, log.Panic("storage must not be nil")
applied: 0,
} }
} 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) { return log
if l.offset != ents[0].Index {
panic("entries loaded don't match offset index")
}
l.ents = ents
l.unstable = l.offset + uint64(len(ents))
} }
func (l *raftLog) String() string { 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, // 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 { switch {
case ci == 0: case ci == 0:
case ci <= l.committed: case ci <= l.committed:
panic("conflict with committed entry") log.Panicf("conflict(%d) with committed entry [committed(%d)]", ci, l.committed)
default: default:
l.append(ci-1, ents[ci-from:]...) 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 { func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 {
l.ents = append(l.slice(l.offset, after+1), ents...) if after < l.committed {
l.unstable = min(l.unstable, after+1) 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() 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 { func (l *raftLog) findConflict(from uint64, ents []pb.Entry) uint64 {
// TODO(xiangli): validate the index of ents // TODO(xiangli): validate the index of ents
for i, ne := range 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 from + uint64(i)
} }
} }
return 0 return 0
} }
func (l *raftLog) unstableEnts() []pb.Entry { func (l *raftLog) unstableEntries() []pb.Entry {
ents := l.slice(l.unstable, l.lastIndex()+1) if len(l.unstableEnts) == 0 {
if ents == nil {
return nil return nil
} }
cpy := make([]pb.Entry, len(ents)) // copy unstable entries to an empty slice
copy(cpy, ents) return append([]pb.Entry{}, l.unstableEnts...)
return cpy
} }
// nextEnts returns all the available entries for execution. // nextEnts returns all the available entries for execution.
// If applied is smaller than the index of snapshot, it returns all committed // If applied is smaller than the index of snapshot, it returns all committed
// entries after the index of snapshot. // entries after the index of snapshot.
func (l *raftLog) nextEnts() (ents []pb.Entry) { func (l *raftLog) nextEnts() (ents []pb.Entry) {
off := max(l.applied, l.snapshot.Index) off := max(l.applied+1, l.firstIndex())
if l.committed > off { if l.committed+1 > off {
return l.slice(off+1, l.committed+1) return l.slice(off, l.committed+1)
} }
return nil 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) { func (l *raftLog) commitTo(tocommit uint64) {
// never decrease commit // never decrease commit
if l.committed < tocommit { if l.committed < tocommit {
if l.lastIndex() < 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 l.committed = tocommit
} }
@ -136,36 +176,52 @@ func (l *raftLog) appliedTo(i uint64) {
return return
} }
if l.committed < i || i < l.applied { 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 l.applied = i
} }
func (l *raftLog) stableTo(i uint64) { func (l *raftLog) stableTo(i uint64) {
if i < l.unstable || i+1-l.unstable > uint64(len(l.unstableEnts)) {
log.Panicf("stableTo(%d) is out of range [unstable(%d), len(unstableEnts)(%d)]",
i, l.unstable, len(l.unstableEnts))
}
l.unstableEnts = l.unstableEnts[i+1-l.unstable:]
l.unstable = i + 1 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 { func (l *raftLog) term(i uint64) uint64 {
if e := l.at(i); e != nil { switch {
return e.Term 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 { 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) 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 // 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. // 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 // 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 { func (l *raftLog) matchTerm(i, term uint64) bool {
if e := l.at(i); e != nil { return l.term(i) == term
return e.Term == term
}
return false
} }
func (l *raftLog) maybeCommit(maxIndex, term uint64) bool { func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
@ -191,43 +244,14 @@ func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
return false 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) { func (l *raftLog) restore(s pb.Snapshot) {
l.ents = []pb.Entry{{Term: s.Term}} err := l.storage.ApplySnapshot(s)
l.unstable = s.Index + 1 if err != nil {
l.committed = s.Index panic(err) // TODO(bdarnell)
l.offset = s.Index
l.snapshot = s
}
func (l *raftLog) at(i uint64) *pb.Entry {
if l.isOutOfBounds(i) {
return nil
} }
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. // 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) { if l.isOutOfBounds(lo) || l.isOutOfBounds(hi-1) {
return nil 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 { func (l *raftLog) isOutOfBounds(i uint64) bool {
if i < l.offset || i > l.lastIndex() { if i < l.firstIndex() || i > l.lastIndex() {
return true
}
return false
}
func (l *raftLog) isOutOfAppliedBounds(i uint64) bool {
if i < l.offset || i > l.applied {
return true return true
} }
return false return false

View File

@ -24,7 +24,7 @@ import (
) )
func TestFindConflict(t *testing.T) { 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 { tests := []struct {
from uint64 from uint64
ents []pb.Entry ents []pb.Entry
@ -34,22 +34,22 @@ func TestFindConflict(t *testing.T) {
{1, []pb.Entry{}, 0}, {1, []pb.Entry{}, 0},
{3, []pb.Entry{}, 0}, {3, []pb.Entry{}, 0},
// no conflict // no conflict
{1, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}, 0}, {1, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0},
{2, []pb.Entry{{Term: 2}, {Term: 3}}, 0}, {2, []pb.Entry{{Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0},
{3, []pb.Entry{{Term: 3}}, 0}, {3, []pb.Entry{{Index: 3, Term: 3}}, 0},
// no conflict, but has new entries // no conflict, but has new entries
{1, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}, {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{{Term: 2}, {Term: 3}, {Term: 4}, {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{{Term: 3}, {Term: 4}, {Term: 4}}, 4}, {3, []pb.Entry{{Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
{4, []pb.Entry{{Term: 4}, {Term: 4}}, 4}, {4, []pb.Entry{{Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
// conflicts with existing entries // conflicts with existing entries
{1, []pb.Entry{{Term: 4}, {Term: 4}}, 1}, {1, []pb.Entry{{Index: 1, Term: 4}, {Index: 2, Term: 4}}, 1},
{2, []pb.Entry{{Term: 1}, {Term: 4}, {Term: 4}}, 2}, {2, []pb.Entry{{Index: 2, Term: 1}, {Index: 3, Term: 4}, {Index: 4, Term: 4}}, 2},
{3, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 4}, {Term: 4}}, 3}, {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 { for i, tt := range tests {
raftLog := newLog() raftLog := newLog(NewMemoryStorage())
raftLog.append(raftLog.lastIndex(), previousEnts...) raftLog.append(raftLog.lastIndex(), previousEnts...)
gconflict := raftLog.findConflict(tt.from, tt.ents) gconflict := raftLog.findConflict(tt.from, tt.ents)
@ -60,8 +60,8 @@ func TestFindConflict(t *testing.T) {
} }
func TestIsUpToDate(t *testing.T) { func TestIsUpToDate(t *testing.T) {
previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}} previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
raftLog := newLog() raftLog := newLog(NewMemoryStorage())
raftLog.append(raftLog.lastIndex(), previousEnts...) raftLog.append(raftLog.lastIndex(), previousEnts...)
tests := []struct { tests := []struct {
lastIndex uint64 lastIndex uint64
@ -91,8 +91,7 @@ func TestIsUpToDate(t *testing.T) {
} }
func TestAppend(t *testing.T) { func TestAppend(t *testing.T) {
previousEnts := []pb.Entry{{Term: 1}, {Term: 2}} previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}
previousUnstable := uint64(3)
tests := []struct { tests := []struct {
after uint64 after uint64
ents []pb.Entry ents []pb.Entry
@ -104,38 +103,39 @@ func TestAppend(t *testing.T) {
2, 2,
[]pb.Entry{}, []pb.Entry{},
2, 2,
[]pb.Entry{{Term: 1}, {Term: 2}}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}},
3, 3,
}, },
{ {
2, 2,
[]pb.Entry{{Term: 2}}, []pb.Entry{{Index: 3, Term: 2}},
3, 3,
[]pb.Entry{{Term: 1}, {Term: 2}, {Term: 2}}, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 2}},
3, 3,
}, },
// conflicts with index 1 // conflicts with index 1
{ {
0, 0,
[]pb.Entry{{Term: 2}}, []pb.Entry{{Index: 1, Term: 2}},
1, 1,
[]pb.Entry{{Term: 2}}, []pb.Entry{{Index: 1, Term: 2}},
1, 1,
}, },
// conflicts with index 2 // conflicts with index 2
{ {
1, 1,
[]pb.Entry{{Term: 3}, {Term: 3}}, []pb.Entry{{Index: 2, Term: 3}, {Index: 3, Term: 3}},
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, 2,
}, },
} }
for i, tt := range tests { for i, tt := range tests {
raftLog := newLog() storage := NewMemoryStorage()
raftLog.append(raftLog.lastIndex(), previousEnts...) storage.Append(previousEnts)
raftLog.unstable = previousUnstable raftLog := newLog(storage)
index := raftLog.append(tt.after, tt.ents...) index := raftLog.append(tt.after, tt.ents...)
if index != tt.windex { if index != tt.windex {
t.Errorf("#%d: lastIndex = %d, want %d", i, 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: // If the given (index, term) does not match with the existing log:
// return false // return false
func TestLogMaybeAppend(t *testing.T) { 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) lastindex := uint64(3)
lastterm := uint64(3) lastterm := uint64(3)
commit := uint64(1) commit := uint64(1)
@ -176,12 +176,12 @@ func TestLogMaybeAppend(t *testing.T) {
}{ }{
// not match: term is different // 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, 0, false, commit, false,
}, },
// not match: index out of bound // 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, 0, false, commit, false,
}, },
// match with the last existing entry // match with the last existing entry
@ -206,42 +206,42 @@ func TestLogMaybeAppend(t *testing.T) {
0, true, commit, false, // commit do not decrease 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, 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, 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 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, lastindex + 2, true, lastindex + 2, false,
}, },
// match with the the entry in the middle // 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, 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, 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 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, lastindex, true, lastindex, false,
}, },
} }
for i, tt := range tests { for i, tt := range tests {
raftLog := newLog() raftLog := newLog(NewMemoryStorage())
raftLog.append(raftLog.lastIndex(), previousEnts...) raftLog.append(raftLog.lastIndex(), previousEnts...)
raftLog.committed = commit raftLog.committed = commit
func() { func() {
@ -278,44 +278,54 @@ func TestLogMaybeAppend(t *testing.T) {
// a compaction. // a compaction.
func TestCompactionSideEffects(t *testing.T) { func TestCompactionSideEffects(t *testing.T) {
var i uint64 var i uint64
// Populate the log with 1000 entries; 750 in stable storage and 250 in unstable.
lastIndex := uint64(1000) lastIndex := uint64(1000)
unstableIndex := uint64(750)
lastTerm := lastIndex lastTerm := lastIndex
raftLog := newLog() storage := NewMemoryStorage()
for i = 1; i <= unstableIndex; i++ {
for i = 0; i < lastIndex; i++ { storage.Append([]pb.Entry{{Term: uint64(i), Index: uint64(i)}})
raftLog.append(uint64(i), pb.Entry{Term: uint64(i + 1), Index: uint64(i + 1)}) }
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.appliedTo(raftLog.committed)
raftLog.compact(500) offset := uint64(500)
storage.Compact(offset, nil, nil)
if raftLog.lastIndex() != lastIndex { if raftLog.lastIndex() != lastIndex {
t.Errorf("lastIndex = %d, want %d", 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 { if raftLog.term(i) != i {
t.Errorf("term(%d) = %d, want %d", i, 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) { if !raftLog.matchTerm(i, i) {
t.Errorf("matchTerm(%d) = false, want true", i) t.Errorf("matchTerm(%d) = false, want true", i)
} }
} }
unstableEnts := raftLog.unstableEnts() unstableEnts := raftLog.unstableEntries()
if g := len(unstableEnts); g != 500 { if g := len(unstableEnts); g != 250 {
t.Errorf("len(unstableEntries) = %d, want = %d", g, 500) t.Errorf("len(unstableEntries) = %d, want = %d", g, 250)
} }
if unstableEnts[0].Index != 501 { if unstableEnts[0].Index != 751 {
t.Errorf("Index = %d, want = %d", unstableEnts[0].Index, 501) t.Errorf("Index = %d, want = %d", unstableEnts[0].Index, 751)
} }
prev := raftLog.lastIndex() 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 { if raftLog.lastIndex() != prev+1 {
t.Errorf("lastIndex = %d, want = %d", 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) { 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{ ents := []pb.Entry{
{Term: 1, Index: 3},
{Term: 1, Index: 4}, {Term: 1, Index: 4},
{Term: 1, Index: 5}, {Term: 1, Index: 5},
{Term: 1, Index: 6}, {Term: 1, Index: 6},
@ -338,15 +349,16 @@ func TestNextEnts(t *testing.T) {
applied uint64 applied uint64
wents []pb.Entry wents []pb.Entry
}{ }{
{0, ents[1:3]}, {0, ents[:2]},
{3, ents[1:3]}, {3, ents[:2]},
{4, ents[2:3]}, {4, ents[1:2]},
{5, nil}, {5, nil},
} }
for i, tt := range tests { for i, tt := range tests {
raftLog := newLog() storage := NewMemoryStorage()
raftLog.restore(snap) storage.ApplySnapshot(snap)
raftLog.load(ents) raftLog := newLog(storage)
raftLog.append(snap.Metadata.Index, ents...)
raftLog.maybeCommit(5, 1) raftLog.maybeCommit(5, 1)
raftLog.appliedTo(tt.applied) 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) { func TestUnstableEnts(t *testing.T) {
previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}} previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}
tests := []struct { tests := []struct {
unstable uint64 unstable uint64
wents []pb.Entry wents []pb.Entry
wunstable uint64
}{ }{
{3, nil, 3}, {3, nil},
{1, previousEnts, 3}, {1, previousEnts},
{0, append([]pb.Entry{{}}, previousEnts...), 3},
} }
for i, tt := range tests { for i, tt := range tests {
raftLog := newLog() // append stable entries to storage
raftLog.append(0, previousEnts...) storage := NewMemoryStorage()
raftLog.unstable = tt.unstable storage.Append(previousEnts[:tt.unstable-1])
ents := raftLog.unstableEnts()
// 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 { if l := len(ents); l > 0 {
raftLog.stableTo(ents[l-1].Index) raftLog.stableTo(ents[l-1].Index)
} }
if !reflect.DeepEqual(ents, tt.wents) { if !reflect.DeepEqual(ents, tt.wents) {
t.Errorf("#%d: unstableEnts = %+v, want %+v", i, ents, tt.wents) t.Errorf("#%d: unstableEnts = %+v, want %+v", i, ents, tt.wents)
} }
if g := raftLog.unstable; g != tt.wunstable { w := previousEnts[len(previousEnts)-1].Index + 1
t.Errorf("#%d: unstable = %d, want %d", i, g, tt.wunstable) 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.append(0, previousEnts...)
raftLog.committed = commit raftLog.committed = commit
raftLog.commitTo(tt.commit) raftLog.commitTo(tt.commit)
@ -423,12 +441,12 @@ func TestStableTo(t *testing.T) {
stable uint64 stable uint64
wunstable uint64 wunstable uint64
}{ }{
{0, 1},
{1, 2}, {1, 2},
{2, 3}, {2, 3},
} }
for i, tt := range tests { for i, tt := range tests {
raftLog := newLog() raftLog := newLog(NewMemoryStorage())
raftLog.append(0, []pb.Entry{{}, {}}...)
raftLog.stableTo(tt.stable) raftLog.stableTo(tt.stable)
if raftLog.unstable != tt.wunstable { if raftLog.unstable != tt.wunstable {
t.Errorf("#%d: unstable = %d, want %d", i, 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) { func TestCompaction(t *testing.T) {
tests := []struct { tests := []struct {
applied uint64
lastIndex uint64 lastIndex uint64
compact []uint64 compact []uint64
wleft []int wleft []int
wallow bool wallow bool
}{ }{
// out of upper bound // out of upper bound
{1000, 1000, []uint64{1001}, []int{-1}, false}, {1000, []uint64{1001}, []int{-1}, false},
{1000, 1000, []uint64{300, 500, 800, 900}, []int{701, 501, 201, 101}, true}, {1000, []uint64{300, 500, 800, 900}, []int{700, 500, 200, 100}, true},
// out of lower bound // out of lower bound
{1000, 1000, []uint64{300, 299}, []int{701, -1}, false}, {1000, []uint64{300, 299}, []int{700, -1}, false},
{0, 1000, []uint64{1}, []int{-1}, false},
} }
for i, tt := range tests { for i, tt := range tests {
@ -458,22 +474,23 @@ func TestCompaction(t *testing.T) {
defer func() { defer func() {
if r := recover(); r != nil { if r := recover(); r != nil {
if tt.wallow == true { if tt.wallow == true {
t.Errorf("%d: allow = %v, want %v", i, false, true) t.Errorf("%d: allow = %v, want %v: %v", i, false, true, r)
} }
} }
}() }()
raftLog := newLog() storage := NewMemoryStorage()
for i := uint64(0); i < tt.lastIndex; i++ { for i := uint64(1); i <= tt.lastIndex; i++ {
raftLog.append(uint64(i), pb.Entry{}) storage.Append([]pb.Entry{{Index: i}})
} }
raftLog.maybeCommit(tt.applied, 0) raftLog := newLog(storage)
raftLog.maybeCommit(tt.lastIndex, 0)
raftLog.appliedTo(raftLog.committed) raftLog.appliedTo(raftLog.committed)
for j := 0; j < len(tt.compact); j++ { for j := 0; j < len(tt.compact); j++ {
raftLog.compact(tt.compact[j]) storage.Compact(tt.compact[j], nil, nil)
if len(raftLog.ents) != tt.wleft[j] { if len(raftLog.allEntries()) != tt.wleft[j] {
t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.ents), 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) { 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) index := uint64(1000)
term := uint64(1000) term := uint64(1000)
raftLog.restore(pb.Snapshot{Index: index, Term: term}) snap := pb.SnapshotMetadata{Index: index, Term: term}
storage := NewMemoryStorage()
storage.ApplySnapshot(pb.Snapshot{Metadata: snap})
raftLog := newLog(storage)
// only has the guard entry if len(raftLog.allEntries()) != 0 {
if len(raftLog.ents) != 1 { t.Errorf("len = %d, want 0", len(raftLog.allEntries()))
t.Errorf("len = %d, want 0", len(raftLog.ents))
} }
if raftLog.offset != index { if raftLog.firstIndex() != index+1 {
t.Errorf("offset = %d, want %d", raftLog.offset, index) t.Errorf("firstIndex = %d, want %d", raftLog.firstIndex(), index+1)
} }
if raftLog.committed != index { if raftLog.committed != index {
t.Errorf("comitted = %d, want %d", 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) { func TestIsOutOfBounds(t *testing.T) {
offset := uint64(100) offset := uint64(100)
num := 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 { tests := []struct {
index uint64 index uint64
w bool w bool
}{ }{
{offset - 1, true}, {offset - 1, true},
{offset, false}, {offset, true},
{offset + num/2, false}, {offset + num/2, false},
{offset + num - 1, false}, {offset + num, false},
{offset + num, true}, {offset + num + 1, true},
} }
for i, tt := range tests { 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 var i uint64
offset := uint64(100) offset := uint64(100)
num := uint64(100) num := uint64(100)
l := &raftLog{offset: offset} storage := NewMemoryStorage()
for i = 0; i < num; i++ { storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
l.ents = append(l.ents, pb.Entry{Term: i}) l := newLog(storage)
for i = 1; i < num; i++ {
l.append(offset+i-1, pb.Entry{Index: i, Term: i})
} }
tests := []struct { tests := []struct {
index uint64 index uint64
w *pb.Entry w uint64
}{ }{
{offset - 1, nil}, {offset - 1, 0},
{offset, &pb.Entry{Term: 0}}, {offset, 0},
{offset + num/2, &pb.Entry{Term: num / 2}}, {offset + num/2, num / 2},
{offset + num - 1, &pb.Entry{Term: num - 1}}, {offset + num - 1, num - 1},
{offset + num, nil}, {offset + num, 0},
} }
for i, tt := range tests { for i, tt := range tests {
g := l.at(tt.index) term := l.term(tt.index)
if !reflect.DeepEqual(g, tt.w) { if !reflect.DeepEqual(term, tt.w) {
t.Errorf("#%d: at = %v, want %v", i, g, 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) offset := uint64(100)
num := uint64(100) num := uint64(100)
l := &raftLog{offset: offset} storage := NewMemoryStorage()
for i = 0; i < num; i++ { storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
l.ents = append(l.ents, pb.Entry{Term: i}) l := newLog(storage)
for i = 1; i < num; i++ {
l.append(offset+i-1, pb.Entry{Index: i, Term: i})
} }
tests := []struct { tests := []struct {
@ -578,9 +598,9 @@ func TestSlice(t *testing.T) {
w []pb.Entry w []pb.Entry
}{ }{
{offset - 1, offset + 1, nil}, {offset - 1, offset + 1, nil},
{offset, offset + 1, []pb.Entry{{Term: 0}}}, {offset, offset + 1, nil},
{offset + num/2, offset + num/2 + 1, []pb.Entry{{Term: num / 2}}}, {offset + num/2, offset + num/2 + 1, []pb.Entry{{Index: num / 2, Term: num / 2}}},
{offset + num - 1, offset + num, []pb.Entry{{Term: num - 1}}}, {offset + num - 1, offset + num, []pb.Entry{{Index: num - 1, Term: num - 1}}},
{offset + num, offset + num + 1, nil}, {offset + num, offset + num + 1, nil},
{offset + num/2, offset + num/2, nil}, {offset + num/2, offset + num/2, nil},

View File

@ -75,12 +75,6 @@ type Ready struct {
Messages []pb.Message Messages []pb.Message
} }
type compact struct {
index uint64
nodes []uint64
data []byte
}
func isHardStateEqual(a, b pb.HardState) bool { func isHardStateEqual(a, b pb.HardState) bool {
return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit 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. // IsEmptySnap returns true if the given Snapshot is empty.
func IsEmptySnap(sp pb.Snapshot) bool { func IsEmptySnap(sp pb.Snapshot) bool {
return sp.Index == 0 return sp.Metadata.Index == 0
} }
func (rd Ready) containsUpdates() bool { func (rd Ready) containsUpdates() bool {
return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) || !IsEmptySnap(rd.Snapshot) || return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) ||
len(rd.Entries) > 0 || len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0 !IsEmptySnap(rd.Snapshot) || len(rd.Entries) > 0 ||
len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
} }
// Node represents a node in a raft cluster. // 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. // It prepares the node to return the next available Ready.
Advance() Advance()
// ApplyConfChange applies config change to the local node. // ApplyConfChange applies config change to the local node.
// TODO: reject existing node when add node // Returns an opaque ConfState protobuf which must be recorded
// TODO: reject non-existant node when remove node // in snapshots. Will never return nil; it returns a pointer only
ApplyConfChange(cc pb.ConfChange) // to match MemoryStorage.Compact.
ApplyConfChange(cc pb.ConfChange) *pb.ConfState
// Stop performs any necessary termination of the Node // Stop performs any necessary termination of the Node
Stop() 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 { 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 // 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. // the election and heartbeat timeouts in units of ticks.
// It appends a ConfChangeAddNode entry for each given peer to the initial log. // 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() n := newNode()
r := newRaft(id, nil, election, heartbeat) r := newRaft(id, nil, election, heartbeat, storage)
for _, peer := range peers { for _, peer := range peers {
cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context} 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} e := pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: r.raftLog.lastIndex() + 1, Data: d}
r.raftLog.append(r.raftLog.lastIndex(), e) 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() r.raftLog.committed = r.raftLog.lastIndex()
go n.run(r) go n.run(r)
return &n return &n
} }
// RestartNode is identical to StartNode but takes an initial State and a slice // RestartNode is identical to StartNode but does not take a list of peers.
// of entries. Generally this is used when restarting from a stable storage // The current membership of the cluster will be restored from the Storage.
// log. func RestartNode(id uint64, election, heartbeat int, storage Storage) Node {
func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, ents []pb.Entry) Node {
n := newNode() n := newNode()
r := newRaft(id, nil, election, heartbeat) r := newRaft(id, nil, election, heartbeat, storage)
if snapshot != nil {
r.restore(*snapshot)
r.raftLog.appliedTo(snapshot.Index)
}
if !isHardStateEqual(st, emptyState) {
r.loadState(st)
}
if len(ents) != 0 {
r.loadEnts(ents)
}
go n.run(r) go n.run(r)
return &n return &n
} }
// node is the canonical implementation of the Node interface // node is the canonical implementation of the Node interface
type node struct { type node struct {
propc chan pb.Message propc chan pb.Message
recvc chan pb.Message recvc chan pb.Message
compactc chan compact confc chan pb.ConfChange
confc chan pb.ConfChange confstatec chan pb.ConfState
readyc chan Ready readyc chan Ready
advancec chan struct{} advancec chan struct{}
tickc chan struct{} tickc chan struct{}
done chan struct{} done chan struct{}
stop chan struct{} stop chan struct{}
} }
func newNode() node { func newNode() node {
return node{ return node{
propc: make(chan pb.Message), propc: make(chan pb.Message),
recvc: make(chan pb.Message), recvc: make(chan pb.Message),
compactc: make(chan compact), confc: make(chan pb.ConfChange),
confc: make(chan pb.ConfChange), confstatec: make(chan pb.ConfState),
readyc: make(chan Ready), readyc: make(chan Ready),
advancec: make(chan struct{}), advancec: make(chan struct{}),
tickc: make(chan struct{}), tickc: make(chan struct{}),
done: make(chan struct{}), done: make(chan struct{}),
stop: make(chan struct{}), stop: make(chan struct{}),
} }
} }
@ -228,18 +210,18 @@ func (n *node) run(r *raft) {
var advancec chan struct{} var advancec chan struct{}
var prevLastUnstablei uint64 var prevLastUnstablei uint64
var havePrevLastUnstablei bool var havePrevLastUnstablei bool
var prevSnapi uint64
var rd Ready var rd Ready
lead := None lead := None
prevSoftSt := r.softState() prevSoftSt := r.softState()
prevHardSt := r.HardState prevHardSt := r.HardState
prevSnapi := r.raftLog.snapshot.Index
for { for {
if advancec != nil { if advancec != nil {
readyc = nil readyc = nil
} else { } else {
rd = newReady(r, prevSoftSt, prevHardSt, prevSnapi) rd = newReady(r, prevSoftSt, prevHardSt)
if rd.containsUpdates() { if rd.containsUpdates() {
readyc = n.readyc readyc = n.readyc
} else { } else {
@ -271,11 +253,13 @@ func (n *node) run(r *raft) {
r.Step(m) r.Step(m)
case m := <-n.recvc: case m := <-n.recvc:
r.Step(m) // raft never returns an error r.Step(m) // raft never returns an error
case c := <-n.compactc:
r.compact(c.index, c.nodes, c.data)
case cc := <-n.confc: case cc := <-n.confc:
if cc.NodeID == None { if cc.NodeID == None {
r.resetPendingConf() r.resetPendingConf()
select {
case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
case <-n.done:
}
break break
} }
switch cc.Type { switch cc.Type {
@ -288,6 +272,10 @@ func (n *node) run(r *raft) {
default: default:
panic("unexpected conf type") panic("unexpected conf type")
} }
select {
case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
case <-n.done:
}
case <-n.tickc: case <-n.tickc:
r.tick() r.tick()
case readyc <- rd: case readyc <- rd:
@ -302,11 +290,11 @@ func (n *node) run(r *raft) {
prevHardSt = rd.HardState prevHardSt = rd.HardState
} }
if !IsEmptySnap(rd.Snapshot) { if !IsEmptySnap(rd.Snapshot) {
prevSnapi = rd.Snapshot.Index if rd.Snapshot.Metadata.Index > prevLastUnstablei {
if prevSnapi > prevLastUnstablei { prevLastUnstablei = rd.Snapshot.Metadata.Index
prevLastUnstablei = prevSnapi
havePrevLastUnstablei = true havePrevLastUnstablei = true
} }
prevSnapi = rd.Snapshot.Metadata.Index
} }
r.msgs = nil r.msgs = nil
advancec = n.advancec advancec = n.advancec
@ -318,6 +306,9 @@ func (n *node) run(r *raft) {
r.raftLog.stableTo(prevLastUnstablei) r.raftLog.stableTo(prevLastUnstablei)
havePrevLastUnstablei = false havePrevLastUnstablei = false
} }
if r.snapshot != nil && r.snapshot.Metadata.Index == prevSnapi {
r.snapshot = nil
}
advancec = nil advancec = nil
case <-n.stop: case <-n.stop:
close(n.done) 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 { select {
case n.confc <- cc: case n.confc <- cc:
case <-n.done: case <-n.done:
} }
}
func (n *node) Compact(index uint64, nodes []uint64, d []byte) {
select { select {
case n.compactc <- compact{index, nodes, d}: case cs = <-n.confstatec:
case <-n.done: 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{ rd := Ready{
Entries: r.raftLog.unstableEnts(), Entries: r.raftLog.unstableEntries(),
CommittedEntries: r.raftLog.nextEnts(), CommittedEntries: r.raftLog.nextEnts(),
Messages: r.msgs, Messages: r.msgs,
} }
@ -415,8 +405,8 @@ func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState, prevSnapi
if !isHardStateEqual(r.HardState, prevHardSt) { if !isHardStateEqual(r.HardState, prevHardSt) {
rd.HardState = r.HardState rd.HardState = r.HardState
} }
if prevSnapi != r.raftLog.snapshot.Index { if r.snapshot != nil {
rd.Snapshot = r.raftLog.snapshot rd.Snapshot = *r.snapshot
} }
return rd return rd
} }

View File

@ -27,7 +27,7 @@ func BenchmarkOneNode(b *testing.B) {
defer cancel() defer cancel()
n := newNode() n := newNode()
r := newRaft(1, []uint64{1}, 10, 1) r := newRaft(1, []uint64{1}, 10, 1, nil)
go n.run(r) go n.run(r)
defer n.Stop() defer n.Stop()

View File

@ -115,11 +115,13 @@ func TestNodePropose(t *testing.T) {
} }
n := newNode() n := newNode()
r := newRaft(1, []uint64{1}, 10, 1) s := NewMemoryStorage()
r := newRaft(1, []uint64{1}, 10, 1, s)
go n.run(r) go n.run(r)
n.Campaign(context.TODO()) n.Campaign(context.TODO())
for { for {
rd := <-n.Ready() rd := <-n.Ready()
s.Append(rd.Entries)
// change the step function to appendStep until this raft becomes leader // change the step function to appendStep until this raft becomes leader
if rd.SoftState.Lead == r.id { if rd.SoftState.Lead == r.id {
r.step = appendStep r.step = appendStep
@ -151,11 +153,13 @@ func TestNodeProposeConfig(t *testing.T) {
} }
n := newNode() n := newNode()
r := newRaft(1, []uint64{1}, 10, 1) s := NewMemoryStorage()
r := newRaft(1, []uint64{1}, 10, 1, s)
go n.run(r) go n.run(r)
n.Campaign(context.TODO()) n.Campaign(context.TODO())
for { for {
rd := <-n.Ready() rd := <-n.Ready()
s.Append(rd.Entries)
// change the step function to appendStep until this raft becomes leader // change the step function to appendStep until this raft becomes leader
if rd.SoftState.Lead == r.id { if rd.SoftState.Lead == r.id {
r.step = appendStep r.step = appendStep
@ -188,7 +192,7 @@ func TestNodeProposeConfig(t *testing.T) {
// who is the current leader. // who is the current leader.
func TestBlockProposal(t *testing.T) { func TestBlockProposal(t *testing.T) {
n := newNode() n := newNode()
r := newRaft(1, []uint64{1}, 10, 1) r := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
go n.run(r) go n.run(r)
defer n.Stop() defer n.Stop()
@ -220,7 +224,8 @@ func TestBlockProposal(t *testing.T) {
// elapsed of the underlying raft state machine. // elapsed of the underlying raft state machine.
func TestNodeTick(t *testing.T) { func TestNodeTick(t *testing.T) {
n := newNode() n := newNode()
r := newRaft(1, []uint64{1}, 10, 1) s := NewMemoryStorage()
r := newRaft(1, []uint64{1}, 10, 1, s)
go n.run(r) go n.run(r)
elapsed := r.elapsed elapsed := r.elapsed
n.Tick() n.Tick()
@ -234,7 +239,8 @@ func TestNodeTick(t *testing.T) {
// processing, and that it is idempotent // processing, and that it is idempotent
func TestNodeStop(t *testing.T) { func TestNodeStop(t *testing.T) {
n := newNode() n := newNode()
r := newRaft(1, []uint64{1}, 10, 1) s := NewMemoryStorage()
r := newRaft(1, []uint64{1}, 10, 1, s)
donec := make(chan struct{}) donec := make(chan struct{})
go func() { go func() {
@ -275,7 +281,7 @@ func TestReadyContainUpdates(t *testing.T) {
{Ready{Entries: make([]raftpb.Entry, 1, 1)}, true}, {Ready{Entries: make([]raftpb.Entry, 1, 1)}, true},
{Ready{CommittedEntries: make([]raftpb.Entry, 1, 1)}, true}, {Ready{CommittedEntries: make([]raftpb.Entry, 1, 1)}, true},
{Ready{Messages: make([]raftpb.Message, 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 { for i, tt := range tests {
@ -302,7 +308,6 @@ func TestNodeStart(t *testing.T) {
SoftState: &SoftState{Lead: 1, Nodes: []uint64{1}, RaftState: StateLeader}, SoftState: &SoftState{Lead: 1, Nodes: []uint64{1}, RaftState: StateLeader},
HardState: raftpb.HardState{Term: 1, Commit: 2}, HardState: raftpb.HardState{Term: 1, Commit: 2},
Entries: []raftpb.Entry{ Entries: []raftpb.Entry{
{},
{Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata}, {Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata},
{Term: 1, Index: 2}, {Term: 1, Index: 2},
}, },
@ -317,12 +322,15 @@ func TestNodeStart(t *testing.T) {
CommittedEntries: []raftpb.Entry{{Term: 1, Index: 3, Data: []byte("foo")}}, 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.ApplyConfChange(cc)
n.Campaign(ctx) n.Campaign(ctx)
if g := <-n.Ready(); !reflect.DeepEqual(g, wants[0]) { g := <-n.Ready()
t.Errorf("#%d: g = %+v,\n w %+v", 1, g, wants[0]) if !reflect.DeepEqual(g, wants[0]) {
t.Fatalf("#%d: g = %+v,\n w %+v", 1, g, wants[0])
} else { } else {
storage.Append(g.Entries)
n.Advance() n.Advance()
} }
@ -330,6 +338,7 @@ func TestNodeStart(t *testing.T) {
if g := <-n.Ready(); !reflect.DeepEqual(g, wants[1]) { if g := <-n.Ready(); !reflect.DeepEqual(g, wants[1]) {
t.Errorf("#%d: g = %+v,\n w %+v", 2, g, wants[1]) t.Errorf("#%d: g = %+v,\n w %+v", 2, g, wants[1])
} else { } else {
storage.Append(g.Entries)
n.Advance() n.Advance()
} }
@ -342,7 +351,6 @@ func TestNodeStart(t *testing.T) {
func TestNodeRestart(t *testing.T) { func TestNodeRestart(t *testing.T) {
entries := []raftpb.Entry{ entries := []raftpb.Entry{
{},
{Term: 1, Index: 1}, {Term: 1, Index: 1},
{Term: 1, Index: 2, Data: []byte("foo")}, {Term: 1, Index: 2, Data: []byte("foo")},
} }
@ -351,15 +359,17 @@ func TestNodeRestart(t *testing.T) {
want := Ready{ want := Ready{
HardState: emptyState, HardState: emptyState,
// commit upto index commit index in st // 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) { if g := <-n.Ready(); !reflect.DeepEqual(g, want) {
t.Errorf("g = %+v,\n w %+v", g, want) t.Errorf("g = %+v,\n w %+v", g, want)
} else {
n.Advance()
} }
n.Advance()
select { select {
case rd := <-n.Ready(): case rd := <-n.Ready():
@ -369,14 +379,14 @@ func TestNodeRestart(t *testing.T) {
} }
func TestNodeRestartFromSnapshot(t *testing.T) { func TestNodeRestartFromSnapshot(t *testing.T) {
snap := &raftpb.Snapshot{ snap := raftpb.Snapshot{
Data: []byte("some data"), Metadata: raftpb.SnapshotMetadata{
Nodes: []uint64{1, 2}, ConfState: raftpb.ConfState{Nodes: []uint64{1, 2}},
Index: 2, Index: 2,
Term: 1, Term: 1,
},
} }
entries := []raftpb.Entry{ entries := []raftpb.Entry{
{Term: 1, Index: 2},
{Term: 1, Index: 3, Data: []byte("foo")}, {Term: 1, Index: 3, Data: []byte("foo")},
} }
st := raftpb.HardState{Term: 1, Commit: 3} st := raftpb.HardState{Term: 1, Commit: 3}
@ -384,10 +394,14 @@ func TestNodeRestartFromSnapshot(t *testing.T) {
want := Ready{ want := Ready{
HardState: emptyState, HardState: emptyState,
// commit upto index commit index in st // 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) { if g := <-n.Ready(); !reflect.DeepEqual(g, want) {
t.Errorf("g = %+v,\n w %+v", g, want) t.Errorf("g = %+v,\n w %+v", g, want)
} else { } 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) { func TestNodeAdvance(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() 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.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1})
n.Campaign(ctx) n.Campaign(ctx)
<-n.Ready() <-n.Ready()
n.Propose(ctx, []byte("foo")) n.Propose(ctx, []byte("foo"))
var rd Ready
select { select {
case rd := <-n.Ready(): case rd = <-n.Ready():
t.Fatalf("unexpected Ready before Advance: %+v", rd) t.Fatalf("unexpected Ready before Advance: %+v", rd)
case <-time.After(time.Millisecond): case <-time.After(time.Millisecond):
} }
storage.Append(rd.Entries)
n.Advance() n.Advance()
select { select {
case <-n.Ready(): case <-n.Ready():

View File

@ -123,6 +123,9 @@ type raft struct {
msgs []pb.Message msgs []pb.Message
// the incoming snapshot, if any.
snapshot *pb.Snapshot
// the leader id // the leader id
lead uint64 lead uint64
@ -137,21 +140,38 @@ type raft struct {
step stepFunc 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 { if id == None {
panic("cannot use none id") 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{ r := &raft{
id: id, id: id,
lead: None, lead: None,
raftLog: newLog(), raftLog: log,
prs: make(map[uint64]*progress), prs: make(map[uint64]*progress),
electionTimeout: election, electionTimeout: election,
heartbeatTimeout: heartbeat, heartbeatTimeout: heartbeat,
} }
r.rand = rand.New(rand.NewSource(int64(id))) r.rand = rand.New(rand.NewSource(int64(id)))
for _, p := range peers { 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) r.becomeFollower(0, None)
return r return r
@ -207,12 +227,19 @@ func (r *raft) sendAppend(to uint64) {
pr := r.prs[to] pr := r.prs[to]
m := pb.Message{} m := pb.Message{}
m.To = to m.To = to
m.Index = pr.next - 1 if r.needSnapshot(pr.next) {
if r.needSnapshot(m.Index) {
m.Type = pb.MsgSnap 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 { } else {
m.Type = pb.MsgApp m.Type = pb.MsgApp
m.Index = pr.next - 1
m.LogTerm = r.raftLog.term(pr.next - 1) m.LogTerm = r.raftLog.term(pr.next - 1)
m.Entries = r.raftLog.entries(pr.next) m.Entries = r.raftLog.entries(pr.next)
m.Commit = r.raftLog.committed m.Commit = r.raftLog.committed
@ -418,6 +445,7 @@ func (r *raft) handleHeartbeat(m pb.Message) {
func (r *raft) handleSnapshot(m pb.Message) { func (r *raft) handleSnapshot(m pb.Message) {
if r.restore(m.Snapshot) { if r.restore(m.Snapshot) {
r.snapshot = &m.Snapshot
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()}) r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
} else { } else {
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed}) 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 // restore recovers the statemachine from a snapshot. It restores the log and the
// configuration of statemachine. // configuration of statemachine.
func (r *raft) restore(s pb.Snapshot) bool { func (r *raft) restore(s pb.Snapshot) bool {
if s.Index <= r.raftLog.committed { if s.Metadata.Index <= r.raftLog.committed {
return false return false
} }
r.raftLog.restore(s) r.raftLog.restore(s)
r.prs = make(map[uint64]*progress) r.prs = make(map[uint64]*progress)
for _, n := range s.Nodes { for _, n := range s.Metadata.ConfState.Nodes {
if n == r.id { if n == r.id {
r.setProgress(n, r.raftLog.lastIndex(), r.raftLog.lastIndex()+1) r.setProgress(n, r.raftLog.lastIndex(), r.raftLog.lastIndex()+1)
} else { } else {
@ -562,13 +578,7 @@ func (r *raft) restore(s pb.Snapshot) bool {
} }
func (r *raft) needSnapshot(i uint64) bool { func (r *raft) needSnapshot(i uint64) bool {
if i < r.raftLog.offset { return i < r.raftLog.firstIndex()
if r.raftLog.snapshot.Term == 0 {
panic("need non-empty snapshot")
}
return true
}
return false
} }
func (r *raft) nodes() []uint64 { func (r *raft) nodes() []uint64 {
@ -595,10 +605,6 @@ func (r *raft) promotable() bool {
return ok return ok
} }
func (r *raft) loadEnts(ents []pb.Entry) {
r.raftLog.load(ents)
}
func (r *raft) loadState(state pb.HardState) { func (r *raft) loadState(state pb.HardState) {
r.raftLog.committed = state.Commit r.raftLog.committed = state.Commit
r.Term = state.Term r.Term = state.Term

View File

@ -52,7 +52,7 @@ func TestLeaderUpdateTermFromMessage(t *testing.T) {
// it immediately reverts to follower state. // it immediately reverts to follower state.
// Reference: section 5.1 // Reference: section 5.1
func testUpdateTermFromMessage(t *testing.T, state StateType) { 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 { switch state {
case StateFollower: case StateFollower:
r.becomeFollower(1, 2) r.becomeFollower(1, 2)
@ -82,7 +82,7 @@ func TestRejectStaleTermMessage(t *testing.T) {
fakeStep := func(r *raft, m pb.Message) { fakeStep := func(r *raft, m pb.Message) {
called = true 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.step = fakeStep
r.loadState(pb.HardState{Term: 2}) 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. // TestStartAsFollower tests that when servers start up, they begin as followers.
// Reference: section 5.2 // Reference: section 5.2
func TestStartAsFollower(t *testing.T) { 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 { if r.state != StateFollower {
t.Errorf("state = %s, want %s", 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) { func TestLeaderBcastBeat(t *testing.T) {
// heartbeat interval // heartbeat interval
hi := 1 hi := 1
r := newRaft(1, []uint64{1, 2, 3}, 10, hi) r := newRaft(1, []uint64{1, 2, 3}, 10, hi, NewMemoryStorage())
r.becomeCandidate() r.becomeCandidate()
r.becomeLeader() r.becomeLeader()
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
r.appendEntry(pb.Entry{}) r.appendEntry(pb.Entry{Index: uint64(i) + 1})
} }
for i := 0; i <= hi; i++ { for i := 0; i <= hi; i++ {
@ -151,7 +151,7 @@ func TestCandidateStartNewElection(t *testing.T) {
func testNonleaderStartElection(t *testing.T, state StateType) { func testNonleaderStartElection(t *testing.T, state StateType) {
// election timeout // election timeout
et := 10 et := 10
r := newRaft(1, []uint64{1, 2, 3}, et, 1) r := newRaft(1, []uint64{1, 2, 3}, et, 1, NewMemoryStorage())
switch state { switch state {
case StateFollower: case StateFollower:
r.becomeFollower(1, 2) r.becomeFollower(1, 2)
@ -215,7 +215,7 @@ func TestLeaderElectionInOneRoundRPC(t *testing.T) {
{5, map[uint64]bool{}, StateCandidate}, {5, map[uint64]bool{}, StateCandidate},
} }
for i, tt := range tests { 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}) r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
for id, vote := range tt.votes { for id, vote := range tt.votes {
@ -248,7 +248,7 @@ func TestFollowerVote(t *testing.T) {
{2, 1, true}, {2, 1, true},
} }
for i, tt := range tests { 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.loadState(pb.HardState{Term: 1, Vote: tt.vote})
r.Step(pb.Message{From: tt.nvote, To: 1, Term: 1, Type: pb.MsgVote}) 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}, {From: 2, To: 1, Term: 2, Type: pb.MsgApp},
} }
for i, tt := range tests { 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}) r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
if r.state != StateCandidate { if r.state != StateCandidate {
t.Fatalf("unexpected state = %s, want %s", 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 // Reference: section 5.2
func testNonleaderElectionTimeoutRandomized(t *testing.T, state StateType) { func testNonleaderElectionTimeoutRandomized(t *testing.T, state StateType) {
et := 10 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) timeouts := make(map[int]bool)
for round := 0; round < 50*et; round++ { for round := 0; round < 50*et; round++ {
switch state { switch state {
@ -345,7 +345,7 @@ func testNonleadersElectionTimeoutNonconflict(t *testing.T, state StateType) {
rs := make([]*raft, size) rs := make([]*raft, size)
ids := idsBySize(size) ids := idsBySize(size)
for k := range rs { for k := range rs {
rs[k] = newRaft(ids[k], ids, et, 1) rs[k] = newRaft(ids[k], ids, et, 1, NewMemoryStorage())
} }
conflicts := 0 conflicts := 0
for round := 0; round < 1000; round++ { 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. // Also, it writes the new entry into stable storage.
// Reference: section 5.3 // Reference: section 5.3
func TestLeaderStartReplication(t *testing.T) { 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.becomeCandidate()
r.becomeLeader() r.becomeLeader()
commitNoopEntry(r) commitNoopEntry(r, s)
li := r.raftLog.lastIndex() li := r.raftLog.lastIndex()
ents := []pb.Entry{{Data: []byte("some data")}} ents := []pb.Entry{{Data: []byte("some data")}}
@ -412,7 +413,7 @@ func TestLeaderStartReplication(t *testing.T) {
if !reflect.DeepEqual(msgs, wmsgs) { if !reflect.DeepEqual(msgs, wmsgs) {
t.Errorf("msgs = %+v, want %+v", 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) t.Errorf("ents = %+v, want %+v", g, wents)
} }
} }
@ -425,10 +426,11 @@ func TestLeaderStartReplication(t *testing.T) {
// servers eventually find out. // servers eventually find out.
// Reference: section 5.3 // Reference: section 5.3
func TestLeaderCommitEntry(t *testing.T) { 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.becomeCandidate()
r.becomeLeader() r.becomeLeader()
commitNoopEntry(r) commitNoopEntry(r, s)
li := r.raftLog.lastIndex() li := r.raftLog.lastIndex()
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) 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}, {5, map[uint64]bool{2: true, 3: true, 4: true, 5: true}, true},
} }
for i, tt := range tests { 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.becomeCandidate()
r.becomeLeader() r.becomeLeader()
commitNoopEntry(r) commitNoopEntry(r, s)
li := r.raftLog.lastIndex() li := r.raftLog.lastIndex()
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) 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}}, {{Term: 1, Index: 1}},
} }
for i, tt := range tests { for i, tt := range tests {
r := newRaft(1, []uint64{1, 2, 3}, 10, 1) storage := NewMemoryStorage()
r.loadEnts(append([]pb.Entry{{}}, tt...)) storage.Append(tt)
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage)
r.loadState(pb.HardState{Term: 2}) r.loadState(pb.HardState{Term: 2})
r.becomeCandidate() r.becomeCandidate()
r.becomeLeader() r.becomeLeader()
@ -566,7 +570,7 @@ func TestFollowerCommitEntry(t *testing.T) {
}, },
} }
for i, tt := range tests { 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.becomeFollower(1, 2)
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 1, Entries: tt.ents, Commit: tt.commit}) 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. // append entries.
// Reference: section 5.3 // Reference: section 5.3
func TestFollowerCheckMsgApp(t *testing.T) { 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 { tests := []struct {
term uint64 term uint64
index uint64 index uint64
wreject bool 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[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}, {3, 3, true},
} }
for i, tt := range tests { for i, tt := range tests {
r := newRaft(1, []uint64{1, 2, 3}, 10, 1) storage := NewMemoryStorage()
r.loadEnts(ents) storage.Append(ents)
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage)
r.loadState(pb.HardState{Commit: 2}) r.loadState(pb.HardState{Commit: 2})
r.becomeFollower(2, 2) r.becomeFollower(2, 2)
@ -632,39 +637,40 @@ func TestFollowerAppendEntries(t *testing.T) {
{ {
2, 2, 2, 2,
[]pb.Entry{{Term: 3, Index: 3}}, []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}}, []pb.Entry{{Term: 3, Index: 3}},
}, },
{ {
1, 1, 1, 1,
[]pb.Entry{{Term: 3, Index: 3}, {Term: 4, Index: 4}}, []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}}, []pb.Entry{{Term: 3, Index: 3}, {Term: 4, Index: 4}},
}, },
{ {
0, 0, 0, 0,
[]pb.Entry{{Term: 1, Index: 1}}, []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, nil,
}, },
{ {
0, 0, 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}},
[]pb.Entry{{Term: 3, Index: 3}}, []pb.Entry{{Term: 3, Index: 3}},
}, },
} }
for i, tt := range tests { for i, tt := range tests {
r := newRaft(1, []uint64{1, 2, 3}, 10, 1) storage := NewMemoryStorage()
r.loadEnts([]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}}) storage.Append([]pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}})
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage)
r.becomeFollower(2, 2) 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}) 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) 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) 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 { for i, tt := range tests {
lead := newRaft(1, []uint64{1, 2, 3}, 10, 1) leadStorage := NewMemoryStorage()
lead.loadEnts(ents) leadStorage.Append(ents)
lead := newRaft(1, []uint64{1, 2, 3}, 10, 1, leadStorage)
lead.loadState(pb.HardState{Commit: lead.raftLog.lastIndex(), Term: term}) lead.loadState(pb.HardState{Commit: lead.raftLog.lastIndex(), Term: term})
follower := newRaft(2, []uint64{1, 2, 3}, 10, 1) followerStorage := NewMemoryStorage()
follower.loadEnts(tt) followerStorage.Append(tt)
follower := newRaft(2, []uint64{1, 2, 3}, 10, 1, followerStorage)
follower.loadState(pb.HardState{Term: term - 1}) follower.loadState(pb.HardState{Term: term - 1})
// It is necessary to have a three-node cluster. // It is necessary to have a three-node cluster.
// The second may have more up-to-date log than the first one, so the // 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}, {[]pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}, 3},
} }
for i, tt := range tests { 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{ r.Step(pb.Message{
From: 2, To: 1, Type: pb.MsgApp, Term: tt.wterm - 1, LogTerm: 0, Index: 0, Entries: tt.ents, 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 wreject bool
}{ }{
// same logterm // same logterm
{[]pb.Entry{{}, {Term: 1, Index: 1}}, 1, 1, false}, {[]pb.Entry{{Term: 1, Index: 1}}, 1, 1, false},
{[]pb.Entry{{}, {Term: 1, Index: 1}}, 1, 2, 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}, {Term: 1, Index: 2}}, 1, 1, true},
// candidate higher logterm // candidate higher logterm
{[]pb.Entry{{}, {Term: 1, Index: 1}}, 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}}, 2, 2, false},
{[]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 1, Index: 2}}, 2, 1, false}, {[]pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}}, 2, 1, false},
// voter higher logterm // voter higher logterm
{[]pb.Entry{{}, {Term: 2, Index: 1}}, 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}}, 1, 2, true},
{[]pb.Entry{{}, {Term: 2, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true}, {[]pb.Entry{{Term: 2, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true},
} }
for i, tt := range tests { for i, tt := range tests {
r := newRaft(1, []uint64{1, 2}, 10, 1) storage := NewMemoryStorage()
r.loadEnts(tt.ents) 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}) 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. // current term are committed by counting replicas.
// Reference: section 5.4.2 // Reference: section 5.4.2
func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) { 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 { tests := []struct {
index uint64 index uint64
wcommit uint64 wcommit uint64
@ -852,8 +861,9 @@ func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) {
{3, 3}, {3, 3},
} }
for i, tt := range tests { for i, tt := range tests {
r := newRaft(1, []uint64{1, 2}, 10, 1) storage := NewMemoryStorage()
r.loadEnts(ents) storage.Append(ents)
r := newRaft(1, []uint64{1, 2}, 10, 1, storage)
r.loadState(pb.HardState{Term: 2}) r.loadState(pb.HardState{Term: 2})
// become leader at term 3 // become leader at term 3
r.becomeCandidate() 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) 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 (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 { if r.state != StateLeader {
panic("it should only be used when it is the leader") 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 // ignore further messages to refresh followers' commmit index
r.readMessages() r.readMessages()
s.Append(r.raftLog.unstableEntries())
r.raftLog.appliedTo(r.raftLog.committed) r.raftLog.appliedTo(r.raftLog.committed)
r.raftLog.stableTo(r.raftLog.lastIndex()) r.raftLog.stableTo(r.raftLog.lastIndex())
} }

View File

@ -22,14 +22,17 @@ import (
"math" "math"
"math/rand" "math/rand"
"reflect" "reflect"
"sort"
"testing" "testing"
pb "github.com/coreos/etcd/raft/raftpb" pb "github.com/coreos/etcd/raft/raftpb"
) )
// nextEnts returns the appliable entries and updates the applied index // 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() ents = r.raftLog.nextEnts()
r.raftLog.appliedTo(r.raftLog.committed) r.raftLog.appliedTo(r.raftLog.committed)
return ents return ents
@ -209,7 +212,7 @@ func TestLogReplication(t *testing.T) {
} }
ents := []pb.Entry{} ents := []pb.Entry{}
for _, e := range nextEnts(sm) { for _, e := range nextEnts(sm, tt.network.storage[j]) {
if e.Data != nil { if e.Data != nil {
ents = append(ents, e) ents = append(ents, e)
} }
@ -318,9 +321,9 @@ func TestCommitWithoutNewTermEntry(t *testing.T) {
} }
func TestDuelingCandidates(t *testing.T) { func TestDuelingCandidates(t *testing.T) {
a := newRaft(1, []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) b := newRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
c := newRaft(3, []uint64{1, 2, 3}, 10, 1) c := newRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
nt := newNetwork(a, b, c) nt := newNetwork(a, b, c)
nt.cut(1, 3) nt.cut(1, 3)
@ -331,7 +334,11 @@ func TestDuelingCandidates(t *testing.T) {
nt.recover() nt.recover()
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup}) 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 { tests := []struct {
sm *raft sm *raft
state StateType state StateType
@ -340,7 +347,7 @@ func TestDuelingCandidates(t *testing.T) {
}{ }{
{a, StateFollower, 2, wlog}, {a, StateFollower, 2, wlog},
{b, StateFollower, 2, wlog}, {b, StateFollower, 2, wlog},
{c, StateFollower, 2, newLog()}, {c, StateFollower, 2, newLog(NewMemoryStorage())},
} }
for i, tt := range tests { for i, tt := range tests {
@ -383,7 +390,13 @@ func TestCandidateConcede(t *testing.T) {
if g := a.Term; g != 1 { if g := a.Term; g != 1 {
t.Errorf("term = %d, want %d", 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 { for i, p := range tt.peers {
if sm, ok := p.(*raft); ok { if sm, ok := p.(*raft); ok {
l := ltoa(sm.raftLog) 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}}}) tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgApp, Term: 1, Entries: []pb.Entry{{Term: 1}}})
l := &raftLog{ l := &raftLog{
ents: []pb.Entry{ storage: &MemoryStorage{
{}, {Data: nil, Term: 1, Index: 1}, ents: []pb.Entry{
{Data: nil, Term: 2, Index: 2}, {Data: nil, Term: 3, Index: 3}, {}, {Data: nil, Term: 1, Index: 1},
{Data: nil, Term: 2, Index: 2}, {Data: nil, Term: 3, Index: 3},
},
}, },
unstable: 4,
committed: 3, committed: 3,
} }
base := ltoa(l) 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.MsgHup})
send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}}) send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}})
wantLog := newLog() wantLog := newLog(NewMemoryStorage())
if tt.success { 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) base := ltoa(wantLog)
for i, p := range tt.peers { for i, p := range tt.peers {
@ -506,7 +527,12 @@ func TestProposalByProxy(t *testing.T) {
// propose via follower // propose via follower
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}}) 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) base := ltoa(wantLog)
for i, p := range tt.peers { for i, p := range tt.peers {
if sm, ok := p.(*raft); ok { 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) { func TestCommit(t *testing.T) {
tests := []struct { tests := []struct {
matches []uint64 matches []uint64
@ -602,7 +584,11 @@ func TestCommit(t *testing.T) {
for j := 0; j < len(tt.matches); j++ { for j := 0; j < len(tt.matches); j++ {
prs[uint64(j)] = &progress{tt.matches[j], tt.matches[j] + 1} 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() sm.maybeCommit()
if g := sm.raftLog.committed; g != tt.w { if g := sm.raftLog.committed; g != tt.w {
t.Errorf("#%d: committed = %d, want %d", i, 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 { for i, tt := range tests {
sm := newRaft(1, []uint64{1}, 10, 1) sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
sm.elapsed = tt.elapse sm.elapsed = tt.elapse
c := 0 c := 0
for j := 0; j < 10000; j++ { for j := 0; j < 10000; j++ {
@ -649,7 +635,7 @@ func TestStepIgnoreOldTermMsg(t *testing.T) {
fakeStep := func(r *raft, m pb.Message) { fakeStep := func(r *raft, m pb.Message) {
called = true called = true
} }
sm := newRaft(1, []uint64{1}, 10, 1) sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
sm.step = fakeStep sm.step = fakeStep
sm.Term = 2 sm.Term = 2
sm.Step(pb.Message{Type: pb.MsgApp, Term: sm.Term - 1}) sm.Step(pb.Message{Type: pb.MsgApp, Term: sm.Term - 1})
@ -692,7 +678,11 @@ func TestHandleMsgApp(t *testing.T) {
sm := &raft{ sm := &raft{
state: StateFollower, state: StateFollower,
HardState: pb.HardState{Term: 2}, 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) sm.handleAppendEntries(tt.m)
@ -724,10 +714,12 @@ func TestHandleHeartbeat(t *testing.T) {
} }
for i, tt := range tests { 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{ sm := &raft{
state: StateFollower, state: StateFollower,
HardState: pb.HardState{Term: 2}, 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.raftLog.commitTo(commit)
sm.handleHeartbeat(tt.m) sm.handleHeartbeat(tt.m)
@ -776,7 +768,7 @@ func TestRecvMsgVote(t *testing.T) {
} }
for i, tt := range tests { for i, tt := range tests {
sm := newRaft(1, []uint64{1}, 10, 1) sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
sm.state = tt.state sm.state = tt.state
switch tt.state { switch tt.state {
case StateFollower: case StateFollower:
@ -787,7 +779,10 @@ func TestRecvMsgVote(t *testing.T) {
sm.step = stepLeader sm.step = stepLeader
} }
sm.HardState = pb.HardState{Vote: tt.voteFor} 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}) 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 sm.state = tt.from
switch tt.to { switch tt.to {
@ -863,16 +858,16 @@ func TestAllServerStepdown(t *testing.T) {
wterm uint64 wterm uint64
windex uint64 windex uint64
}{ }{
{StateFollower, StateFollower, 3, 1}, {StateFollower, StateFollower, 3, 0},
{StateCandidate, StateFollower, 3, 1}, {StateCandidate, StateFollower, 3, 0},
{StateLeader, StateFollower, 3, 2}, {StateLeader, StateFollower, 3, 1},
} }
tmsgTypes := [...]pb.MessageType{pb.MsgVote, pb.MsgApp} tmsgTypes := [...]pb.MessageType{pb.MsgVote, pb.MsgApp}
tterm := uint64(3) tterm := uint64(3)
for i, tt := range tests { 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 { switch tt.state {
case StateFollower: case StateFollower:
sm.becomeFollower(1, None) sm.becomeFollower(1, None)
@ -892,8 +887,11 @@ func TestAllServerStepdown(t *testing.T) {
if sm.Term != tt.wterm { if sm.Term != tt.wterm {
t.Errorf("#%d.%d term = %v , want %v", i, j, 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 { if uint64(sm.raftLog.lastIndex()) != tt.windex {
t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.ents), 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) wlead := uint64(2)
if msgType == pb.MsgVote { if msgType == pb.MsgVote {
@ -928,8 +926,11 @@ func TestLeaderAppResp(t *testing.T) {
for i, tt := range tests { for i, tt := range tests {
// sm term is 1 after it becomes the leader. // sm term is 1 after it becomes the leader.
// thus the last log term must be 1 to be committed. // thus the last log term must be 1 to be committed.
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1) sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
sm.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}} sm.raftLog = &raftLog{
storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}},
unstable: 3,
}
sm.becomeCandidate() sm.becomeCandidate()
sm.becomeLeader() sm.becomeLeader()
sm.readMessages() sm.readMessages()
@ -965,18 +966,21 @@ func TestBcastBeat(t *testing.T) {
offset := uint64(1000) offset := uint64(1000)
// make a state machine with log.offset = 1000 // make a state machine with log.offset = 1000
s := pb.Snapshot{ s := pb.Snapshot{
Index: offset, Metadata: pb.SnapshotMetadata{
Term: 1, Index: offset,
Nodes: []uint64{1, 2, 3}, 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.Term = 1
sm.restore(s)
sm.becomeCandidate() sm.becomeCandidate()
sm.becomeLeader() sm.becomeLeader()
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
sm.appendEntry(pb.Entry{}) sm.appendEntry(pb.Entry{Index: uint64(i) + 1})
} }
// slow follower // slow follower
sm.prs[2].match, sm.prs[2].next = 5, 6 sm.prs[2].match, sm.prs[2].next = 5, 6
@ -1029,8 +1033,8 @@ func TestRecvMsgBeat(t *testing.T) {
} }
for i, tt := range tests { for i, tt := range tests {
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1) sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
sm.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}} sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}}}
sm.Term = 1 sm.Term = 1
sm.state = tt.state sm.state = tt.state
switch tt.state { switch tt.state {
@ -1072,7 +1076,7 @@ func TestLeaderIncreaseNext(t *testing.T) {
} }
for i, tt := range tests { 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.raftLog.append(0, previousEnts...)
sm.becomeCandidate() sm.becomeCandidate()
sm.becomeLeader() sm.becomeLeader()
@ -1088,28 +1092,28 @@ func TestLeaderIncreaseNext(t *testing.T) {
func TestRestore(t *testing.T) { func TestRestore(t *testing.T) {
s := pb.Snapshot{ s := pb.Snapshot{
Index: 11, // magic number Metadata: pb.SnapshotMetadata{
Term: 11, // magic number Index: 11, // magic number
Nodes: []uint64{1, 2, 3}, 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 { if ok := sm.restore(s); !ok {
t.Fatal("restore fail, want succeed") t.Fatal("restore fail, want succeed")
} }
if sm.raftLog.lastIndex() != s.Index { if sm.raftLog.lastIndex() != s.Metadata.Index {
t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Index) t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Metadata.Index)
} }
if 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.Index), s.Term) t.Errorf("log.lastTerm = %d, want %d", sm.raftLog.term(s.Metadata.Index), s.Metadata.Term)
} }
sg := sm.nodes() sg := sm.nodes()
if !reflect.DeepEqual(sg, s.Nodes) { if !reflect.DeepEqual(sg, s.Metadata.ConfState.Nodes) {
t.Errorf("sm.Nodes = %+v, want %+v", sg, s.Nodes) t.Errorf("sm.Nodes = %+v, want %+v", sg, s.Metadata.ConfState.Nodes)
}
if !reflect.DeepEqual(sm.raftLog.snapshot, s) {
t.Errorf("snapshot = %+v, want %+v", sm.raftLog.snapshot, s)
} }
if ok := sm.restore(s); ok { if ok := sm.restore(s); ok {
@ -1118,14 +1122,17 @@ func TestRestore(t *testing.T) {
} }
func TestProvideSnap(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 // restore the statemachin from a snapshot
// so it has a compacted log and 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.restore(s)
sm.becomeCandidate() sm.becomeCandidate()
@ -1133,7 +1140,7 @@ func TestProvideSnap(t *testing.T) {
// force set the next of node 1, so that // force set the next of node 1, so that
// node 1 needs a snapshot // 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}) sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs[2].next - 1, Reject: true})
msgs := sm.readMessages() msgs := sm.readMessages()
@ -1148,18 +1155,18 @@ func TestProvideSnap(t *testing.T) {
func TestRestoreFromSnapMsg(t *testing.T) { func TestRestoreFromSnapMsg(t *testing.T) {
s := pb.Snapshot{ s := pb.Snapshot{
Index: 11, // magic number Metadata: pb.SnapshotMetadata{
Term: 11, // magic number Index: 11, // magic number
Nodes: []uint64{1, 2}, Term: 11, // magic number
ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
},
} }
m := pb.Message{Type: pb.MsgSnap, From: 1, Term: 2, Snapshot: s} 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) sm.Step(m)
if !reflect.DeepEqual(sm.raftLog.snapshot, s) { // TODO(bdarnell): what should this test?
t.Errorf("snapshot = %+v, want %+v", sm.raftLog.snapshot, s)
}
} }
func TestSlowNodeRestore(t *testing.T) { 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{{}}}) nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
} }
lead := nt.peers[1].(*raft) lead := nt.peers[1].(*raft)
nextEnts(lead) nextEnts(lead, nt.storage[1])
lead.compact(lead.raftLog.applied, lead.nodes(), nil) nt.storage[1].Compact(lead.raftLog.applied, &pb.ConfState{Nodes: lead.nodes()}, nil)
nt.recover() nt.recover()
// trigger a snapshot // trigger a snapshot
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}}) nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
follower := nt.peers[3].(*raft) 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 // trigger a commit
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}}) 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. // it appends the entry to log and sets pendingConf to be true.
func TestStepConfig(t *testing.T) { func TestStepConfig(t *testing.T) {
// a raft that cannot make progress // 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.becomeCandidate()
r.becomeLeader() r.becomeLeader()
index := r.raftLog.lastIndex() index := r.raftLog.lastIndex()
@ -1211,7 +1215,7 @@ func TestStepConfig(t *testing.T) {
// the proposal and keep its original state. // the proposal and keep its original state.
func TestStepIgnoreConfig(t *testing.T) { func TestStepIgnoreConfig(t *testing.T) {
// a raft that cannot make progress // 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.becomeCandidate()
r.becomeLeader() r.becomeLeader()
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange}}}) 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}, {pb.EntryConfChange, true},
} }
for i, tt := range tests { 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.appendEntry(pb.Entry{Type: tt.entType})
r.becomeCandidate() r.becomeCandidate()
r.becomeLeader() r.becomeLeader()
@ -1256,7 +1260,7 @@ func TestRecoverDoublePendingConfig(t *testing.T) {
t.Errorf("expect panic, but nothing happens") 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.appendEntry(pb.Entry{Type: pb.EntryConfChange}) r.appendEntry(pb.Entry{Type: pb.EntryConfChange})
r.becomeCandidate() r.becomeCandidate()
@ -1266,7 +1270,7 @@ func TestRecoverDoublePendingConfig(t *testing.T) {
// TestAddNode tests that addNode could update pendingConf and nodes correctly. // TestAddNode tests that addNode could update pendingConf and nodes correctly.
func TestAddNode(t *testing.T) { func TestAddNode(t *testing.T) {
r := newRaft(1, []uint64{1}, 10, 1) r := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
r.pendingConf = true r.pendingConf = true
r.addNode(2) r.addNode(2)
if r.pendingConf != false { if r.pendingConf != false {
@ -1282,7 +1286,7 @@ func TestAddNode(t *testing.T) {
// TestRemoveNode tests that removeNode could update pendingConf, nodes and // TestRemoveNode tests that removeNode could update pendingConf, nodes and
// and removed list correctly. // and removed list correctly.
func TestRemoveNode(t *testing.T) { 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.pendingConf = true
r.removeNode(2) r.removeNode(2)
if r.pendingConf != false { if r.pendingConf != false {
@ -1331,7 +1335,7 @@ func TestRaftNodes(t *testing.T) {
}, },
} }
for i, tt := range tests { 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) { if !reflect.DeepEqual(r.nodes(), tt.wids) {
t.Errorf("#%d: nodes = %+v, want %+v", i, 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 { func ents(terms ...uint64) *raft {
ents := []pb.Entry{{}} ents := []pb.Entry{{}}
for _, term := range terms { for i, term := range terms {
ents = append(ents, pb.Entry{Term: term}) 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) sm.reset(0)
return sm return sm
} }
type network struct { type network struct {
peers map[uint64]Interface peers map[uint64]Interface
storage map[uint64]*MemoryStorage
dropm map[connem]float64 dropm map[connem]float64
ignorem map[pb.MessageType]bool ignorem map[pb.MessageType]bool
} }
@ -1364,12 +1374,14 @@ func newNetwork(peers ...Interface) *network {
peerAddrs := idsBySize(size) peerAddrs := idsBySize(size)
npeers := make(map[uint64]Interface, size) npeers := make(map[uint64]Interface, size)
nstorage := make(map[uint64]*MemoryStorage, size)
for i, p := range peers { for i, p := range peers {
id := peerAddrs[i] id := peerAddrs[i]
switch v := p.(type) { switch v := p.(type) {
case nil: case nil:
sm := newRaft(id, peerAddrs, 10, 1) nstorage[id] = NewMemoryStorage()
sm := newRaft(id, peerAddrs, 10, 1, nstorage[id])
npeers[id] = sm npeers[id] = sm
case *raft: case *raft:
v.id = id v.id = id
@ -1387,6 +1399,7 @@ func newNetwork(peers ...Interface) *network {
} }
return &network{ return &network{
peers: npeers, peers: npeers,
storage: nstorage,
dropm: make(map[connem]float64), dropm: make(map[connem]float64),
ignorem: make(map[pb.MessageType]bool), ignorem: make(map[pb.MessageType]bool),
} }

View File

@ -10,9 +10,11 @@
It has these top-level messages: It has these top-level messages:
Entry Entry
SnapshotMetadata
Snapshot Snapshot
Message Message
HardState HardState
ConfState
ConfChange ConfChange
*/ */
package raftpb package raftpb
@ -163,12 +165,21 @@ func (m *Entry) Reset() { *m = Entry{} }
func (m *Entry) String() string { return proto.CompactTextString(m) } func (m *Entry) String() string { return proto.CompactTextString(m) }
func (*Entry) ProtoMessage() {} 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 { type Snapshot struct {
Data []byte `protobuf:"bytes,1,req,name=data" json:"data"` Data []byte `protobuf:"bytes,1,opt,name=data" json:"data"`
Nodes []uint64 `protobuf:"varint,2,rep,name=nodes" json:"nodes"` Metadata SnapshotMetadata `protobuf:"bytes,2,req,name=metadata" json:"metadata"`
Index uint64 `protobuf:"varint,3,req,name=index" json:"index"` XXX_unrecognized []byte `json:"-"`
Term uint64 `protobuf:"varint,4,req,name=term" json:"term"`
XXX_unrecognized []byte `json:"-"`
} }
func (m *Snapshot) Reset() { *m = Snapshot{} } 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 (m *HardState) String() string { return proto.CompactTextString(m) }
func (*HardState) ProtoMessage() {} 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 { type ConfChange struct {
ID uint64 `protobuf:"varint,1,req" json:"ID"` ID uint64 `protobuf:"varint,1,req" json:"ID"`
Type ConfChangeType `protobuf:"varint,2,req,enum=raftpb.ConfChangeType" json:"Type"` Type ConfChangeType `protobuf:"varint,2,req,enum=raftpb.ConfChangeType" json:"Type"`
@ -330,6 +350,102 @@ func (m *Entry) Unmarshal(data []byte) error {
} }
return nil 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 { func (m *Snapshot) Unmarshal(data []byte) error {
l := len(data) l := len(data)
index := 0 index := 0
@ -372,52 +488,29 @@ func (m *Snapshot) Unmarshal(data []byte) error {
m.Data = append(m.Data, data[index:postIndex]...) m.Data = append(m.Data, data[index:postIndex]...)
index = postIndex index = postIndex
case 2: case 2:
if wireType != 0 { if wireType != 2 {
return code_google_com_p_gogoprotobuf_proto.ErrWrongType return code_google_com_p_gogoprotobuf_proto.ErrWrongType
} }
var v uint64 var msglen int
for shift := uint(0); ; shift += 7 { for shift := uint(0); ; shift += 7 {
if index >= l { if index >= l {
return io.ErrUnexpectedEOF return io.ErrUnexpectedEOF
} }
b := data[index] b := data[index]
index++ index++
v |= (uint64(b) & 0x7F) << shift msglen |= (int(b) & 0x7F) << shift
if b < 0x80 { if b < 0x80 {
break break
} }
} }
m.Nodes = append(m.Nodes, v) postIndex := index + msglen
case 3: if postIndex > l {
if wireType != 0 { return io.ErrUnexpectedEOF
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
} }
for shift := uint(0); ; shift += 7 { if err := m.Metadata.Unmarshal(data[index:postIndex]); err != nil {
if index >= l { return err
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
}
} }
index = postIndex
default: default:
var sizeOfWire int var sizeOfWire int
for { for {
@ -739,6 +832,65 @@ func (m *HardState) Unmarshal(data []byte) error {
} }
return nil 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 { func (m *ConfChange) Unmarshal(data []byte) error {
l := len(data) l := len(data)
index := 0 index := 0
@ -861,18 +1013,25 @@ func (m *Entry) Size() (n int) {
} }
return n 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) { func (m *Snapshot) Size() (n int) {
var l int var l int
_ = l _ = l
l = len(m.Data) l = len(m.Data)
n += 1 + l + sovRaft(uint64(l)) n += 1 + l + sovRaft(uint64(l))
if len(m.Nodes) > 0 { l = m.Metadata.Size()
for _, e := range m.Nodes { n += 1 + l + sovRaft(uint64(l))
n += 1 + sovRaft(uint64(e))
}
}
n += 1 + sovRaft(uint64(m.Index))
n += 1 + sovRaft(uint64(m.Term))
if m.XXX_unrecognized != nil { if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized) n += len(m.XXX_unrecognized)
} }
@ -913,6 +1072,19 @@ func (m *HardState) Size() (n int) {
} }
return n 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) { func (m *ConfChange) Size() (n int) {
var l int var l int
_ = l _ = l
@ -973,6 +1145,40 @@ func (m *Entry) MarshalTo(data []byte) (n int, err error) {
} }
return i, nil 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) { func (m *Snapshot) Marshal() (data []byte, err error) {
size := m.Size() size := m.Size()
data = make([]byte, size) data = make([]byte, size)
@ -992,25 +1198,14 @@ func (m *Snapshot) MarshalTo(data []byte) (n int, err error) {
i++ i++
i = encodeVarintRaft(data, i, uint64(len(m.Data))) i = encodeVarintRaft(data, i, uint64(len(m.Data)))
i += copy(data[i:], m.Data) i += copy(data[i:], m.Data)
if len(m.Nodes) > 0 { data[i] = 0x12
for _, num := range m.Nodes { i++
data[i] = 0x10 i = encodeVarintRaft(data, i, uint64(m.Metadata.Size()))
i++ n2, err := m.Metadata.MarshalTo(data[i:])
for num >= 1<<7 { if err != nil {
data[i] = uint8(uint64(num)&0x7f | 0x80) return 0, err
num >>= 7
i++
}
data[i] = uint8(num)
i++
}
} }
data[i] = 0x18 i += n2
i++
i = encodeVarintRaft(data, i, uint64(m.Index))
data[i] = 0x20
i++
i = encodeVarintRaft(data, i, uint64(m.Term))
if m.XXX_unrecognized != nil { if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized) i += copy(data[i:], m.XXX_unrecognized)
} }
@ -1067,11 +1262,11 @@ func (m *Message) MarshalTo(data []byte) (n int, err error) {
data[i] = 0x4a data[i] = 0x4a
i++ i++
i = encodeVarintRaft(data, i, uint64(m.Snapshot.Size())) 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 { if err != nil {
return 0, err return 0, err
} }
i += n1 i += n3
data[i] = 0x50 data[i] = 0x50
i++ i++
if m.Reject { if m.Reject {
@ -1114,6 +1309,39 @@ func (m *HardState) MarshalTo(data []byte) (n int, err error) {
} }
return i, nil 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) { func (m *ConfChange) Marshal() (data []byte, err error) {
size := m.Size() size := m.Size()
data = make([]byte, size) data = make([]byte, size)

View File

@ -20,11 +20,15 @@ message Entry {
optional bytes Data = 4 [(gogoproto.nullable) = false]; 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 { message Snapshot {
required bytes data = 1 [(gogoproto.nullable) = false]; optional bytes data = 1 [(gogoproto.nullable) = false];
repeated uint64 nodes = 2 [(gogoproto.nullable) = false]; required SnapshotMetadata metadata = 2 [(gogoproto.nullable) = false];
required uint64 index = 3 [(gogoproto.nullable) = false];
required uint64 term = 4 [(gogoproto.nullable) = false];
} }
enum MessageType { enum MessageType {
@ -57,6 +61,10 @@ message HardState {
required uint64 commit = 3 [(gogoproto.nullable) = false]; required uint64 commit = 3 [(gogoproto.nullable) = false];
} }
message ConfState {
repeated uint64 nodes = 1 [(gogoproto.nullable) = false];
}
enum ConfChangeType { enum ConfChangeType {
ConfChangeAddNode = 0; ConfChangeAddNode = 0;
ConfChangeRemoveNode = 1; ConfChangeRemoveNode = 1;

187
raft/storage.go Normal file
View File

@ -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...)
}

54
raft/util.go Normal file
View File

@ -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))
}

View File

@ -61,7 +61,7 @@ func (s *Snapshotter) SaveSnap(snapshot raftpb.Snapshot) error {
} }
func (s *Snapshotter) save(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) b := pbutil.MustMarshal(snapshot)
crc := crc32.Update(0, crcTable, b) crc := crc32.Update(0, crcTable, b)
snap := snappb.Snapshot{Crc: crc, Data: b} snap := snappb.Snapshot{Crc: crc, Data: b}

View File

@ -29,10 +29,14 @@ import (
) )
var testSnap = &raftpb.Snapshot{ var testSnap = &raftpb.Snapshot{
Data: []byte("some snapshot"), Data: []byte("some snapshot"),
Nodes: []uint64{1, 2, 3}, Metadata: raftpb.SnapshotMetadata{
Index: 1, ConfState: raftpb.ConfState{
Term: 1, Nodes: []uint64{1, 2, 3},
},
Index: 1,
Term: 1,
},
} }
func TestSaveAndLoad(t *testing.T) { func TestSaveAndLoad(t *testing.T) {
@ -156,7 +160,7 @@ func TestLoadNewestSnap(t *testing.T) {
} }
newSnap := *testSnap newSnap := *testSnap
newSnap.Index = 5 newSnap.Metadata.Index = 5
err = ss.save(&newSnap) err = ss.save(&newSnap)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)

2
test
View File

@ -39,7 +39,7 @@ split=(${TEST// / })
TEST=${split[@]/#/${REPO_PATH}/} TEST=${split[@]/#/${REPO_PATH}/}
echo "Running tests..." echo "Running tests..."
go test ${COVER} $@ ${TEST} --race go test -timeout 60s ${COVER} $@ ${TEST} --race
echo "Checking gofmt..." echo "Checking gofmt..."
fmtRes=$(gofmt -l $FMT) fmtRes=$(gofmt -l $FMT)