Browse Source

raft: introduce log storage interface.

This change splits the raftLog.entries array into an in-memory
"unstable" list and a pluggable interface for retrieving entries that
have been persisted to disk. An in-memory implementation of this
interface is provided which behaves the same as the old version;
in a future commit etcdserver could replace the MemoryStorage with
one backed by the WAL.
Ben Darnell 11 years ago
parent
commit
25b6590547
16 changed files with 604 additions and 253 deletions
  1. 4 3
      etcdserver/force_cluster.go
  2. 24 18
      etcdserver/server.go
  3. 90 68
      etcdserver/server_test.go
  4. 1 1
      raft/diff_test.go
  5. 1 1
      raft/example_test.go
  6. 121 35
      raft/log.go
  7. 56 40
      raft/log_test.go
  8. 7 5
      raft/node.go
  9. 1 1
      raft/node_bench_test.go
  10. 16 10
      raft/node_test.go
  11. 3 3
      raft/raft.go
  12. 26 25
      raft/raft_paper_test.go
  13. 91 42
      raft/raft_test.go
  14. 108 0
      raft/storage.go
  15. 54 0
      raft/util.go
  16. 1 1
      test

+ 4 - 3
etcdserver/force_cluster.go

@@ -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)
 
 
@@ -53,8 +53,9 @@ func restartAsStandaloneNode(cfg *ServerConfig, index uint64, snapshot *raftpb.S
 	}
 	}
 
 
 	log.Printf("etcdserver: forcing restart of member %s in cluster %s at commit index %d", id, cfg.Cluster.ID(), st.Commit)
 	log.Printf("etcdserver: forcing restart of member %s in cluster %s at commit index %d", id, cfg.Cluster.ID(), st.Commit)
-	n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents)
-	return id, n, w
+	s := raft.NewMemoryStorage()
+	n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents, s)
+	return id, n, s, w
 }
 }
 
 
 // getIDs returns an ordered set of IDs included in the given snapshot and
 // getIDs returns an ordered set of IDs included in the given snapshot and

+ 24 - 18
etcdserver/server.go

@@ -156,8 +156,9 @@ type EtcdServer struct {
 
 
 	Cluster *Cluster
 	Cluster *Cluster
 
 
-	node  raft.Node
-	store store.Store
+	node        raft.Node
+	raftStorage *raft.MemoryStorage
+	store       store.Store
 
 
 	stats  *stats.ServerStats
 	stats  *stats.ServerStats
 	lstats *stats.LeaderStats
 	lstats *stats.LeaderStats
@@ -190,6 +191,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
 	haveWAL := wal.Exist(cfg.WALDir())
 	haveWAL := wal.Exist(cfg.WALDir())
 	switch {
 	switch {
@@ -204,7 +206,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
 		}
 		}
 		cfg.Cluster.SetID(cl.id)
 		cfg.Cluster.SetID(cl.id)
 		cfg.Cluster.SetStore(st)
 		cfg.Cluster.SetStore(st)
-		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
@@ -221,7 +223,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
 		}
 		}
 		cfg.Cluster.SetStore(st)
 		cfg.Cluster.SetStore(st)
 		log.Printf("etcdserver: initial cluster members: %s", cfg.Cluster)
 		log.Printf("etcdserver: initial cluster members: %s", cfg.Cluster)
-		id, n, w = startNode(cfg, cfg.Cluster.MemberIDs())
+		id, n, s, w = startNode(cfg, cfg.Cluster.MemberIDs())
 	case haveWAL:
 	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())
@@ -241,9 +243,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, snapshot)
 		} else {
 		} else {
-			id, n, w = restartAsStandaloneNode(cfg, index, snapshot)
+			id, n, s, w = restartAsStandaloneNode(cfg, index, snapshot)
 		}
 		}
 	default:
 	default:
 		return nil, fmt.Errorf("unsupported bootstrap config")
 		return nil, fmt.Errorf("unsupported bootstrap config")
@@ -256,12 +258,13 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
 	lstats := stats.NewLeaderStats(id.String())
 	lstats := stats.NewLeaderStats(id.String())
 
 
 	shub := newSendHub(cfg.Transport, cfg.Cluster, sstats, lstats)
 	shub := newSendHub(cfg.Transport, cfg.Cluster, sstats, lstats)
-	s := &EtcdServer{
-		store:      st,
-		node:       n,
-		id:         id,
-		attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
-		Cluster:    cfg.Cluster,
+	srv := &EtcdServer{
+		store:       st,
+		node:        n,
+		raftStorage: s,
+		id:          id,
+		attributes:  Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
+		Cluster:     cfg.Cluster,
 		storage: struct {
 		storage: struct {
 			*wal.WAL
 			*wal.WAL
 			*snap.Snapshotter
 			*snap.Snapshotter
@@ -273,7 +276,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
 		SyncTicker: time.Tick(500 * time.Millisecond),
 		SyncTicker: time.Tick(500 * time.Millisecond),
 		snapCount:  cfg.SnapCount,
 		snapCount:  cfg.SnapCount,
 	}
 	}
-	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
@@ -327,6 +330,7 @@ func (s *EtcdServer) run() {
 				}
 				}
 			}
 			}
 
 
+			s.raftStorage.Append(rd.Entries)
 			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)
 			}
 			}
@@ -722,7 +726,7 @@ func GetClusterFromPeers(urls []string) (*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(
@@ -744,7 +748,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 node %s in cluster %s", id, cfg.Cluster.ID())
 	log.Printf("etcdserver: start node %s in cluster %s", id, cfg.Cluster.ID())
-	n = raft.StartNode(uint64(id), peers, 10, 1)
+	s = raft.NewMemoryStorage()
+	n = raft.StartNode(uint64(id), peers, 10, 1, s)
 	return
 	return
 }
 }
 
 
@@ -762,13 +767,14 @@ 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)
-	return id, n, w
+	s := raft.NewMemoryStorage()
+	n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents, s)
+	return id, n, s, w
 }
 }
 
 
 func readWAL(waldir string, index uint64) (w *wal.WAL, id, cid types.ID, st raftpb.HardState, ents []raftpb.Entry) {
 func readWAL(waldir string, index uint64) (w *wal.WAL, id, cid types.ID, st raftpb.HardState, ents []raftpb.Entry) {

+ 90 - 68
etcdserver/server_test.go

@@ -519,24 +519,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,
-			store:   st,
-			sender:  &fakeSender{ss},
-			storage: &storageRecorder{},
-			Ticker:  tk.C,
-			Cluster: cl,
+			node:        n,
+			raftStorage: s,
+			store:       st,
+			sender:      &fakeSender{ss},
+			storage:     &storageRecorder{},
+			Ticker:      tk.C,
+			Cluster:     cl,
 		}
 		}
-		srv.start()
 		ss[i] = srv
 		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",
@@ -587,7 +593,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
@@ -595,12 +602,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,
-			store:   st,
-			sender:  &nopSender{},
-			storage: &storageRecorder{},
-			Ticker:  tk,
-			Cluster: cl,
+			node:        n,
+			raftStorage: s,
+			store:       st,
+			sender:      &nopSender{},
+			storage:     &storageRecorder{},
+			Ticker:      tk,
+			Cluster:     cl,
 		}
 		}
 		srv.start()
 		srv.start()
 		resp, err := srv.Do(ctx, tt)
 		resp, err := srv.Do(ctx, tt)
@@ -623,14 +631,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,
-		store: st,
-		w:     wait,
+		node:        n,
+		raftStorage: s,
+		store:       st,
+		w:           wait,
 	}
 	}
 
 
 	done := make(chan struct{})
 	done := make(chan struct{})
@@ -671,18 +681,20 @@ 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
 	close(tk)
 	close(tk)
 	srv := &EtcdServer{
 	srv := &EtcdServer{
 		// TODO: use fake node for better testability
 		// TODO: use fake node for better testability
-		node:    n,
-		store:   st,
-		sender:  &nopSender{},
-		storage: &storageRecorder{},
-		Ticker:  tk,
+		node:        n,
+		raftStorage: s,
+		store:       st,
+		sender:      &nopSender{},
+		storage:     &storageRecorder{},
+		Ticker:      tk,
 	}
 	}
 	srv.start()
 	srv.start()
 
 
@@ -788,11 +800,12 @@ func TestSyncTrigger(t *testing.T) {
 	}
 	}
 	st := make(chan time.Time, 1)
 	st := make(chan time.Time, 1)
 	srv := &EtcdServer{
 	srv := &EtcdServer{
-		node:       n,
-		store:      &storeRecorder{},
-		sender:     &nopSender{},
-		storage:    &storageRecorder{},
-		SyncTicker: st,
+		node:        n,
+		raftStorage: raft.NewMemoryStorage(),
+		store:       &storeRecorder{},
+		sender:      &nopSender{},
+		storage:     &storageRecorder{},
+		SyncTicker:  st,
 	}
 	}
 	srv.start()
 	srv.start()
 	// trigger the server to become a leader and accept sync requests
 	// trigger the server to become a leader and accept sync requests
@@ -822,17 +835,19 @@ func TestSyncTrigger(t *testing.T) {
 // snapshot should snapshot the store and cut the persistent
 // 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()
 	st := &storeRecorder{}
 	st := &storeRecorder{}
 	p := &storageRecorder{}
 	p := &storageRecorder{}
-	s := &EtcdServer{
-		store:   st,
-		storage: p,
-		node:    n,
+	srv := &EtcdServer{
+		store:       st,
+		storage:     p,
+		node:        n,
+		raftStorage: s,
 	}
 	}
 
 
-	s.snapshot(0, []uint64{1})
+	srv.snapshot(0, []uint64{1})
 	gaction := st.Action()
 	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))
@@ -853,8 +868,10 @@ func TestSnapshot(t *testing.T) {
 // 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)
-	<-n.Ready()
+	s := raft.NewMemoryStorage()
+	n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s)
+	rd := <-n.Ready()
+	s.Append(rd.Entries)
 	n.Advance()
 	n.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)
@@ -862,26 +879,27 @@ 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{
-		store:     st,
-		sender:    &nopSender{},
-		storage:   p,
-		node:      n,
-		snapCount: 10,
-		Cluster:   cl,
+	srv := &EtcdServer{
+		store:       st,
+		sender:      &nopSender{},
+		storage:     p,
+		node:        n,
+		raftStorage: s,
+		snapCount:   10,
+		Cluster:     cl,
 	}
 	}
 
 
-	s.start()
-	for i := 0; uint64(i) < s.snapCount-1; i++ {
-		s.Do(ctx, pb.Request{Method: "PUT", ID: 1})
+	srv.start()
+	for i := 0; uint64(i) < srv.snapCount-1; i++ {
+		srv.Do(ctx, pb.Request{Method: "PUT", ID: 1})
 	}
 	}
 	time.Sleep(time.Millisecond)
 	time.Sleep(time.Millisecond)
-	s.Stop()
+	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)
 	}
 	}
@@ -897,10 +915,11 @@ func TestRecvSnapshot(t *testing.T) {
 	st := &storeRecorder{}
 	st := &storeRecorder{}
 	p := &storageRecorder{}
 	p := &storageRecorder{}
 	s := &EtcdServer{
 	s := &EtcdServer{
-		store:   st,
-		sender:  &nopSender{},
-		storage: p,
-		node:    n,
+		store:       st,
+		sender:      &nopSender{},
+		storage:     p,
+		node:        n,
+		raftStorage: raft.NewMemoryStorage(),
 	}
 	}
 
 
 	s.start()
 	s.start()
@@ -925,10 +944,11 @@ func TestRecvSlowSnapshot(t *testing.T) {
 	n := newReadyNode()
 	n := newReadyNode()
 	st := &storeRecorder{}
 	st := &storeRecorder{}
 	s := &EtcdServer{
 	s := &EtcdServer{
-		store:   st,
-		sender:  &nopSender{},
-		storage: &storageRecorder{},
-		node:    n,
+		store:       st,
+		sender:      &nopSender{},
+		storage:     &storageRecorder{},
+		node:        n,
+		raftStorage: raft.NewMemoryStorage(),
 	}
 	}
 
 
 	s.start()
 	s.start()
@@ -959,11 +979,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,
-		store:   &storeRecorder{},
-		sender:  &nopSender{},
-		storage: &storageRecorder{},
-		Cluster: cl,
+		node:        n,
+		raftStorage: raft.NewMemoryStorage(),
+		store:       &storeRecorder{},
+		sender:      &nopSender{},
+		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"}}}
@@ -994,11 +1015,12 @@ func TestRemoveMember(t *testing.T) {
 	}
 	}
 	cl := newTestCluster([]Member{{ID: 1234}})
 	cl := newTestCluster([]Member{{ID: 1234}})
 	s := &EtcdServer{
 	s := &EtcdServer{
-		node:    n,
-		store:   &storeRecorder{},
-		sender:  &nopSender{},
-		storage: &storageRecorder{},
-		Cluster: cl,
+		node:        n,
+		raftStorage: raft.NewMemoryStorage(),
+		store:       &storeRecorder{},
+		sender:      &nopSender{},
+		storage:     &storageRecorder{},
+		Cluster:     cl,
 	}
 	}
 	s.start()
 	s.start()
 	err := s.RemoveMember(context.TODO(), 1234)
 	err := s.RemoveMember(context.TODO(), 1234)

+ 1 - 1
raft/diff_test.go

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

+ 1 - 1
raft/example_test.go

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

+ 121 - 35
raft/log.go

@@ -24,33 +24,54 @@ import (
 )
 )
 
 
 type raftLog struct {
 type raftLog struct {
-	ents      []pb.Entry
-	unstable  uint64
+	// storage contains all stable entries since the last snapshot.
+	storage Storage
+	// unstableEnts contains all entries that have not yet been written
+	// to storage.
+	unstableEnts []pb.Entry
+	// unstableEnts[i] has raft log position i+unstable.  Note that
+	// unstable may be less than the highest log position in storage;
+	// this means that the next write to storage will truncate the log
+	// before persisting unstableEnts.
+	unstable uint64
+	// committed is the highest log position that is known to be in
+	// stable storage on a quorum of nodes.
+	// Invariant: committed < unstable
 	committed uint64
 	committed uint64
-	applied   uint64
-	offset    uint64
-	snapshot  pb.Snapshot
+	// applied is the highest log position that the application has
+	// been instructed to apply to its state machine.
+	// Invariant: applied <= committed
+	applied  uint64
+	snapshot pb.Snapshot
 }
 }
 
 
-func newLog() *raftLog {
+func newLog(storage Storage) *raftLog {
+	if storage == nil {
+		storage = NewMemoryStorage()
+	}
+	lastIndex, err := storage.GetLastIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
 	return &raftLog{
 	return &raftLog{
-		ents:      make([]pb.Entry, 1),
-		unstable:  0,
-		committed: 0,
-		applied:   0,
+		storage:  storage,
+		unstable: lastIndex + 1,
 	}
 	}
 }
 }
 
 
 func (l *raftLog) load(ents []pb.Entry) {
 func (l *raftLog) load(ents []pb.Entry) {
-	if l.offset != ents[0].Index {
+	// TODO(bdarnell): does this method need to support other Storage impls or does it go away?
+	ms := l.storage.(*MemoryStorage)
+	if ms.offset != ents[0].Index {
 		panic("entries loaded don't match offset index")
 		panic("entries loaded don't match offset index")
 	}
 	}
-	l.ents = ents
-	l.unstable = l.offset + uint64(len(ents))
+	ms.ents = ents
+	l.unstable = ms.offset + uint64(len(ents))
 }
 }
 
 
 func (l *raftLog) String() string {
 func (l *raftLog) String() string {
-	return fmt.Sprintf("offset=%d committed=%d applied=%d len(ents)=%d", l.offset, l.committed, l.applied, len(l.ents))
+	return fmt.Sprintf("unstable=%d committed=%d applied=%d", l.unstable, l.committed, l.applied)
+
 }
 }
 
 
 // maybeAppend returns (0, false) if the entries cannot be appended. Otherwise,
 // maybeAppend returns (0, false) if the entries cannot be appended. Otherwise,
@@ -78,7 +99,15 @@ func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry
 }
 }
 
 
 func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 {
 func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 {
-	l.ents = append(l.slice(l.offset, after+1), ents...)
+	if after < l.unstable {
+		// The log is being truncated to before our current unstable
+		// portion, so discard it and reset unstable.
+		l.unstableEnts = nil
+		l.unstable = after + 1
+	}
+	// Truncate any unstable entries that are being replaced, then
+	// append the new ones.
+	l.unstableEnts = append(l.unstableEnts[0:1+after-l.unstable], ents...)
 	l.unstable = min(l.unstable, after+1)
 	l.unstable = min(l.unstable, after+1)
 	return l.lastIndex()
 	return l.lastIndex()
 }
 }
@@ -104,13 +133,12 @@ func (l *raftLog) findConflict(from uint64, ents []pb.Entry) uint64 {
 	return 0
 	return 0
 }
 }
 
 
-func (l *raftLog) unstableEnts() []pb.Entry {
-	ents := l.slice(l.unstable, l.lastIndex()+1)
-	if ents == nil {
+func (l *raftLog) unstableEntries() []pb.Entry {
+	if len(l.unstableEnts) == 0 {
 		return nil
 		return nil
 	}
 	}
-	cpy := make([]pb.Entry, len(ents))
-	copy(cpy, ents)
+	cpy := make([]pb.Entry, len(l.unstableEnts))
+	copy(cpy, l.unstableEnts)
 	return cpy
 	return cpy
 }
 }
 
 
@@ -123,6 +151,25 @@ func (l *raftLog) nextEnts() (ents []pb.Entry) {
 	return nil
 	return nil
 }
 }
 
 
+func (l *raftLog) firstIndex() uint64 {
+	index, err := l.storage.GetFirstIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	return index
+}
+
+func (l *raftLog) lastIndex() uint64 {
+	if len(l.unstableEnts) > 0 {
+		return l.unstable + uint64(len(l.unstableEnts)) - 1
+	}
+	index, err := l.storage.GetLastIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	return index
+}
+
 func (l *raftLog) appliedTo(i uint64) {
 func (l *raftLog) appliedTo(i uint64) {
 	if i == 0 {
 	if i == 0 {
 		return
 		return
@@ -137,12 +184,13 @@ func (l *raftLog) stableTo(i uint64) {
 	if i == 0 {
 	if i == 0 {
 		return
 		return
 	}
 	}
+	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 {
 	if e := l.at(i); e != nil {
@@ -155,12 +203,19 @@ func (l *raftLog) entries(i uint64) []pb.Entry {
 	// never send out the first entry
 	// never send out the first entry
 	// first entry is only used for matching
 	// first entry is only used for matching
 	// prevLogTerm
 	// prevLogTerm
-	if i == l.offset {
+	if i == 0 {
 		panic("cannot return the first entry in log")
 		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, including the initial
+// entry that is only used for prevLogTerm validation. This method
+// should only be used for testing.
+func (l *raftLog) allEntries() []pb.Entry {
+	return l.slice(l.firstIndex(), l.lastIndex()+1)
+}
+
 // isUpToDate determines if the given (lastIndex,term) log is more up-to-date
 // 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
@@ -193,12 +248,22 @@ func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
 // the number of entries after compaction will be returned.
 // the number of entries after compaction will be returned.
 func (l *raftLog) compact(i uint64) uint64 {
 func (l *raftLog) compact(i uint64) uint64 {
 	if l.isOutOfAppliedBounds(i) {
 	if l.isOutOfAppliedBounds(i) {
-		panic(fmt.Sprintf("compact %d out of bounds [%d:%d]", i, l.offset, l.applied))
+		panic(fmt.Sprintf("compact %d out of bounds (applied up to %d)", i, l.applied))
+	}
+	err := l.storage.Compact(i)
+	if err != nil {
+		panic(err) // TODO(bdarnell)
 	}
 	}
-	l.ents = l.slice(i, l.lastIndex()+1)
 	l.unstable = max(i+1, l.unstable)
 	l.unstable = max(i+1, l.unstable)
-	l.offset = i
-	return uint64(len(l.ents))
+	firstIndex, err := l.storage.GetFirstIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	lastIndex, err := l.storage.GetLastIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	return lastIndex - firstIndex
 }
 }
 
 
 func (l *raftLog) snap(d []byte, index, term uint64, nodes []uint64) {
 func (l *raftLog) snap(d []byte, index, term uint64, nodes []uint64) {
@@ -211,19 +276,23 @@ func (l *raftLog) snap(d []byte, index, term uint64, nodes []uint64) {
 }
 }
 
 
 func (l *raftLog) restore(s pb.Snapshot) {
 func (l *raftLog) restore(s pb.Snapshot) {
-	l.ents = []pb.Entry{{Term: s.Term}}
+	l.storage = &MemoryStorage{
+		ents:   []pb.Entry{{Term: s.Term}},
+		offset: s.Index,
+	}
 	l.unstable = s.Index + 1
 	l.unstable = s.Index + 1
+	l.unstableEnts = nil
 	l.committed = s.Index
 	l.committed = s.Index
 	l.applied = s.Index
 	l.applied = s.Index
-	l.offset = s.Index
 	l.snapshot = s
 	l.snapshot = s
 }
 }
 
 
 func (l *raftLog) at(i uint64) *pb.Entry {
 func (l *raftLog) at(i uint64) *pb.Entry {
-	if l.isOutOfBounds(i) {
+	ents := l.slice(i, i+1)
+	if len(ents) == 0 {
 		return nil
 		return nil
 	}
 	}
-	return &l.ents[i-l.offset]
+	return &ents[0]
 }
 }
 
 
 // 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.
@@ -234,18 +303,35 @@ 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.GetEntries(lo, min(hi, l.unstable))
+		if err != nil {
+			panic(err) // TODO(bdarnell)
+		}
+		ents = append(ents, storedEnts...)
+	}
+	if len(l.unstableEnts) > 0 && hi > l.unstable {
+		var firstUnstable uint64
+		if lo < l.unstable {
+			firstUnstable = l.unstable
+		} else {
+			firstUnstable = lo
+		}
+		ents = append(ents, l.unstableEnts[firstUnstable-l.unstable:hi-l.unstable]...)
+	}
+	return ents
 }
 }
 
 
 func (l *raftLog) isOutOfBounds(i uint64) bool {
 func (l *raftLog) isOutOfBounds(i uint64) bool {
-	if i < l.offset || i > l.lastIndex() {
+	if i < l.firstIndex() || i > l.lastIndex() {
 		return true
 		return true
 	}
 	}
 	return false
 	return false
 }
 }
 
 
 func (l *raftLog) isOutOfAppliedBounds(i uint64) bool {
 func (l *raftLog) isOutOfAppliedBounds(i uint64) bool {
-	if i < l.offset || i > l.applied {
+	if i < l.firstIndex() || i > l.applied {
 		return true
 		return true
 	}
 	}
 	return false
 	return false

+ 56 - 40
raft/log_test.go

@@ -49,7 +49,7 @@ func TestFindConflict(t *testing.T) {
 	}
 	}
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
-		raftLog := newLog()
+		raftLog := newLog(nil)
 		raftLog.append(raftLog.lastIndex(), previousEnts...)
 		raftLog.append(raftLog.lastIndex(), previousEnts...)
 
 
 		gconflict := raftLog.findConflict(tt.from, tt.ents)
 		gconflict := raftLog.findConflict(tt.from, tt.ents)
@@ -61,7 +61,7 @@ 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{{Term: 1}, {Term: 2}, {Term: 3}}
-	raftLog := newLog()
+	raftLog := newLog(nil)
 	raftLog.append(raftLog.lastIndex(), previousEnts...)
 	raftLog.append(raftLog.lastIndex(), previousEnts...)
 	tests := []struct {
 	tests := []struct {
 		lastIndex uint64
 		lastIndex uint64
@@ -92,7 +92,6 @@ 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{{Term: 1}, {Term: 2}}
-	previousUnstable := uint64(3)
 	tests := []struct {
 	tests := []struct {
 		after     uint64
 		after     uint64
 		ents      []pb.Entry
 		ents      []pb.Entry
@@ -133,9 +132,10 @@ func TestAppend(t *testing.T) {
 	}
 	}
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
-		raftLog := newLog()
-		raftLog.append(raftLog.lastIndex(), previousEnts...)
-		raftLog.unstable = previousUnstable
+		storage := NewMemoryStorage()
+		storage.Append(previousEnts)
+		raftLog := newLog(storage)
+
 		index := raftLog.append(tt.after, tt.ents...)
 		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)
@@ -241,7 +241,7 @@ func TestLogMaybeAppend(t *testing.T) {
 	}
 	}
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
-		raftLog := newLog()
+		raftLog := newLog(nil)
 		raftLog.append(raftLog.lastIndex(), previousEnts...)
 		raftLog.append(raftLog.lastIndex(), previousEnts...)
 		raftLog.committed = commit
 		raftLog.committed = commit
 		func() {
 		func() {
@@ -278,40 +278,50 @@ 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++ {
+		storage.Append([]pb.Entry{{Term: uint64(i), Index: uint64(i)}})
+	}
+	raftLog := newLog(storage)
+	for i = unstableIndex; i < lastIndex; i++ {
+		raftLog.append(i, pb.Entry{Term: uint64(i + 1), Index: uint64(i + 1)})
+	}
 
 
-	for i = 0; i < lastIndex; i++ {
-		raftLog.append(uint64(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)
+	raftLog.compact(offset)
 
 
 	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()
-	if g := len(unstableEnts); g != 500 {
-		t.Errorf("len(unstableEntries) = %d, want = %d", g, 500)
+	unstableEnts := raftLog.unstableEntries()
+	if g := len(unstableEnts); g != 250 {
+		t.Errorf("len(unstableEntries) = %d, want = %d", g, 250)
 	}
 	}
-	if unstableEnts[0].Index != 501 {
-		t.Errorf("Index = %d, want = %d", unstableEnts[0].Index, 501)
+	if unstableEnts[0].Index != 751 {
+		t.Errorf("Index = %d, want = %d", unstableEnts[0].Index, 751)
 	}
 	}
 
 
 	prev := raftLog.lastIndex()
 	prev := raftLog.lastIndex()
@@ -338,10 +348,11 @@ func TestUnstableEnts(t *testing.T) {
 	}
 	}
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
-		raftLog := newLog()
-		raftLog.append(0, previousEnts...)
-		raftLog.unstable = tt.unstable
-		ents := raftLog.unstableEnts()
+		storage := NewMemoryStorage()
+		storage.Append(previousEnts[:tt.unstable-1])
+		raftLog := newLog(storage)
+		raftLog.append(raftLog.lastIndex(), previousEnts[tt.unstable-1:]...)
+		ents := raftLog.unstableEntries()
 		raftLog.stableTo(raftLog.lastIndex())
 		raftLog.stableTo(raftLog.lastIndex())
 		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)
@@ -374,22 +385,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(0); i < tt.lastIndex; i++ {
-				raftLog.append(uint64(i), pb.Entry{})
+				storage.Append([]pb.Entry{{}})
 			}
 			}
+			raftLog := newLog(storage)
 			raftLog.maybeCommit(tt.applied, 0)
 			raftLog.maybeCommit(tt.applied, 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])
 				raftLog.compact(tt.compact[j])
-				if len(raftLog.ents) != tt.wleft[j] {
-					t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.ents), tt.wleft[j])
+				if len(raftLog.allEntries()) != tt.wleft[j] {
+					t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.allEntries()), tt.wleft[j])
 				}
 				}
 			}
 			}
 		}()
 		}()
@@ -398,7 +410,7 @@ func TestCompaction(t *testing.T) {
 
 
 func TestLogRestore(t *testing.T) {
 func TestLogRestore(t *testing.T) {
 	var i uint64
 	var i uint64
-	raftLog := newLog()
+	raftLog := newLog(nil)
 	for i = 0; i < 100; i++ {
 	for i = 0; i < 100; i++ {
 		raftLog.append(i, pb.Entry{Term: i + 1})
 		raftLog.append(i, pb.Entry{Term: i + 1})
 	}
 	}
@@ -408,11 +420,11 @@ func TestLogRestore(t *testing.T) {
 	raftLog.restore(pb.Snapshot{Index: index, Term: term})
 	raftLog.restore(pb.Snapshot{Index: index, Term: term})
 
 
 	// only has the guard entry
 	// only has the guard entry
-	if len(raftLog.ents) != 1 {
-		t.Errorf("len = %d, want 0", len(raftLog.ents))
+	if len(raftLog.allEntries()) != 1 {
+		t.Errorf("len = %d, want 1", len(raftLog.allEntries()))
 	}
 	}
-	if raftLog.offset != index {
-		t.Errorf("offset = %d, want %d", raftLog.offset, index)
+	if raftLog.firstIndex() != index {
+		t.Errorf("firstIndex = %d, want %d", raftLog.firstIndex(), index)
 	}
 	}
 	if raftLog.applied != index {
 	if raftLog.applied != index {
 		t.Errorf("applied = %d, want %d", raftLog.applied, index)
 		t.Errorf("applied = %d, want %d", raftLog.applied, index)
@@ -431,7 +443,9 @@ func TestLogRestore(t *testing.T) {
 func TestIsOutOfBounds(t *testing.T) {
 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)}
+	l := newLog(nil)
+	l.restore(pb.Snapshot{Index: offset})
+	l.append(offset, make([]pb.Entry, num)...)
 
 
 	tests := []struct {
 	tests := []struct {
 		index uint64
 		index uint64
@@ -440,8 +454,8 @@ func TestIsOutOfBounds(t *testing.T) {
 		{offset - 1, true},
 		{offset - 1, true},
 		{offset, false},
 		{offset, false},
 		{offset + num/2, false},
 		{offset + num/2, false},
-		{offset + num - 1, false},
-		{offset + num, true},
+		{offset + num, false},
+		{offset + num + 1, true},
 	}
 	}
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
@@ -457,9 +471,10 @@ func TestAt(t *testing.T) {
 	offset := uint64(100)
 	offset := uint64(100)
 	num := uint64(100)
 	num := uint64(100)
 
 
-	l := &raftLog{offset: offset}
+	l := newLog(nil)
+	l.restore(pb.Snapshot{Index: offset})
 	for i = 0; i < num; i++ {
 	for i = 0; i < num; i++ {
-		l.ents = append(l.ents, pb.Entry{Term: i})
+		l.append(offset+i-1, pb.Entry{Term: i})
 	}
 	}
 
 
 	tests := []struct {
 	tests := []struct {
@@ -486,9 +501,10 @@ func TestSlice(t *testing.T) {
 	offset := uint64(100)
 	offset := uint64(100)
 	num := uint64(100)
 	num := uint64(100)
 
 
-	l := &raftLog{offset: offset}
+	l := newLog(nil)
+	l.restore(pb.Snapshot{Index: offset})
 	for i = 0; i < num; i++ {
 	for i = 0; i < num; i++ {
-		l.ents = append(l.ents, pb.Entry{Term: i})
+		l.append(offset+i-1, pb.Entry{Term: i})
 	}
 	}
 
 
 	tests := []struct {
 	tests := []struct {

+ 7 - 5
raft/node.go

@@ -144,9 +144,9 @@ type Peer struct {
 // StartNode returns a new Node given a unique raft id, a list of raft peers, and
 // 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 also builds ConfChangeAddNode entry for each peer and puts them at the head of the log.
 // It also builds ConfChangeAddNode entry for each peer and puts them at the head of the log.
-func StartNode(id uint64, peers []Peer, election, heartbeat int) Node {
+func StartNode(id uint64, peers []Peer, election, heartbeat int, storage Storage) Node {
 	n := newNode()
 	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}
@@ -166,9 +166,11 @@ func StartNode(id uint64, peers []Peer, election, heartbeat int) Node {
 // RestartNode is identical to StartNode but takes an initial State and a slice
 // RestartNode is identical to StartNode but takes an initial State and a slice
 // of entries. Generally this is used when restarting from a stable storage
 // of entries. Generally this is used when restarting from a stable storage
 // log.
 // log.
-func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, ents []pb.Entry) Node {
+// TODO(bdarnell): remove args that are unnecessary with storage.
+// Maybe this function goes away and is replaced by StartNode with a non-empty Storage.
+func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, ents []pb.Entry, storage Storage) Node {
 	n := newNode()
 	n := newNode()
-	r := newRaft(id, nil, election, heartbeat)
+	r := newRaft(id, nil, election, heartbeat, storage)
 	if snapshot != nil {
 	if snapshot != nil {
 		r.restore(*snapshot)
 		r.restore(*snapshot)
 	}
 	}
@@ -387,7 +389,7 @@ func (n *node) Compact(index uint64, nodes []uint64, d []byte) {
 
 
 func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState, prevSnapi uint64) Ready {
 func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState, prevSnapi uint64) 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,
 	}
 	}

+ 1 - 1
raft/node_bench_test.go

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

+ 16 - 10
raft/node_test.go

@@ -112,7 +112,7 @@ func TestNodeStepUnblock(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, nil)
 	go n.run(r)
 	go n.run(r)
 	defer n.Stop()
 	defer n.Stop()
 
 
@@ -175,7 +175,6 @@ func TestNode(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},
 			},
 			},
@@ -190,12 +189,15 @@ func TestNode(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()
+	if !reflect.DeepEqual(g, wants[0]) {
 		t.Errorf("#%d: g = %+v,\n             w   %+v", 1, g, wants[0])
 		t.Errorf("#%d: g = %+v,\n             w   %+v", 1, g, wants[0])
 	} else {
 	} else {
+		storage.Append(g.Entries)
 		n.Advance()
 		n.Advance()
 	}
 	}
 
 
@@ -203,6 +205,7 @@ func TestNode(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()
 	}
 	}
 
 
@@ -227,7 +230,7 @@ func TestNodeRestart(t *testing.T) {
 		CommittedEntries: entries[1 : st.Commit+1],
 		CommittedEntries: entries[1 : st.Commit+1],
 	}
 	}
 
 
-	n := RestartNode(1, 10, 1, nil, st, entries)
+	n := RestartNode(1, 10, 1, nil, st, entries, nil)
 	if g := <-n.Ready(); !reflect.DeepEqual(g, want) {
 	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 {
@@ -246,7 +249,8 @@ func TestNodeRestart(t *testing.T) {
 func TestNodeCompact(t *testing.T) {
 func TestNodeCompact(t *testing.T) {
 	ctx := context.Background()
 	ctx := context.Background()
 	n := newNode()
 	n := newNode()
-	r := newRaft(1, []uint64{1}, 10, 1)
+	storage := NewMemoryStorage()
+	r := newRaft(1, []uint64{1}, 10, 1, storage)
 	go n.run(r)
 	go n.run(r)
 
 
 	n.Campaign(ctx)
 	n.Campaign(ctx)
@@ -261,7 +265,8 @@ func TestNodeCompact(t *testing.T) {
 
 
 	testutil.ForceGosched()
 	testutil.ForceGosched()
 	select {
 	select {
-	case <-n.Ready():
+	case rd := <-n.Ready():
+		storage.Append(rd.Entries)
 		n.Advance()
 		n.Advance()
 	default:
 	default:
 		t.Fatalf("unexpected proposal failure: unable to commit entry")
 		t.Fatalf("unexpected proposal failure: unable to commit entry")
@@ -274,6 +279,7 @@ func TestNodeCompact(t *testing.T) {
 		if !reflect.DeepEqual(rd.Snapshot, w) {
 		if !reflect.DeepEqual(rd.Snapshot, w) {
 			t.Errorf("snap = %+v, want %+v", rd.Snapshot, w)
 			t.Errorf("snap = %+v, want %+v", rd.Snapshot, w)
 		}
 		}
+		storage.Append(rd.Entries)
 		n.Advance()
 		n.Advance()
 	default:
 	default:
 		t.Fatalf("unexpected compact failure: unable to create a snapshot")
 		t.Fatalf("unexpected compact failure: unable to create a snapshot")
@@ -288,8 +294,8 @@ func TestNodeCompact(t *testing.T) {
 	}
 	}
 	n.Stop()
 	n.Stop()
 
 
-	if r.raftLog.offset != w.Index {
-		t.Errorf("log.offset = %d, want %d", r.raftLog.offset, w.Index)
+	if r.raftLog.firstIndex() != w.Index {
+		t.Errorf("log.offset = %d, want %d", r.raftLog.firstIndex(), w.Index)
 	}
 	}
 }
 }
 
 
@@ -297,7 +303,7 @@ func TestNodeAdvance(t *testing.T) {
 	ctx, cancel := context.WithCancel(context.Background())
 	ctx, cancel := context.WithCancel(context.Background())
 	defer cancel()
 	defer cancel()
 
 
-	n := StartNode(1, []Peer{{ID: 1}}, 10, 1)
+	n := StartNode(1, []Peer{{ID: 1}}, 10, 1, nil)
 	n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1})
 	n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1})
 	n.Campaign(ctx)
 	n.Campaign(ctx)
 	<-n.Ready()
 	<-n.Ready()

+ 3 - 3
raft/raft.go

@@ -119,14 +119,14 @@ 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")
 	}
 	}
 	r := &raft{
 	r := &raft{
 		id:               id,
 		id:               id,
 		lead:             None,
 		lead:             None,
-		raftLog:          newLog(),
+		raftLog:          newLog(storage),
 		prs:              make(map[uint64]*progress),
 		prs:              make(map[uint64]*progress),
 		electionTimeout:  election,
 		electionTimeout:  election,
 		heartbeatTimeout: heartbeat,
 		heartbeatTimeout: heartbeat,
@@ -517,7 +517,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 {
+	if i < r.raftLog.firstIndex() {
 		if r.raftLog.snapshot.Term == 0 {
 		if r.raftLog.snapshot.Term == 0 {
 			panic("need non-empty snapshot")
 			panic("need non-empty snapshot")
 		}
 		}

+ 26 - 25
raft/raft_paper_test.go

@@ -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, nil)
 	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, nil)
 	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, nil)
 	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,7 +109,7 @@ 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, nil)
 	r.becomeCandidate()
 	r.becomeCandidate()
 	r.becomeLeader()
 	r.becomeLeader()
 	for i := 0; i < 10; i++ {
 	for i := 0; i < 10; 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, nil)
 	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, nil)
 
 
 		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, nil)
 		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, nil)
 		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, nil)
 	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, nil)
 	}
 	}
 	conflicts := 0
 	conflicts := 0
 	for round := 0; round < 1000; round++ {
 	for round := 0; round < 1000; round++ {
@@ -387,7 +387,7 @@ func testNonleadersElectionTimeoutNonconflict(t *testing.T, state StateType) {
 // Also, it writes the new entry into stable storage.
 // 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)
+	r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil)
 	r.becomeCandidate()
 	r.becomeCandidate()
 	r.becomeLeader()
 	r.becomeLeader()
 	commitNoopEntry(r)
 	commitNoopEntry(r)
@@ -412,7 +412,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,7 +425,7 @@ 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)
+	r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil)
 	r.becomeCandidate()
 	r.becomeCandidate()
 	r.becomeLeader()
 	r.becomeLeader()
 	commitNoopEntry(r)
 	commitNoopEntry(r)
@@ -478,7 +478,7 @@ 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)
+		r := newRaft(1, idsBySize(tt.size), 10, 1, nil)
 		r.becomeCandidate()
 		r.becomeCandidate()
 		r.becomeLeader()
 		r.becomeLeader()
 		commitNoopEntry(r)
 		commitNoopEntry(r)
@@ -510,7 +510,7 @@ 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)
+		r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil)
 		r.loadEnts(append([]pb.Entry{{}}, tt...))
 		r.loadEnts(append([]pb.Entry{{}}, tt...))
 		r.loadState(pb.HardState{Term: 2})
 		r.loadState(pb.HardState{Term: 2})
 		r.becomeCandidate()
 		r.becomeCandidate()
@@ -566,7 +566,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, nil)
 		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})
@@ -601,7 +601,7 @@ func TestFollowerCheckMsgApp(t *testing.T) {
 		{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)
+		r := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil)
 		r.loadEnts(ents)
 		r.loadEnts(ents)
 		r.loadState(pb.HardState{Commit: 2})
 		r.loadState(pb.HardState{Commit: 2})
 		r.becomeFollower(2, 2)
 		r.becomeFollower(2, 2)
@@ -656,16 +656,16 @@ func TestFollowerAppendEntries(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, nil)
 		r.loadEnts([]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}})
 		r.loadEnts([]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}})
 		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)
 		}
 		}
 	}
 	}
@@ -724,10 +724,10 @@ func TestLeaderSyncFollowerLog(t *testing.T) {
 		},
 		},
 	}
 	}
 	for i, tt := range tests {
 	for i, tt := range tests {
-		lead := newRaft(1, []uint64{1, 2, 3}, 10, 1)
+		lead := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil)
 		lead.loadEnts(ents)
 		lead.loadEnts(ents)
 		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)
+		follower := newRaft(2, []uint64{1, 2, 3}, 10, 1, nil)
 		follower.loadEnts(tt)
 		follower.loadEnts(tt)
 		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.
@@ -757,7 +757,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, nil)
 		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,
 		})
 		})
@@ -818,7 +818,7 @@ func TestVoter(t *testing.T) {
 		{[]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)
+		r := newRaft(1, []uint64{1, 2}, 10, 1, nil)
 		r.loadEnts(tt.ents)
 		r.loadEnts(tt.ents)
 
 
 		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})
@@ -853,7 +853,7 @@ 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)
+		r := newRaft(1, []uint64{1, 2}, 10, 1, nil)
 		r.loadEnts(ents)
 		r.loadEnts(ents)
 		r.loadState(pb.HardState{Term: 2})
 		r.loadState(pb.HardState{Term: 2})
 		// become leader at term 3
 		// become leader at term 3
@@ -891,6 +891,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()
+	r.raftLog.storage.(*MemoryStorage).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())
 }
 }

+ 91 - 42
raft/raft_test.go

@@ -30,6 +30,11 @@ import (
 
 
 // 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) (ents []pb.Entry) {
+	// Transfer all unstable entries to "stable" storage.
+	memStorage := r.raftLog.storage.(*MemoryStorage)
+	memStorage.Append(r.raftLog.unstableEntries())
+	r.raftLog.stableTo(r.raftLog.lastIndex())
+
 	ents = r.raftLog.nextEnts()
 	ents = r.raftLog.nextEnts()
 	r.raftLog.appliedTo(r.raftLog.committed)
 	r.raftLog.appliedTo(r.raftLog.committed)
 	return ents
 	return ents
@@ -280,9 +285,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)
-	b := newRaft(2, []uint64{1, 2, 3}, 10, 1)
-	c := newRaft(3, []uint64{1, 2, 3}, 10, 1)
+	a := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil)
+	b := newRaft(2, []uint64{1, 2, 3}, 10, 1, nil)
+	c := newRaft(3, []uint64{1, 2, 3}, 10, 1, nil)
 
 
 	nt := newNetwork(a, b, c)
 	nt := newNetwork(a, b, c)
 	nt.cut(1, 3)
 	nt.cut(1, 3)
@@ -293,7 +298,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
@@ -302,7 +311,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(nil)},
 	}
 	}
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
@@ -345,7 +354,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)
@@ -378,10 +393,13 @@ func TestOldMessages(t *testing.T) {
 	tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgApp, Term: 1, Entries: []pb.Entry{{Term: 1}}})
 	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{
-			{}, {Data: nil, Term: 1, Index: 1},
-			{Data: nil, Term: 2, Index: 2}, {Data: nil, Term: 3, Index: 3},
+		storage: &MemoryStorage{
+			ents: []pb.Entry{
+				{}, {Data: nil, Term: 1, Index: 1},
+				{Data: nil, Term: 2, Index: 2}, {Data: nil, Term: 3, Index: 3},
+			},
 		},
 		},
+		unstable:  4,
 		committed: 3,
 		committed: 3,
 	}
 	}
 	base := ltoa(l)
 	base := ltoa(l)
@@ -432,9 +450,14 @@ func TestProposal(t *testing.T) {
 		send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
 		send(pb.Message{From: 1, To: 1, Type: pb.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(nil)
 		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 {
@@ -468,7 +491,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 {
@@ -513,13 +541,15 @@ func TestCompact(t *testing.T) {
 				raftLog: &raftLog{
 				raftLog: &raftLog{
 					committed: 2,
 					committed: 2,
 					applied:   2,
 					applied:   2,
-					ents:      []pb.Entry{{}, {Term: 1}, {Term: 1}, {Term: 1}},
+					storage: &MemoryStorage{
+						ents: []pb.Entry{{}, {Term: 1}, {Term: 1}, {Term: 1}},
+					},
 				},
 				},
 			}
 			}
 			sm.compact(tt.compacti, tt.nodes, tt.snapd)
 			sm.compact(tt.compacti, tt.nodes, tt.snapd)
 			sort.Sort(uint64Slice(sm.raftLog.snapshot.Nodes))
 			sort.Sort(uint64Slice(sm.raftLog.snapshot.Nodes))
-			if sm.raftLog.offset != tt.compacti {
-				t.Errorf("%d: log.offset = %d, want %d", i, sm.raftLog.offset, tt.compacti)
+			if sm.raftLog.firstIndex() != tt.compacti {
+				t.Errorf("%d: log.firstIndex = %d, want %d", i, sm.raftLog.firstIndex(), tt.compacti)
 			}
 			}
 			if !reflect.DeepEqual(sm.raftLog.snapshot.Nodes, tt.nodes) {
 			if !reflect.DeepEqual(sm.raftLog.snapshot.Nodes, tt.nodes) {
 				t.Errorf("%d: snap.nodes = %v, want %v", i, sm.raftLog.snapshot.Nodes, tt.nodes)
 				t.Errorf("%d: snap.nodes = %v, want %v", i, sm.raftLog.snapshot.Nodes, tt.nodes)
@@ -564,7 +594,11 @@ func TestCommit(t *testing.T) {
 		for j := 0; j < len(tt.matches); j++ {
 		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)
@@ -586,7 +620,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, nil)
 		sm.elapsed = tt.elapse
 		sm.elapsed = tt.elapse
 		c := 0
 		c := 0
 		for j := 0; j < 10000; j++ {
 		for j := 0; j < 10000; j++ {
@@ -611,7 +645,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, nil)
 	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})
@@ -654,7 +688,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)
@@ -709,7 +747,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, nil)
 		sm.state = tt.state
 		sm.state = tt.state
 		switch tt.state {
 		switch tt.state {
 		case StateFollower:
 		case StateFollower:
@@ -720,7 +758,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{{}, {Term: 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})
 
 
@@ -766,7 +807,7 @@ func TestStateTransition(t *testing.T) {
 				}
 				}
 			}()
 			}()
 
 
-			sm := newRaft(1, []uint64{1}, 10, 1)
+			sm := newRaft(1, []uint64{1}, 10, 1, nil)
 			sm.state = tt.from
 			sm.state = tt.from
 
 
 			switch tt.to {
 			switch tt.to {
@@ -805,7 +846,7 @@ func TestAllServerStepdown(t *testing.T) {
 	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, nil)
 		switch tt.state {
 		switch tt.state {
 		case StateFollower:
 		case StateFollower:
 			sm.becomeFollower(1, None)
 			sm.becomeFollower(1, None)
@@ -825,8 +866,8 @@ 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 {
-				t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.ents), tt.windex)
+			if uint64(len(sm.raftLog.allEntries())) != tt.windex {
+				t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.allEntries()), tt.windex)
 			}
 			}
 			wlead := uint64(2)
 			wlead := uint64(2)
 			if msgType == pb.MsgVote {
 			if msgType == pb.MsgVote {
@@ -861,8 +902,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.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}
+		sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil)
+		sm.raftLog = &raftLog{
+			storage:  &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}},
+			unstable: 3,
+		}
 		sm.becomeCandidate()
 		sm.becomeCandidate()
 		sm.becomeLeader()
 		sm.becomeLeader()
 		sm.readMessages()
 		sm.readMessages()
@@ -902,7 +946,7 @@ func TestBcastBeat(t *testing.T) {
 		Term:  1,
 		Term:  1,
 		Nodes: []uint64{1, 2, 3},
 		Nodes: []uint64{1, 2, 3},
 	}
 	}
-	sm := newRaft(1, []uint64{1, 2, 3}, 10, 1)
+	sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil)
 	sm.Term = 1
 	sm.Term = 1
 	sm.restore(s)
 	sm.restore(s)
 
 
@@ -952,8 +996,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.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}
+		sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, nil)
+		sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}}
 		sm.Term = 1
 		sm.Term = 1
 		sm.state = tt.state
 		sm.state = tt.state
 		switch tt.state {
 		switch tt.state {
@@ -985,7 +1029,7 @@ func TestRestore(t *testing.T) {
 		Nodes: []uint64{1, 2, 3},
 		Nodes: []uint64{1, 2, 3},
 	}
 	}
 
 
-	sm := newRaft(1, []uint64{1, 2}, 10, 1)
+	sm := newRaft(1, []uint64{1, 2}, 10, 1, nil)
 	if ok := sm.restore(s); !ok {
 	if ok := sm.restore(s); !ok {
 		t.Fatal("restore fail, want succeed")
 		t.Fatal("restore fail, want succeed")
 	}
 	}
@@ -1016,7 +1060,7 @@ func TestProvideSnap(t *testing.T) {
 		Term:  11, // magic number
 		Term:  11, // magic number
 		Nodes: []uint64{1, 2},
 		Nodes: []uint64{1, 2},
 	}
 	}
-	sm := newRaft(1, []uint64{1}, 10, 1)
+	sm := newRaft(1, []uint64{1}, 10, 1, nil)
 	// restore the statemachin from a snapshot
 	// restore the statemachin from a snapshot
 	// so it has a compacted log and a snapshot
 	// so it has a compacted log and a snapshot
 	sm.restore(s)
 	sm.restore(s)
@@ -1026,7 +1070,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()
@@ -1047,7 +1091,7 @@ func TestRestoreFromSnapMsg(t *testing.T) {
 	}
 	}
 	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, nil)
 	sm.Step(m)
 	sm.Step(m)
 
 
 	if !reflect.DeepEqual(sm.raftLog.snapshot, s) {
 	if !reflect.DeepEqual(sm.raftLog.snapshot, s) {
@@ -1086,7 +1130,7 @@ func TestSlowNodeRestore(t *testing.T) {
 // it appends the entry to log and sets pendingConf to be true.
 // 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, nil)
 	r.becomeCandidate()
 	r.becomeCandidate()
 	r.becomeLeader()
 	r.becomeLeader()
 	index := r.raftLog.lastIndex()
 	index := r.raftLog.lastIndex()
@@ -1104,7 +1148,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, nil)
 	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}}})
@@ -1130,7 +1174,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, nil)
 		r.appendEntry(pb.Entry{Type: tt.entType})
 		r.appendEntry(pb.Entry{Type: tt.entType})
 		r.becomeCandidate()
 		r.becomeCandidate()
 		r.becomeLeader()
 		r.becomeLeader()
@@ -1149,7 +1193,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, nil)
 		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()
@@ -1159,7 +1203,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, nil)
 	r.pendingConf = true
 	r.pendingConf = true
 	r.addNode(2)
 	r.addNode(2)
 	if r.pendingConf != false {
 	if r.pendingConf != false {
@@ -1176,7 +1220,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, nil)
 	r.pendingConf = true
 	r.pendingConf = true
 	r.removeNode(2)
 	r.removeNode(2)
 	if r.pendingConf != false {
 	if r.pendingConf != false {
@@ -1216,7 +1260,12 @@ func ents(terms ...uint64) *raft {
 		ents = append(ents, pb.Entry{Term: term})
 		ents = append(ents, pb.Entry{Term: term})
 	}
 	}
 
 
-	sm := &raft{raftLog: &raftLog{ents: ents}}
+	sm := &raft{
+		raftLog: &raftLog{
+			storage:  &MemoryStorage{ents: ents},
+			unstable: uint64(len(ents)),
+		},
+	}
 	sm.reset(0)
 	sm.reset(0)
 	return sm
 	return sm
 }
 }
@@ -1241,7 +1290,7 @@ func newNetwork(peers ...Interface) *network {
 		id := peerAddrs[i]
 		id := peerAddrs[i]
 		switch v := p.(type) {
 		switch v := p.(type) {
 		case nil:
 		case nil:
-			sm := newRaft(id, peerAddrs, 10, 1)
+			sm := newRaft(id, peerAddrs, 10, 1, nil)
 			npeers[id] = sm
 			npeers[id] = sm
 		case *raft:
 		case *raft:
 			v.id = id
 			v.id = id

+ 108 - 0
raft/storage.go

@@ -0,0 +1,108 @@
+/*
+   Copyright 2014 CoreOS, Inc.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+*/
+
+package raft
+
+import (
+	"sync"
+
+	pb "github.com/coreos/etcd/raft/raftpb"
+)
+
+// Storage is an interface that may be implemented by the application
+// to retrieve log entries from storage. If no storage implementation
+// is supplied by the application, a MemoryStorage will be used, which
+// retains all log entries in memory.
+//
+// If any Storage method returns an error, the raft instance will
+// become inoperable and refuse to participate in elections; the
+// application is responsible for cleanup and recovery in this case.
+type Storage interface {
+	// GetEntries returns a slice of log entries in the range [lo,hi).
+	GetEntries(lo, hi uint64) ([]pb.Entry, error)
+	// GetLastIndex returns the index of the last entry in the log.
+	GetLastIndex() (uint64, error)
+	// GetFirstIndex returns the index of the first log entry that is
+	// available via GetEntries (older entries have been incorporated
+	// into the latest Snapshot).
+	GetFirstIndex() (uint64, error)
+	// Compact discards all log entries prior to i, creating a snapshot
+	// which can be used to reconstruct the state at that point.
+	Compact(i uint64) error
+}
+
+// MemoryStorage implements the Storage interface backed by an
+// in-memory array.
+type MemoryStorage struct {
+	// Protects access to all fields. Most methods of MemoryStorage are
+	// run on the raft goroutine, but Append() is run on an application
+	// goroutine.
+	sync.Mutex
+
+	ents []pb.Entry
+	// offset is the position of the last compaction.
+	// ents[i] has raft log position i+offset.
+	offset uint64
+}
+
+// NewMemoryStorage creates an empty MemoryStorage.
+func NewMemoryStorage() *MemoryStorage {
+	return &MemoryStorage{
+		// Populate the list with a dummy entry at term zero.
+		ents: make([]pb.Entry, 1),
+	}
+}
+
+// GetEntries implements the Storage interface.
+func (ms *MemoryStorage) GetEntries(lo, hi uint64) ([]pb.Entry, error) {
+	ms.Lock()
+	defer ms.Unlock()
+	return ms.ents[lo-ms.offset : hi-ms.offset], nil
+}
+
+// GetLastIndex implements the Storage interface.
+func (ms *MemoryStorage) GetLastIndex() (uint64, error) {
+	ms.Lock()
+	defer ms.Unlock()
+	return ms.offset + uint64(len(ms.ents)) - 1, nil
+}
+
+// GetFirstIndex implements the Storage interface.
+func (ms *MemoryStorage) GetFirstIndex() (uint64, error) {
+	ms.Lock()
+	defer ms.Unlock()
+	return ms.offset, nil
+}
+
+// Compact implements the Storage interface.
+func (ms *MemoryStorage) Compact(i uint64) error {
+	ms.Lock()
+	defer ms.Unlock()
+	i -= ms.offset
+	ents := make([]pb.Entry, 1, 1+uint64(len(ms.ents))-i)
+	ents[0].Term = ms.ents[i].Term
+	ents = append(ents, ms.ents[i+1:]...)
+	ms.ents = ents
+	ms.offset += i
+	return nil
+}
+
+// Append the new entries to storage.
+func (ms *MemoryStorage) Append(entries []pb.Entry) {
+	ms.Lock()
+	defer ms.Unlock()
+	ms.ents = append(ms.ents, entries...)
+}

+ 54 - 0
raft/util.go

@@ -0,0 +1,54 @@
+/*
+   Copyright 2014 CoreOS, Inc.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+*/
+
+package raft
+
+import (
+	"bytes"
+	"fmt"
+
+	pb "github.com/coreos/etcd/raft/raftpb"
+)
+
+// DescribeMessage returns a concise human-readable description of a
+// Message for debugging.
+func DescribeMessage(m pb.Message) string {
+	var buf bytes.Buffer
+	fmt.Fprintf(&buf, "%d->%d %s Term:%d Log:%d/%d", m.From, m.To, m.Type, m.Term, m.LogTerm, m.Index)
+	if m.Reject {
+		fmt.Fprintf(&buf, " Rejected")
+	}
+	if m.Commit != 0 {
+		fmt.Fprintf(&buf, " Commit:%d", m.Commit)
+	}
+	if len(m.Entries) > 0 {
+		fmt.Fprintf(&buf, " Entries:[")
+		for _, e := range m.Entries {
+			buf.WriteString(DescribeEntry(e))
+		}
+		fmt.Fprintf(&buf, "]")
+	}
+	if !IsEmptySnap(m.Snapshot) {
+		fmt.Fprintf(&buf, " Snapshot:%v", m.Snapshot)
+	}
+	return buf.String()
+}
+
+// DescribeEntry returns a concise human-readable description of an
+// Entry for debugging.
+func DescribeEntry(e pb.Entry) string {
+	return fmt.Sprintf("%d/%d %s %q", e.Term, e.Index, e.Type, string(e.Data))
+}

+ 1 - 1
test

@@ -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 10s ${COVER} $@ ${TEST} --race
 
 
 echo "Checking gofmt..."
 echo "Checking gofmt..."
 fmtRes=$(gofmt -l $FMT)
 fmtRes=$(gofmt -l $FMT)