mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge branch 'log_interface'
Conflicts: raft/log.go
This commit is contained in:
commit
ab2a40ea37
@ -60,7 +60,7 @@ func handleBackup(c *cli.Context) {
|
||||
}
|
||||
var index uint64
|
||||
if snapshot != nil {
|
||||
index = snapshot.Index
|
||||
index = snapshot.Metadata.Index
|
||||
newss := snap.New(destSnap)
|
||||
if err := newss.SaveSnap(*snapshot); err != nil {
|
||||
log.Fatal(err)
|
||||
|
@ -28,7 +28,7 @@ import (
|
||||
"github.com/coreos/etcd/wal"
|
||||
)
|
||||
|
||||
func restartAsStandaloneNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (types.ID, raft.Node, *wal.WAL) {
|
||||
func restartAsStandaloneNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (types.ID, raft.Node, *raft.MemoryStorage, *wal.WAL) {
|
||||
w, id, cid, st, ents := readWAL(cfg.WALDir(), index)
|
||||
cfg.Cluster.SetID(cid)
|
||||
|
||||
@ -57,8 +57,14 @@ func restartAsStandaloneNode(cfg *ServerConfig, index uint64, snapshot *raftpb.S
|
||||
}
|
||||
|
||||
log.Printf("etcdserver: forcing restart of member %s in cluster %s at commit index %d", id, cfg.Cluster.ID(), st.Commit)
|
||||
n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents)
|
||||
return id, n, w
|
||||
s := raft.NewMemoryStorage()
|
||||
if snapshot != nil {
|
||||
s.ApplySnapshot(*snapshot)
|
||||
}
|
||||
s.SetHardState(st)
|
||||
s.Append(ents)
|
||||
n := raft.RestartNode(uint64(id), 10, 1, s)
|
||||
return id, n, s, w
|
||||
}
|
||||
|
||||
// getIDs returns an ordered set of IDs included in the given snapshot and
|
||||
@ -69,7 +75,7 @@ func restartAsStandaloneNode(cfg *ServerConfig, index uint64, snapshot *raftpb.S
|
||||
func getIDs(snap *raftpb.Snapshot, ents []raftpb.Entry) []uint64 {
|
||||
ids := make(map[uint64]bool)
|
||||
if snap != nil {
|
||||
for _, id := range snap.Nodes {
|
||||
for _, id := range snap.Metadata.ConfState.Nodes {
|
||||
ids[id] = true
|
||||
}
|
||||
}
|
||||
|
@ -34,21 +34,30 @@ func TestGetIDs(t *testing.T) {
|
||||
normalEntry := raftpb.Entry{Type: raftpb.EntryNormal}
|
||||
|
||||
tests := []struct {
|
||||
snap *raftpb.Snapshot
|
||||
ents []raftpb.Entry
|
||||
confState *raftpb.ConfState
|
||||
ents []raftpb.Entry
|
||||
|
||||
widSet []uint64
|
||||
}{
|
||||
{nil, []raftpb.Entry{}, []uint64{}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{}, []uint64{1}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry}, []uint64{1, 2}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry, removeEntry}, []uint64{1}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry, normalEntry}, []uint64{1, 2}},
|
||||
{&raftpb.Snapshot{Nodes: []uint64{1}}, []raftpb.Entry{addEntry, removeEntry, normalEntry}, []uint64{1}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{}, []uint64{1}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{addEntry}, []uint64{1, 2}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{addEntry, removeEntry}, []uint64{1}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{addEntry, normalEntry}, []uint64{1, 2}},
|
||||
{&raftpb.ConfState{Nodes: []uint64{1}},
|
||||
[]raftpb.Entry{addEntry, removeEntry, normalEntry}, []uint64{1}},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
idSet := getIDs(tt.snap, tt.ents)
|
||||
var snap raftpb.Snapshot
|
||||
if tt.confState != nil {
|
||||
snap.Metadata.ConfState = *tt.confState
|
||||
}
|
||||
idSet := getIDs(&snap, tt.ents)
|
||||
if !reflect.DeepEqual(idSet, tt.widSet) {
|
||||
t.Errorf("#%d: idset = %#v, want %#v", i, idSet, tt.widSet)
|
||||
}
|
||||
|
@ -165,8 +165,9 @@ type EtcdServer struct {
|
||||
|
||||
Cluster *Cluster
|
||||
|
||||
node raft.Node
|
||||
store store.Store
|
||||
node raft.Node
|
||||
raftStorage *raft.MemoryStorage
|
||||
store store.Store
|
||||
|
||||
stats *stats.ServerStats
|
||||
lstats *stats.LeaderStats
|
||||
@ -211,6 +212,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
||||
st := store.New()
|
||||
var w *wal.WAL
|
||||
var n raft.Node
|
||||
var s *raft.MemoryStorage
|
||||
var id types.ID
|
||||
walVersion := wal.DetectVersion(cfg.DataDir)
|
||||
if walVersion == wal.WALUnknown {
|
||||
@ -243,7 +245,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
||||
cfg.Cluster.SetID(existingCluster.id)
|
||||
cfg.Cluster.SetStore(st)
|
||||
cfg.Print()
|
||||
id, n, w = startNode(cfg, nil)
|
||||
id, n, s, w = startNode(cfg, nil)
|
||||
case !haveWAL && cfg.NewCluster:
|
||||
if err := cfg.VerifyBootstrapConfig(); err != nil {
|
||||
return nil, err
|
||||
@ -263,7 +265,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
||||
}
|
||||
cfg.Cluster.SetStore(st)
|
||||
cfg.PrintWithInitial()
|
||||
id, n, w = startNode(cfg, cfg.Cluster.MemberIDs())
|
||||
id, n, s, w = startNode(cfg, cfg.Cluster.MemberIDs())
|
||||
case haveWAL:
|
||||
if cfg.ShouldDiscover() {
|
||||
log.Printf("etcdserver: warn: ignoring discovery: etcd has already been initialized and has a valid log in %q", cfg.WALDir())
|
||||
@ -274,9 +276,9 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
||||
return nil, err
|
||||
}
|
||||
if snapshot != nil {
|
||||
log.Printf("etcdserver: recovering from snapshot at index %d", snapshot.Index)
|
||||
log.Printf("etcdserver: recovering from snapshot at index %d", snapshot.Metadata.Index)
|
||||
st.Recovery(snapshot.Data)
|
||||
index = snapshot.Index
|
||||
index = snapshot.Metadata.Index
|
||||
}
|
||||
cfg.Cluster = NewClusterFromStore(cfg.Cluster.token, st)
|
||||
cfg.Print()
|
||||
@ -284,9 +286,9 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
||||
log.Printf("etcdserver: loaded peers from snapshot: %s", cfg.Cluster)
|
||||
}
|
||||
if !cfg.ForceNewCluster {
|
||||
id, n, w = restartNode(cfg, index, snapshot)
|
||||
id, n, s, w = restartNode(cfg, index+1, snapshot)
|
||||
} else {
|
||||
id, n, w = restartAsStandaloneNode(cfg, index, snapshot)
|
||||
id, n, s, w = restartAsStandaloneNode(cfg, index+1, snapshot)
|
||||
}
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported bootstrap config")
|
||||
@ -298,12 +300,13 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
||||
}
|
||||
lstats := stats.NewLeaderStats(id.String())
|
||||
|
||||
s := &EtcdServer{
|
||||
store: st,
|
||||
node: n,
|
||||
id: id,
|
||||
attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
|
||||
Cluster: cfg.Cluster,
|
||||
srv := &EtcdServer{
|
||||
store: st,
|
||||
node: n,
|
||||
raftStorage: s,
|
||||
id: id,
|
||||
attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
|
||||
Cluster: cfg.Cluster,
|
||||
storage: struct {
|
||||
*wal.WAL
|
||||
*snap.Snapshotter
|
||||
@ -314,8 +317,8 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
||||
SyncTicker: time.Tick(500 * time.Millisecond),
|
||||
snapCount: cfg.SnapCount,
|
||||
}
|
||||
s.sendhub = newSendHub(cfg.Transport, cfg.Cluster, s, sstats, lstats)
|
||||
return s, nil
|
||||
srv.sendhub = newSendHub(cfg.Transport, cfg.Cluster, srv, sstats, lstats)
|
||||
return srv, nil
|
||||
}
|
||||
|
||||
// Start prepares and starts server in a new goroutine. It is no longer safe to
|
||||
@ -386,21 +389,31 @@ func (s *EtcdServer) run() {
|
||||
}
|
||||
}
|
||||
|
||||
// apply snapshot to storage if it is more updated than current snapi
|
||||
if !raft.IsEmptySnap(rd.Snapshot) && rd.Snapshot.Metadata.Index > snapi {
|
||||
if err := s.storage.SaveSnap(rd.Snapshot); err != nil {
|
||||
log.Fatalf("etcdserver: create snapshot error: %v", err)
|
||||
}
|
||||
s.raftStorage.ApplySnapshot(rd.Snapshot)
|
||||
snapi = rd.Snapshot.Metadata.Index
|
||||
}
|
||||
|
||||
if err := s.storage.Save(rd.HardState, rd.Entries); err != nil {
|
||||
log.Fatalf("etcdserver: save state and entries error: %v", err)
|
||||
}
|
||||
if err := s.storage.SaveSnap(rd.Snapshot); err != nil {
|
||||
log.Fatalf("etcdserver: create snapshot error: %v", err)
|
||||
}
|
||||
s.raftStorage.Append(rd.Entries)
|
||||
|
||||
s.sendhub.Send(rd.Messages)
|
||||
|
||||
// recover from snapshot if it is more updated than current applied
|
||||
if rd.Snapshot.Index > appliedi {
|
||||
if err := s.store.Recovery(rd.Snapshot.Data); err != nil {
|
||||
log.Panicf("recovery store error: %v", err)
|
||||
if !raft.IsEmptySnap(rd.Snapshot) && rd.Snapshot.Metadata.Index > appliedi {
|
||||
{
|
||||
if err := s.store.Recovery(rd.Snapshot.Data); err != nil {
|
||||
log.Panicf("recovery store error: %v", err)
|
||||
}
|
||||
s.Cluster.Recover()
|
||||
appliedi = rd.Snapshot.Metadata.Index
|
||||
}
|
||||
s.Cluster.Recover()
|
||||
appliedi = rd.Snapshot.Index
|
||||
}
|
||||
// TODO(bmizerany): do this in the background, but take
|
||||
// care to apply entries in a single goroutine, and not
|
||||
@ -409,6 +422,7 @@ func (s *EtcdServer) run() {
|
||||
firsti := rd.CommittedEntries[0].Index
|
||||
if appliedi == 0 {
|
||||
appliedi = firsti - 1
|
||||
snapi = appliedi
|
||||
}
|
||||
if firsti > appliedi+1 {
|
||||
log.Panicf("etcdserver: first index of committed entry[%d] should <= appliedi[%d] + 1", firsti, appliedi)
|
||||
@ -424,9 +438,6 @@ func (s *EtcdServer) run() {
|
||||
|
||||
s.node.Advance()
|
||||
|
||||
if rd.Snapshot.Index > snapi {
|
||||
snapi = rd.Snapshot.Index
|
||||
}
|
||||
if appliedi-snapi > s.snapCount {
|
||||
s.snapshot(appliedi, nodes)
|
||||
snapi = appliedi
|
||||
@ -798,10 +809,26 @@ func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) {
|
||||
if err != nil {
|
||||
log.Panicf("store save should never fail: %v", err)
|
||||
}
|
||||
s.node.Compact(snapi, snapnodes, d)
|
||||
err = s.raftStorage.Compact(snapi, &raftpb.ConfState{Nodes: snapnodes}, d)
|
||||
if err != nil {
|
||||
// the snapshot was done asynchronously with the progress of raft.
|
||||
// raft might have already got a newer snapshot and called compact.
|
||||
if err == raft.ErrCompacted {
|
||||
return
|
||||
}
|
||||
log.Panicf("etcdserver: unexpected compaction error %v", err)
|
||||
}
|
||||
|
||||
if err := s.storage.Cut(); err != nil {
|
||||
log.Panicf("rotate wal file should never fail: %v", err)
|
||||
}
|
||||
snap, err := s.raftStorage.Snapshot()
|
||||
if err != nil {
|
||||
log.Panicf("etcdserver: snapshot error: %v", err)
|
||||
}
|
||||
if err := s.storage.SaveSnap(snap); err != nil {
|
||||
log.Fatalf("etcdserver: create snapshot error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// checkClientURLsEmptyFromPeers does its best to get the cluster from peers,
|
||||
@ -875,7 +902,7 @@ func getClusterFromPeers(urls []string, logerr bool) (*Cluster, error) {
|
||||
return nil, fmt.Errorf("etcdserver: could not retrieve cluster information from the given urls")
|
||||
}
|
||||
|
||||
func startNode(cfg *ServerConfig, ids []types.ID) (id types.ID, n raft.Node, w *wal.WAL) {
|
||||
func startNode(cfg *ServerConfig, ids []types.ID) (id types.ID, n raft.Node, s *raft.MemoryStorage, w *wal.WAL) {
|
||||
var err error
|
||||
member := cfg.Cluster.MemberByName(cfg.Name)
|
||||
metadata := pbutil.MustMarshal(
|
||||
@ -897,7 +924,8 @@ func startNode(cfg *ServerConfig, ids []types.ID) (id types.ID, n raft.Node, w *
|
||||
}
|
||||
id = member.ID
|
||||
log.Printf("etcdserver: start member %s in cluster %s", id, cfg.Cluster.ID())
|
||||
n = raft.StartNode(uint64(id), peers, 10, 1)
|
||||
s = raft.NewMemoryStorage()
|
||||
n = raft.StartNode(uint64(id), peers, 10, 1, s)
|
||||
return
|
||||
}
|
||||
|
||||
@ -915,13 +943,19 @@ func getOtherPeerURLs(cl ClusterInfo, self string) []string {
|
||||
return us
|
||||
}
|
||||
|
||||
func restartNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (types.ID, raft.Node, *wal.WAL) {
|
||||
func restartNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (types.ID, raft.Node, *raft.MemoryStorage, *wal.WAL) {
|
||||
w, id, cid, st, ents := readWAL(cfg.WALDir(), index)
|
||||
cfg.Cluster.SetID(cid)
|
||||
|
||||
log.Printf("etcdserver: restart member %s in cluster %s at commit index %d", id, cfg.Cluster.ID(), st.Commit)
|
||||
n := raft.RestartNode(uint64(id), 10, 1, snapshot, st, ents)
|
||||
return id, n, w
|
||||
s := raft.NewMemoryStorage()
|
||||
if snapshot != nil {
|
||||
s.ApplySnapshot(*snapshot)
|
||||
}
|
||||
s.SetHardState(st)
|
||||
s.Append(ents)
|
||||
n := raft.RestartNode(uint64(id), 10, 1, s)
|
||||
return id, n, s, w
|
||||
}
|
||||
|
||||
func readWAL(waldir string, index uint64) (w *wal.WAL, id, cid types.ID, st raftpb.HardState, ents []raftpb.Entry) {
|
||||
|
@ -560,24 +560,30 @@ func testServer(t *testing.T, ns uint64) {
|
||||
members := mustMakePeerSlice(t, ids...)
|
||||
for i := uint64(0); i < ns; i++ {
|
||||
id := i + 1
|
||||
n := raft.StartNode(id, members, 10, 1)
|
||||
s := raft.NewMemoryStorage()
|
||||
n := raft.StartNode(id, members, 10, 1, s)
|
||||
tk := time.NewTicker(10 * time.Millisecond)
|
||||
defer tk.Stop()
|
||||
st := store.New()
|
||||
cl := newCluster("abc")
|
||||
cl.SetStore(st)
|
||||
srv := &EtcdServer{
|
||||
node: n,
|
||||
store: st,
|
||||
sendhub: &fakeSender{ss},
|
||||
storage: &storageRecorder{},
|
||||
Ticker: tk.C,
|
||||
Cluster: cl,
|
||||
node: n,
|
||||
raftStorage: s,
|
||||
store: st,
|
||||
sendhub: &fakeSender{ss},
|
||||
storage: &storageRecorder{},
|
||||
Ticker: tk.C,
|
||||
Cluster: cl,
|
||||
}
|
||||
srv.start()
|
||||
ss[i] = srv
|
||||
}
|
||||
|
||||
// Start the servers after they're all created to avoid races in send().
|
||||
for i := uint64(0); i < ns; i++ {
|
||||
ss[i].start()
|
||||
}
|
||||
|
||||
for i := 1; i <= 10; i++ {
|
||||
r := pb.Request{
|
||||
Method: "PUT",
|
||||
@ -628,7 +634,8 @@ func TestDoProposal(t *testing.T) {
|
||||
|
||||
for i, tt := range tests {
|
||||
ctx, _ := context.WithCancel(context.Background())
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1)
|
||||
s := raft.NewMemoryStorage()
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s)
|
||||
st := &storeRecorder{}
|
||||
tk := make(chan time.Time)
|
||||
// this makes <-tk always successful, which accelerates internal clock
|
||||
@ -636,12 +643,13 @@ func TestDoProposal(t *testing.T) {
|
||||
cl := newCluster("abc")
|
||||
cl.SetStore(store.New())
|
||||
srv := &EtcdServer{
|
||||
node: n,
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Ticker: tk,
|
||||
Cluster: cl,
|
||||
node: n,
|
||||
raftStorage: s,
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Ticker: tk,
|
||||
Cluster: cl,
|
||||
}
|
||||
srv.start()
|
||||
resp, err := srv.Do(ctx, tt)
|
||||
@ -664,14 +672,16 @@ func TestDoProposal(t *testing.T) {
|
||||
func TestDoProposalCancelled(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
// node cannot make any progress because there are two nodes
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0, 0xBAD1), 10, 1)
|
||||
s := raft.NewMemoryStorage()
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0, 0xBAD1), 10, 1, s)
|
||||
st := &storeRecorder{}
|
||||
wait := &waitRecorder{}
|
||||
srv := &EtcdServer{
|
||||
// TODO: use fake node for better testability
|
||||
node: n,
|
||||
store: st,
|
||||
w: wait,
|
||||
node: n,
|
||||
raftStorage: s,
|
||||
store: st,
|
||||
w: wait,
|
||||
}
|
||||
|
||||
done := make(chan struct{})
|
||||
@ -712,7 +722,8 @@ func TestDoProposalStopped(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
// node cannot make any progress because there are two nodes
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0, 0xBAD1), 10, 1)
|
||||
s := raft.NewMemoryStorage()
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0, 0xBAD1), 10, 1, s)
|
||||
st := &storeRecorder{}
|
||||
tk := make(chan time.Time)
|
||||
// this makes <-tk always successful, which accelarates internal clock
|
||||
@ -721,12 +732,13 @@ func TestDoProposalStopped(t *testing.T) {
|
||||
cl.SetStore(store.New())
|
||||
srv := &EtcdServer{
|
||||
// TODO: use fake node for better testability
|
||||
node: n,
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Ticker: tk,
|
||||
Cluster: cl,
|
||||
node: n,
|
||||
raftStorage: s,
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Ticker: tk,
|
||||
Cluster: cl,
|
||||
}
|
||||
srv.start()
|
||||
|
||||
@ -832,11 +844,12 @@ func TestSyncTrigger(t *testing.T) {
|
||||
}
|
||||
st := make(chan time.Time, 1)
|
||||
srv := &EtcdServer{
|
||||
node: n,
|
||||
store: &storeRecorder{},
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
SyncTicker: st,
|
||||
node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
store: &storeRecorder{},
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
SyncTicker: st,
|
||||
}
|
||||
srv.start()
|
||||
// trigger the server to become a leader and accept sync requests
|
||||
@ -866,17 +879,45 @@ func TestSyncTrigger(t *testing.T) {
|
||||
// snapshot should snapshot the store and cut the persistent
|
||||
// TODO: node.Compact is called... we need to make the node an interface
|
||||
func TestSnapshot(t *testing.T) {
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1)
|
||||
s := raft.NewMemoryStorage()
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s)
|
||||
defer n.Stop()
|
||||
|
||||
// Progress the node to the point where it has something to snapshot.
|
||||
// TODO(bdarnell): this could be improved with changes in the raft internals.
|
||||
// First, we must apply the initial conf changes so we can have an election.
|
||||
rd := <-n.Ready()
|
||||
s.Append(rd.Entries)
|
||||
for _, e := range rd.CommittedEntries {
|
||||
if e.Type == raftpb.EntryConfChange {
|
||||
var cc raftpb.ConfChange
|
||||
err := cc.Unmarshal(e.Data)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
n.ApplyConfChange(cc)
|
||||
}
|
||||
}
|
||||
n.Advance()
|
||||
|
||||
// Now we can have an election and persist the rest of the log.
|
||||
// This causes HardState.Commit to advance. HardState.Commit must
|
||||
// be > 0 to snapshot.
|
||||
n.Campaign(context.Background())
|
||||
rd = <-n.Ready()
|
||||
s.Append(rd.Entries)
|
||||
n.Advance()
|
||||
|
||||
st := &storeRecorder{}
|
||||
p := &storageRecorder{}
|
||||
s := &EtcdServer{
|
||||
store: st,
|
||||
storage: p,
|
||||
node: n,
|
||||
srv := &EtcdServer{
|
||||
store: st,
|
||||
storage: p,
|
||||
node: n,
|
||||
raftStorage: s,
|
||||
}
|
||||
|
||||
s.snapshot(0, []uint64{1})
|
||||
srv.snapshot(1, []uint64{1})
|
||||
gaction := st.Action()
|
||||
if len(gaction) != 1 {
|
||||
t.Fatalf("len(action) = %d, want 1", len(gaction))
|
||||
@ -886,19 +927,24 @@ func TestSnapshot(t *testing.T) {
|
||||
}
|
||||
|
||||
gaction = p.Action()
|
||||
if len(gaction) != 1 {
|
||||
t.Fatalf("len(action) = %d, want 1", len(gaction))
|
||||
if len(gaction) != 2 {
|
||||
t.Fatalf("len(action) = %d, want 2", len(gaction))
|
||||
}
|
||||
if !reflect.DeepEqual(gaction[0], action{name: "Cut"}) {
|
||||
t.Errorf("action = %s, want Cut", gaction[0])
|
||||
}
|
||||
if !reflect.DeepEqual(gaction[1], action{name: "SaveSnap"}) {
|
||||
t.Errorf("action = %s, want SaveSnap", gaction[1])
|
||||
}
|
||||
}
|
||||
|
||||
// Applied > SnapCount should trigger a SaveSnap event
|
||||
func TestTriggerSnap(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1)
|
||||
<-n.Ready()
|
||||
s := raft.NewMemoryStorage()
|
||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1, s)
|
||||
rd := <-n.Ready()
|
||||
s.Append(rd.Entries)
|
||||
n.Advance()
|
||||
n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 0xBAD0})
|
||||
n.Campaign(ctx)
|
||||
@ -906,26 +952,30 @@ func TestTriggerSnap(t *testing.T) {
|
||||
p := &storageRecorder{}
|
||||
cl := newCluster("abc")
|
||||
cl.SetStore(store.New())
|
||||
s := &EtcdServer{
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: p,
|
||||
node: n,
|
||||
snapCount: 10,
|
||||
Cluster: cl,
|
||||
srv := &EtcdServer{
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: p,
|
||||
node: n,
|
||||
raftStorage: s,
|
||||
snapCount: 10,
|
||||
Cluster: cl,
|
||||
}
|
||||
|
||||
s.start()
|
||||
for i := 0; uint64(i) < s.snapCount-1; i++ {
|
||||
s.Do(ctx, pb.Request{Method: "PUT", ID: 1})
|
||||
}
|
||||
srv.start()
|
||||
// wait for saving nop
|
||||
time.Sleep(time.Millisecond)
|
||||
s.Stop()
|
||||
for i := 0; uint64(i) < srv.snapCount-1; i++ {
|
||||
srv.Do(ctx, pb.Request{Method: "PUT", ID: 1})
|
||||
}
|
||||
// wait for saving the last entry
|
||||
time.Sleep(time.Millisecond)
|
||||
srv.Stop()
|
||||
|
||||
gaction := p.Action()
|
||||
// each operation is recorded as a Save
|
||||
// BootstrapConfig/Nop + (SnapCount - 1) * Puts + Cut + SaveSnap = Save + (SnapCount - 1) * Save + Cut + SaveSnap
|
||||
wcnt := 2 + int(s.snapCount)
|
||||
wcnt := 2 + int(srv.snapCount)
|
||||
if len(gaction) != wcnt {
|
||||
t.Fatalf("len(action) = %d, want %d", len(gaction), wcnt)
|
||||
}
|
||||
@ -943,15 +993,16 @@ func TestRecvSnapshot(t *testing.T) {
|
||||
cl := newCluster("abc")
|
||||
cl.SetStore(store.New())
|
||||
s := &EtcdServer{
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: p,
|
||||
node: n,
|
||||
Cluster: cl,
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: p,
|
||||
node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
Cluster: cl,
|
||||
}
|
||||
|
||||
s.start()
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Index: 1}}
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}}}
|
||||
// make goroutines move forward to receive snapshot
|
||||
testutil.ForceGosched()
|
||||
s.Stop()
|
||||
@ -960,7 +1011,7 @@ func TestRecvSnapshot(t *testing.T) {
|
||||
if g := st.Action(); !reflect.DeepEqual(g, wactions) {
|
||||
t.Errorf("store action = %v, want %v", g, wactions)
|
||||
}
|
||||
wactions = []action{action{name: "Save"}, action{name: "SaveSnap"}}
|
||||
wactions = []action{action{name: "SaveSnap"}, action{name: "Save"}}
|
||||
if g := p.Action(); !reflect.DeepEqual(g, wactions) {
|
||||
t.Errorf("storage action = %v, want %v", g, wactions)
|
||||
}
|
||||
@ -975,20 +1026,21 @@ func TestRecvSlowSnapshot(t *testing.T) {
|
||||
cl := newCluster("abc")
|
||||
cl.SetStore(store.New())
|
||||
s := &EtcdServer{
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
node: n,
|
||||
Cluster: cl,
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
Cluster: cl,
|
||||
}
|
||||
|
||||
s.start()
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Index: 1}}
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}}}
|
||||
// make goroutines move forward to receive snapshot
|
||||
testutil.ForceGosched()
|
||||
action := st.Action()
|
||||
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Index: 1}}
|
||||
n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}}}
|
||||
// make goroutines move forward to receive snapshot
|
||||
testutil.ForceGosched()
|
||||
s.Stop()
|
||||
@ -1005,18 +1057,20 @@ func TestApplySnapshotAndCommittedEntries(t *testing.T) {
|
||||
st := &storeRecorder{}
|
||||
cl := newCluster("abc")
|
||||
cl.SetStore(store.New())
|
||||
storage := raft.NewMemoryStorage()
|
||||
s := &EtcdServer{
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
node: n,
|
||||
Cluster: cl,
|
||||
store: st,
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
node: n,
|
||||
raftStorage: storage,
|
||||
Cluster: cl,
|
||||
}
|
||||
|
||||
s.start()
|
||||
req := &pb.Request{Method: "QGET"}
|
||||
n.readyc <- raft.Ready{
|
||||
Snapshot: raftpb.Snapshot{Index: 1},
|
||||
Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}},
|
||||
CommittedEntries: []raftpb.Entry{
|
||||
{Index: 2, Data: pbutil.MustMarshal(req)},
|
||||
},
|
||||
@ -1049,11 +1103,12 @@ func TestAddMember(t *testing.T) {
|
||||
cl := newTestCluster(nil)
|
||||
cl.SetStore(store.New())
|
||||
s := &EtcdServer{
|
||||
node: n,
|
||||
store: &storeRecorder{},
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Cluster: cl,
|
||||
node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
store: &storeRecorder{},
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Cluster: cl,
|
||||
}
|
||||
s.start()
|
||||
m := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"foo"}}}
|
||||
@ -1086,11 +1141,12 @@ func TestRemoveMember(t *testing.T) {
|
||||
cl.SetStore(store.New())
|
||||
cl.AddMember(&Member{ID: 1234})
|
||||
s := &EtcdServer{
|
||||
node: n,
|
||||
store: &storeRecorder{},
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Cluster: cl,
|
||||
node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
store: &storeRecorder{},
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Cluster: cl,
|
||||
}
|
||||
s.start()
|
||||
err := s.RemoveMember(context.TODO(), 1234)
|
||||
@ -1122,11 +1178,12 @@ func TestUpdateMember(t *testing.T) {
|
||||
cl.SetStore(store.New())
|
||||
cl.AddMember(&Member{ID: 1234})
|
||||
s := &EtcdServer{
|
||||
node: n,
|
||||
store: &storeRecorder{},
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Cluster: cl,
|
||||
node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
store: &storeRecorder{},
|
||||
sendhub: &nopSender{},
|
||||
storage: &storageRecorder{},
|
||||
Cluster: cl,
|
||||
}
|
||||
s.start()
|
||||
wm := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"http://127.0.0.1:1"}}}
|
||||
@ -1488,12 +1545,12 @@ func (n *readyNode) Propose(ctx context.Context, data []byte) error { return nil
|
||||
func (n *readyNode) ProposeConfChange(ctx context.Context, conf raftpb.ConfChange) error {
|
||||
return nil
|
||||
}
|
||||
func (n *readyNode) Step(ctx context.Context, msg raftpb.Message) error { return nil }
|
||||
func (n *readyNode) Ready() <-chan raft.Ready { return n.readyc }
|
||||
func (n *readyNode) Advance() {}
|
||||
func (n *readyNode) ApplyConfChange(conf raftpb.ConfChange) {}
|
||||
func (n *readyNode) Stop() {}
|
||||
func (n *readyNode) Compact(index uint64, nodes []uint64, d []byte) {}
|
||||
func (n *readyNode) Step(ctx context.Context, msg raftpb.Message) error { return nil }
|
||||
func (n *readyNode) Ready() <-chan raft.Ready { return n.readyc }
|
||||
func (n *readyNode) Advance() {}
|
||||
func (n *readyNode) ApplyConfChange(conf raftpb.ConfChange) *raftpb.ConfState { return nil }
|
||||
func (n *readyNode) Stop() {}
|
||||
func (n *readyNode) Compact(index uint64, nodes []uint64, d []byte) {}
|
||||
|
||||
type nodeRecorder struct {
|
||||
recorder
|
||||
@ -1519,8 +1576,9 @@ func (n *nodeRecorder) Step(ctx context.Context, msg raftpb.Message) error {
|
||||
}
|
||||
func (n *nodeRecorder) Ready() <-chan raft.Ready { return nil }
|
||||
func (n *nodeRecorder) Advance() {}
|
||||
func (n *nodeRecorder) ApplyConfChange(conf raftpb.ConfChange) {
|
||||
func (n *nodeRecorder) ApplyConfChange(conf raftpb.ConfChange) *raftpb.ConfState {
|
||||
n.record(action{name: "ApplyConfChange", params: []interface{}{conf}})
|
||||
return nil
|
||||
}
|
||||
func (n *nodeRecorder) Stop() {
|
||||
n.record(action{name: "Stop"})
|
||||
@ -1580,8 +1638,9 @@ func (n *nodeConfChangeCommitterRecorder) ProposeConfChange(ctx context.Context,
|
||||
func (n *nodeConfChangeCommitterRecorder) Ready() <-chan raft.Ready {
|
||||
return n.readyc
|
||||
}
|
||||
func (n *nodeConfChangeCommitterRecorder) ApplyConfChange(conf raftpb.ConfChange) {
|
||||
func (n *nodeConfChangeCommitterRecorder) ApplyConfChange(conf raftpb.ConfChange) *raftpb.ConfState {
|
||||
n.record(action{name: "ApplyConfChange:" + conf.Type.String()})
|
||||
return nil
|
||||
}
|
||||
|
||||
type waitWithResponse struct {
|
||||
|
@ -86,7 +86,7 @@ func Migrate4To5(dataDir string, name string) error {
|
||||
|
||||
// If we've got the most recent snapshot, we can use it's committed index. Still likely less than the current actual index, but worth it for the replay.
|
||||
if snap5 != nil {
|
||||
st5.Commit = snap5.Index
|
||||
st5.Commit = snap5.Metadata.Index
|
||||
}
|
||||
|
||||
ents5, err := Entries4To5(ents4)
|
||||
|
@ -178,10 +178,14 @@ func (s *Snapshot4) Snapshot5() *raftpb.Snapshot {
|
||||
}
|
||||
|
||||
snap5 := raftpb.Snapshot{
|
||||
Data: newState,
|
||||
Index: s.LastIndex,
|
||||
Term: s.LastTerm,
|
||||
Nodes: nodeList,
|
||||
Data: newState,
|
||||
Metadata: raftpb.SnapshotMetadata{
|
||||
Index: s.LastIndex,
|
||||
Term: s.LastTerm,
|
||||
ConfState: raftpb.ConfState{
|
||||
Nodes: nodeList,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
return &snap5
|
||||
|
@ -60,7 +60,7 @@ func mustTemp(pre, body string) string {
|
||||
func ltoa(l *raftLog) string {
|
||||
s := fmt.Sprintf("committed: %d\n", l.committed)
|
||||
s += fmt.Sprintf("applied: %d\n", l.applied)
|
||||
for i, e := range l.ents {
|
||||
for i, e := range l.allEntries() {
|
||||
s += fmt.Sprintf("#%d: %+v\n", i, e)
|
||||
}
|
||||
return s
|
||||
|
@ -26,7 +26,7 @@ func saveStateToDisk(st pb.HardState) {}
|
||||
func saveToDisk(ents []pb.Entry) {}
|
||||
|
||||
func Example_Node() {
|
||||
n := StartNode(0, nil, 0, 0)
|
||||
n := StartNode(0, nil, 0, 0, nil)
|
||||
|
||||
// stuff to n happens in other goroutines
|
||||
|
||||
|
253
raft/log.go
253
raft/log.go
@ -24,33 +24,56 @@ import (
|
||||
)
|
||||
|
||||
type raftLog struct {
|
||||
ents []pb.Entry
|
||||
unstable uint64
|
||||
// storage contains all stable entries since the last snapshot.
|
||||
storage Storage
|
||||
|
||||
// the incoming unstable snapshot, if any.
|
||||
unstableSnapshot *pb.Snapshot
|
||||
// unstableEnts contains all entries that have not yet been written
|
||||
// to storage.
|
||||
unstableEnts []pb.Entry
|
||||
// unstableEnts[i] has raft log position i+unstable. Note that
|
||||
// unstable may be less than the highest log position in storage;
|
||||
// this means that the next write to storage will truncate the log
|
||||
// before persisting unstableEnts.
|
||||
unstable uint64
|
||||
// committed is the highest log position that is known to be in
|
||||
// stable storage on a quorum of nodes.
|
||||
// Invariant: committed < unstable
|
||||
committed uint64
|
||||
applied uint64
|
||||
offset uint64
|
||||
snapshot pb.Snapshot
|
||||
// applied is the highest log position that the application has
|
||||
// been instructed to apply to its state machine.
|
||||
// Invariant: applied <= committed
|
||||
applied uint64
|
||||
}
|
||||
|
||||
func newLog() *raftLog {
|
||||
return &raftLog{
|
||||
ents: make([]pb.Entry, 1),
|
||||
unstable: 0,
|
||||
committed: 0,
|
||||
applied: 0,
|
||||
// newLog returns log using the given storage. It recovers the log to the state
|
||||
// that it just commits and applies the lastest snapshot.
|
||||
func newLog(storage Storage) *raftLog {
|
||||
if storage == nil {
|
||||
log.Panic("storage must not be nil")
|
||||
}
|
||||
}
|
||||
log := &raftLog{
|
||||
storage: storage,
|
||||
}
|
||||
firstIndex, err := storage.FirstIndex()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
lastIndex, err := storage.LastIndex()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
log.unstable = lastIndex + 1
|
||||
// Initialize our committed and applied pointers to the time of the last compaction.
|
||||
log.committed = firstIndex - 1
|
||||
log.applied = firstIndex - 1
|
||||
|
||||
func (l *raftLog) load(ents []pb.Entry) {
|
||||
if l.offset != ents[0].Index {
|
||||
panic("entries loaded don't match offset index")
|
||||
}
|
||||
l.ents = ents
|
||||
l.unstable = l.offset + uint64(len(ents))
|
||||
return log
|
||||
}
|
||||
|
||||
func (l *raftLog) String() string {
|
||||
return fmt.Sprintf("offset=%d committed=%d applied=%d len(ents)=%d", l.offset, l.committed, l.applied, len(l.ents))
|
||||
return fmt.Sprintf("unstable=%d committed=%d applied=%d len(unstableEntries)=%d", l.unstable, l.committed, l.applied, len(l.unstableEnts))
|
||||
}
|
||||
|
||||
// maybeAppend returns (0, false) if the entries cannot be appended. Otherwise,
|
||||
@ -63,7 +86,7 @@ func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry
|
||||
switch {
|
||||
case ci == 0:
|
||||
case ci <= l.committed:
|
||||
panic("conflict with committed entry")
|
||||
log.Panicf("entry %d conflict with committed entry [committed(%d)]", ci, l.committed)
|
||||
default:
|
||||
l.append(ci-1, ents[ci-from:]...)
|
||||
}
|
||||
@ -74,8 +97,18 @@ func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry
|
||||
}
|
||||
|
||||
func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 {
|
||||
l.ents = append(l.slice(l.offset, after+1), ents...)
|
||||
l.unstable = min(l.unstable, after+1)
|
||||
if after < l.committed {
|
||||
log.Panicf("after(%d) is out of range [committed(%d)]", after, l.committed)
|
||||
}
|
||||
if after < l.unstable {
|
||||
// The log is being truncated to before our current unstable
|
||||
// portion, so discard it and reset unstable.
|
||||
l.unstableEnts = nil
|
||||
l.unstable = after + 1
|
||||
}
|
||||
// Truncate any unstable entries that are being replaced, then
|
||||
// append the new ones.
|
||||
l.unstableEnts = append(l.unstableEnts[:after+1-l.unstable], ents...)
|
||||
return l.lastIndex()
|
||||
}
|
||||
|
||||
@ -93,39 +126,59 @@ func (l *raftLog) append(after uint64, ents ...pb.Entry) uint64 {
|
||||
func (l *raftLog) findConflict(from uint64, ents []pb.Entry) uint64 {
|
||||
// TODO(xiangli): validate the index of ents
|
||||
for i, ne := range ents {
|
||||
if oe := l.at(from + uint64(i)); oe == nil || oe.Term != ne.Term {
|
||||
if !l.matchTerm(from+uint64(i), ne.Term) {
|
||||
return from + uint64(i)
|
||||
}
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (l *raftLog) unstableEnts() []pb.Entry {
|
||||
ents := l.slice(l.unstable, l.lastIndex()+1)
|
||||
if ents == nil {
|
||||
func (l *raftLog) unstableEntries() []pb.Entry {
|
||||
if len(l.unstableEnts) == 0 {
|
||||
return nil
|
||||
}
|
||||
cpy := make([]pb.Entry, len(ents))
|
||||
copy(cpy, ents)
|
||||
return cpy
|
||||
// copy unstable entries to an empty slice
|
||||
return append([]pb.Entry{}, l.unstableEnts...)
|
||||
}
|
||||
|
||||
// nextEnts returns all the available entries for execution.
|
||||
// If applied is smaller than the index of snapshot, it returns all committed
|
||||
// entries after the index of snapshot.
|
||||
func (l *raftLog) nextEnts() (ents []pb.Entry) {
|
||||
off := max(l.applied, l.snapshot.Index)
|
||||
if l.committed > off {
|
||||
return l.slice(off+1, l.committed+1)
|
||||
off := max(l.applied+1, l.firstIndex())
|
||||
if l.committed+1 > off {
|
||||
return l.slice(off, l.committed+1)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *raftLog) snapshot() (pb.Snapshot, error) {
|
||||
if l.unstableSnapshot != nil {
|
||||
return *l.unstableSnapshot, nil
|
||||
}
|
||||
return l.storage.Snapshot()
|
||||
}
|
||||
|
||||
func (l *raftLog) firstIndex() uint64 {
|
||||
if l.unstableSnapshot != nil {
|
||||
return l.unstableSnapshot.Metadata.Index + 1
|
||||
}
|
||||
index, err := l.storage.FirstIndex()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
return index
|
||||
}
|
||||
|
||||
func (l *raftLog) lastIndex() uint64 {
|
||||
return l.unstable + uint64(len(l.unstableEnts)) - 1
|
||||
}
|
||||
|
||||
func (l *raftLog) commitTo(tocommit uint64) {
|
||||
// never decrease commit
|
||||
if l.committed < tocommit {
|
||||
if l.lastIndex() < tocommit {
|
||||
panic("committed out of range")
|
||||
log.Panicf("tocommit(%d) is out of range [lastIndex(%d)]", tocommit, l.lastIndex())
|
||||
}
|
||||
l.committed = tocommit
|
||||
}
|
||||
@ -136,36 +189,58 @@ func (l *raftLog) appliedTo(i uint64) {
|
||||
return
|
||||
}
|
||||
if l.committed < i || i < l.applied {
|
||||
log.Panicf("applied[%d] is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed)
|
||||
log.Panicf("applied(%d) is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed)
|
||||
}
|
||||
l.applied = i
|
||||
}
|
||||
|
||||
func (l *raftLog) stableTo(i uint64) {
|
||||
if i < l.unstable || i+1-l.unstable > uint64(len(l.unstableEnts)) {
|
||||
log.Panicf("stableTo(%d) is out of range [unstable(%d), len(unstableEnts)(%d)]",
|
||||
i, l.unstable, len(l.unstableEnts))
|
||||
}
|
||||
l.unstableEnts = l.unstableEnts[i+1-l.unstable:]
|
||||
l.unstable = i + 1
|
||||
}
|
||||
|
||||
func (l *raftLog) lastIndex() uint64 { return uint64(len(l.ents)) - 1 + l.offset }
|
||||
|
||||
func (l *raftLog) lastTerm() uint64 { return l.term(l.lastIndex()) }
|
||||
func (l *raftLog) lastTerm() uint64 {
|
||||
return l.term(l.lastIndex())
|
||||
}
|
||||
|
||||
func (l *raftLog) term(i uint64) uint64 {
|
||||
if e := l.at(i); e != nil {
|
||||
return e.Term
|
||||
switch {
|
||||
case i > l.lastIndex():
|
||||
return 0
|
||||
case i < l.unstable:
|
||||
if snap := l.unstableSnapshot; snap != nil {
|
||||
if i == snap.Metadata.Index {
|
||||
return snap.Metadata.Term
|
||||
}
|
||||
return 0
|
||||
}
|
||||
t, err := l.storage.Term(i)
|
||||
switch err {
|
||||
case nil:
|
||||
return t
|
||||
case ErrCompacted:
|
||||
return 0
|
||||
default:
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
default:
|
||||
return l.unstableEnts[i-l.unstable].Term
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (l *raftLog) entries(i uint64) []pb.Entry {
|
||||
// never send out the first entry
|
||||
// first entry is only used for matching
|
||||
// prevLogTerm
|
||||
if i == l.offset {
|
||||
panic("cannot return the first entry in log")
|
||||
}
|
||||
return l.slice(i, l.lastIndex()+1)
|
||||
}
|
||||
|
||||
// allEntries returns all entries in the log.
|
||||
func (l *raftLog) allEntries() []pb.Entry {
|
||||
return l.entries(l.firstIndex())
|
||||
}
|
||||
|
||||
// isUpToDate determines if the given (lastIndex,term) log is more up-to-date
|
||||
// by comparing the index and term of the last entries in the existing logs.
|
||||
// If the logs have last entries with different terms, then the log with the
|
||||
@ -177,10 +252,7 @@ func (l *raftLog) isUpToDate(lasti, term uint64) bool {
|
||||
}
|
||||
|
||||
func (l *raftLog) matchTerm(i, term uint64) bool {
|
||||
if e := l.at(i); e != nil {
|
||||
return e.Term == term
|
||||
}
|
||||
return false
|
||||
return l.term(i) == term
|
||||
}
|
||||
|
||||
func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
|
||||
@ -191,43 +263,11 @@ func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
|
||||
return false
|
||||
}
|
||||
|
||||
// compact compacts all log entries until i.
|
||||
// It removes the log entries before i, exclusive.
|
||||
// i must be not smaller than the index of the first entry
|
||||
// and not greater than the index of the last entry.
|
||||
// the number of entries after compaction will be returned.
|
||||
func (l *raftLog) compact(i uint64) uint64 {
|
||||
if l.isOutOfAppliedBounds(i) {
|
||||
panic(fmt.Sprintf("compact %d out of bounds [%d:%d]", i, l.offset, l.applied))
|
||||
}
|
||||
l.ents = l.slice(i, l.lastIndex()+1)
|
||||
l.unstable = max(i+1, l.unstable)
|
||||
l.offset = i
|
||||
return uint64(len(l.ents))
|
||||
}
|
||||
|
||||
func (l *raftLog) snap(d []byte, index, term uint64, nodes []uint64) {
|
||||
l.snapshot = pb.Snapshot{
|
||||
Data: d,
|
||||
Nodes: nodes,
|
||||
Index: index,
|
||||
Term: term,
|
||||
}
|
||||
}
|
||||
|
||||
func (l *raftLog) restore(s pb.Snapshot) {
|
||||
l.ents = []pb.Entry{{Index: s.Index, Term: s.Term}}
|
||||
l.unstable = s.Index
|
||||
l.committed = s.Index
|
||||
l.offset = s.Index
|
||||
l.snapshot = s
|
||||
}
|
||||
|
||||
func (l *raftLog) at(i uint64) *pb.Entry {
|
||||
if l.isOutOfBounds(i) {
|
||||
return nil
|
||||
}
|
||||
return &l.ents[i-l.offset]
|
||||
l.committed = s.Metadata.Index
|
||||
l.unstable = l.committed
|
||||
l.unstableEnts = []pb.Entry{{Index: s.Metadata.Index, Term: s.Metadata.Term}}
|
||||
l.unstableSnapshot = &s
|
||||
}
|
||||
|
||||
// slice returns a slice of log entries from lo through hi-1, inclusive.
|
||||
@ -238,33 +278,28 @@ func (l *raftLog) slice(lo uint64, hi uint64) []pb.Entry {
|
||||
if l.isOutOfBounds(lo) || l.isOutOfBounds(hi-1) {
|
||||
return nil
|
||||
}
|
||||
return l.ents[lo-l.offset : hi-l.offset]
|
||||
var ents []pb.Entry
|
||||
if lo < l.unstable {
|
||||
storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable))
|
||||
if err == ErrCompacted {
|
||||
// This should never fail because it has been checked before.
|
||||
log.Panicf("entries[%d:%d) from storage is out of bound", lo, min(hi, l.unstable))
|
||||
return nil
|
||||
} else if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
ents = append(ents, storedEnts...)
|
||||
}
|
||||
if hi > l.unstable {
|
||||
firstUnstable := max(lo, l.unstable)
|
||||
ents = append(ents, l.unstableEnts[firstUnstable-l.unstable:hi-l.unstable]...)
|
||||
}
|
||||
return ents
|
||||
}
|
||||
|
||||
func (l *raftLog) isOutOfBounds(i uint64) bool {
|
||||
if i < l.offset || i > l.lastIndex() {
|
||||
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 false
|
||||
}
|
||||
|
||||
func min(a, b uint64) uint64 {
|
||||
if a > b {
|
||||
return b
|
||||
}
|
||||
return a
|
||||
}
|
||||
|
||||
func max(a, b uint64) uint64 {
|
||||
if a > b {
|
||||
return a
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
278
raft/log_test.go
278
raft/log_test.go
@ -24,7 +24,7 @@ import (
|
||||
)
|
||||
|
||||
func TestFindConflict(t *testing.T) {
|
||||
previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}
|
||||
previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
|
||||
tests := []struct {
|
||||
from uint64
|
||||
ents []pb.Entry
|
||||
@ -34,22 +34,22 @@ func TestFindConflict(t *testing.T) {
|
||||
{1, []pb.Entry{}, 0},
|
||||
{3, []pb.Entry{}, 0},
|
||||
// no conflict
|
||||
{1, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}, 0},
|
||||
{2, []pb.Entry{{Term: 2}, {Term: 3}}, 0},
|
||||
{3, []pb.Entry{{Term: 3}}, 0},
|
||||
{1, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0},
|
||||
{2, []pb.Entry{{Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0},
|
||||
{3, []pb.Entry{{Index: 3, Term: 3}}, 0},
|
||||
// no conflict, but has new entries
|
||||
{1, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}, {Term: 4}, {Term: 4}}, 4},
|
||||
{2, []pb.Entry{{Term: 2}, {Term: 3}, {Term: 4}, {Term: 4}}, 4},
|
||||
{3, []pb.Entry{{Term: 3}, {Term: 4}, {Term: 4}}, 4},
|
||||
{4, []pb.Entry{{Term: 4}, {Term: 4}}, 4},
|
||||
{1, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
|
||||
{2, []pb.Entry{{Index: 2, Term: 2}, {Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
|
||||
{3, []pb.Entry{{Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
|
||||
{4, []pb.Entry{{Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
|
||||
// conflicts with existing entries
|
||||
{1, []pb.Entry{{Term: 4}, {Term: 4}}, 1},
|
||||
{2, []pb.Entry{{Term: 1}, {Term: 4}, {Term: 4}}, 2},
|
||||
{3, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 4}, {Term: 4}}, 3},
|
||||
{1, []pb.Entry{{Index: 1, Term: 4}, {Index: 2, Term: 4}}, 1},
|
||||
{2, []pb.Entry{{Index: 2, Term: 1}, {Index: 3, Term: 4}, {Index: 4, Term: 4}}, 2},
|
||||
{3, []pb.Entry{{Index: 3, Term: 1}, {Index: 4, Term: 2}, {Index: 5, Term: 4}, {Index: 6, Term: 4}}, 3},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
raftLog := newLog()
|
||||
raftLog := newLog(NewMemoryStorage())
|
||||
raftLog.append(raftLog.lastIndex(), previousEnts...)
|
||||
|
||||
gconflict := raftLog.findConflict(tt.from, tt.ents)
|
||||
@ -60,8 +60,8 @@ func TestFindConflict(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestIsUpToDate(t *testing.T) {
|
||||
previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}
|
||||
raftLog := newLog()
|
||||
previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
|
||||
raftLog := newLog(NewMemoryStorage())
|
||||
raftLog.append(raftLog.lastIndex(), previousEnts...)
|
||||
tests := []struct {
|
||||
lastIndex uint64
|
||||
@ -91,8 +91,7 @@ func TestIsUpToDate(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestAppend(t *testing.T) {
|
||||
previousEnts := []pb.Entry{{Term: 1}, {Term: 2}}
|
||||
previousUnstable := uint64(3)
|
||||
previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}
|
||||
tests := []struct {
|
||||
after uint64
|
||||
ents []pb.Entry
|
||||
@ -104,38 +103,39 @@ func TestAppend(t *testing.T) {
|
||||
2,
|
||||
[]pb.Entry{},
|
||||
2,
|
||||
[]pb.Entry{{Term: 1}, {Term: 2}},
|
||||
[]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}},
|
||||
3,
|
||||
},
|
||||
{
|
||||
2,
|
||||
[]pb.Entry{{Term: 2}},
|
||||
[]pb.Entry{{Index: 3, Term: 2}},
|
||||
3,
|
||||
[]pb.Entry{{Term: 1}, {Term: 2}, {Term: 2}},
|
||||
[]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 2}},
|
||||
3,
|
||||
},
|
||||
// conflicts with index 1
|
||||
{
|
||||
0,
|
||||
[]pb.Entry{{Term: 2}},
|
||||
[]pb.Entry{{Index: 1, Term: 2}},
|
||||
1,
|
||||
[]pb.Entry{{Term: 2}},
|
||||
[]pb.Entry{{Index: 1, Term: 2}},
|
||||
1,
|
||||
},
|
||||
// conflicts with index 2
|
||||
{
|
||||
1,
|
||||
[]pb.Entry{{Term: 3}, {Term: 3}},
|
||||
[]pb.Entry{{Index: 2, Term: 3}, {Index: 3, Term: 3}},
|
||||
3,
|
||||
[]pb.Entry{{Term: 1}, {Term: 3}, {Term: 3}},
|
||||
[]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 3}, {Index: 3, Term: 3}},
|
||||
2,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
raftLog := newLog()
|
||||
raftLog.append(raftLog.lastIndex(), previousEnts...)
|
||||
raftLog.unstable = previousUnstable
|
||||
storage := NewMemoryStorage()
|
||||
storage.Append(previousEnts)
|
||||
raftLog := newLog(storage)
|
||||
|
||||
index := raftLog.append(tt.after, tt.ents...)
|
||||
if index != tt.windex {
|
||||
t.Errorf("#%d: lastIndex = %d, want %d", i, index, tt.windex)
|
||||
@ -158,7 +158,7 @@ func TestAppend(t *testing.T) {
|
||||
// If the given (index, term) does not match with the existing log:
|
||||
// return false
|
||||
func TestLogMaybeAppend(t *testing.T) {
|
||||
previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}
|
||||
previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
|
||||
lastindex := uint64(3)
|
||||
lastterm := uint64(3)
|
||||
commit := uint64(1)
|
||||
@ -176,12 +176,12 @@ func TestLogMaybeAppend(t *testing.T) {
|
||||
}{
|
||||
// not match: term is different
|
||||
{
|
||||
lastterm - 1, lastindex, lastindex, []pb.Entry{{Term: 4}},
|
||||
lastterm - 1, lastindex, lastindex, []pb.Entry{{Index: lastindex + 1, Term: 4}},
|
||||
0, false, commit, false,
|
||||
},
|
||||
// not match: index out of bound
|
||||
{
|
||||
lastterm, lastindex + 1, lastindex, []pb.Entry{{Term: 4}},
|
||||
lastterm, lastindex + 1, lastindex, []pb.Entry{{Index: lastindex + 2, Term: 4}},
|
||||
0, false, commit, false,
|
||||
},
|
||||
// match with the last existing entry
|
||||
@ -206,42 +206,42 @@ func TestLogMaybeAppend(t *testing.T) {
|
||||
0, true, commit, false, // commit do not decrease
|
||||
},
|
||||
{
|
||||
lastterm, lastindex, lastindex, []pb.Entry{{Term: 4}},
|
||||
lastterm, lastindex, lastindex, []pb.Entry{{Index: lastindex + 1, Term: 4}},
|
||||
lastindex + 1, true, lastindex, false,
|
||||
},
|
||||
{
|
||||
lastterm, lastindex, lastindex + 1, []pb.Entry{{Term: 4}},
|
||||
lastterm, lastindex, lastindex + 1, []pb.Entry{{Index: lastindex + 1, Term: 4}},
|
||||
lastindex + 1, true, lastindex + 1, false,
|
||||
},
|
||||
{
|
||||
lastterm, lastindex, lastindex + 2, []pb.Entry{{Term: 4}},
|
||||
lastterm, lastindex, lastindex + 2, []pb.Entry{{Index: lastindex + 1, Term: 4}},
|
||||
lastindex + 1, true, lastindex + 1, false, // do not increase commit higher than lastnewi
|
||||
},
|
||||
{
|
||||
lastterm, lastindex, lastindex + 2, []pb.Entry{{Term: 4}, {Term: 4}},
|
||||
lastterm, lastindex, lastindex + 2, []pb.Entry{{Index: lastindex + 1, Term: 4}, {Index: lastindex + 2, Term: 4}},
|
||||
lastindex + 2, true, lastindex + 2, false,
|
||||
},
|
||||
// match with the the entry in the middle
|
||||
{
|
||||
lastterm - 1, lastindex - 1, lastindex, []pb.Entry{{Term: 4}},
|
||||
lastterm - 1, lastindex - 1, lastindex, []pb.Entry{{Index: lastindex, Term: 4}},
|
||||
lastindex, true, lastindex, false,
|
||||
},
|
||||
{
|
||||
lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Term: 4}},
|
||||
lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Index: lastindex - 1, Term: 4}},
|
||||
lastindex - 1, true, lastindex - 1, false,
|
||||
},
|
||||
{
|
||||
lastterm - 3, lastindex - 3, lastindex, []pb.Entry{{Term: 4}},
|
||||
lastterm - 3, lastindex - 3, lastindex, []pb.Entry{{Index: lastindex - 2, Term: 4}},
|
||||
lastindex - 2, true, lastindex - 2, true, // conflict with existing committed entry
|
||||
},
|
||||
{
|
||||
lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Term: 4}, {Term: 4}},
|
||||
lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Index: lastindex - 1, Term: 4}, {Index: lastindex, Term: 4}},
|
||||
lastindex, true, lastindex, false,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
raftLog := newLog()
|
||||
raftLog := newLog(NewMemoryStorage())
|
||||
raftLog.append(raftLog.lastIndex(), previousEnts...)
|
||||
raftLog.committed = commit
|
||||
func() {
|
||||
@ -278,44 +278,54 @@ func TestLogMaybeAppend(t *testing.T) {
|
||||
// a compaction.
|
||||
func TestCompactionSideEffects(t *testing.T) {
|
||||
var i uint64
|
||||
// Populate the log with 1000 entries; 750 in stable storage and 250 in unstable.
|
||||
lastIndex := uint64(1000)
|
||||
unstableIndex := uint64(750)
|
||||
lastTerm := lastIndex
|
||||
raftLog := newLog()
|
||||
|
||||
for i = 0; i < lastIndex; i++ {
|
||||
raftLog.append(uint64(i), pb.Entry{Term: uint64(i + 1), Index: uint64(i + 1)})
|
||||
storage := NewMemoryStorage()
|
||||
for i = 1; i <= unstableIndex; i++ {
|
||||
storage.Append([]pb.Entry{{Term: uint64(i), Index: uint64(i)}})
|
||||
}
|
||||
raftLog := newLog(storage)
|
||||
for i = unstableIndex; i < lastIndex; i++ {
|
||||
raftLog.append(i, pb.Entry{Term: uint64(i + 1), Index: uint64(i + 1)})
|
||||
}
|
||||
|
||||
ok := raftLog.maybeCommit(lastIndex, lastTerm)
|
||||
if !ok {
|
||||
t.Fatalf("maybeCommit returned false")
|
||||
}
|
||||
raftLog.maybeCommit(lastIndex, lastTerm)
|
||||
raftLog.appliedTo(raftLog.committed)
|
||||
|
||||
raftLog.compact(500)
|
||||
offset := uint64(500)
|
||||
storage.Compact(offset, nil, nil)
|
||||
|
||||
if raftLog.lastIndex() != lastIndex {
|
||||
t.Errorf("lastIndex = %d, want %d", raftLog.lastIndex(), lastIndex)
|
||||
}
|
||||
|
||||
for i := raftLog.offset; i <= raftLog.lastIndex(); i++ {
|
||||
for i := offset; i <= raftLog.lastIndex(); i++ {
|
||||
if raftLog.term(i) != i {
|
||||
t.Errorf("term(%d) = %d, want %d", i, raftLog.term(i), i)
|
||||
}
|
||||
}
|
||||
|
||||
for i := raftLog.offset; i <= raftLog.lastIndex(); i++ {
|
||||
for i := offset; i <= raftLog.lastIndex(); i++ {
|
||||
if !raftLog.matchTerm(i, i) {
|
||||
t.Errorf("matchTerm(%d) = false, want true", i)
|
||||
}
|
||||
}
|
||||
|
||||
unstableEnts := raftLog.unstableEnts()
|
||||
if g := len(unstableEnts); g != 500 {
|
||||
t.Errorf("len(unstableEntries) = %d, want = %d", g, 500)
|
||||
unstableEnts := raftLog.unstableEntries()
|
||||
if g := len(unstableEnts); g != 250 {
|
||||
t.Errorf("len(unstableEntries) = %d, want = %d", g, 250)
|
||||
}
|
||||
if unstableEnts[0].Index != 501 {
|
||||
t.Errorf("Index = %d, want = %d", unstableEnts[0].Index, 501)
|
||||
if unstableEnts[0].Index != 751 {
|
||||
t.Errorf("Index = %d, want = %d", unstableEnts[0].Index, 751)
|
||||
}
|
||||
|
||||
prev := raftLog.lastIndex()
|
||||
raftLog.append(raftLog.lastIndex(), pb.Entry{Term: raftLog.lastIndex() + 1})
|
||||
raftLog.append(raftLog.lastIndex(), pb.Entry{Index: raftLog.lastIndex() + 1, Term: raftLog.lastIndex() + 1})
|
||||
if raftLog.lastIndex() != prev+1 {
|
||||
t.Errorf("lastIndex = %d, want = %d", raftLog.lastIndex(), prev+1)
|
||||
}
|
||||
@ -327,9 +337,10 @@ func TestCompactionSideEffects(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestNextEnts(t *testing.T) {
|
||||
snap := pb.Snapshot{Term: 1, Index: 3}
|
||||
snap := pb.Snapshot{
|
||||
Metadata: pb.SnapshotMetadata{Term: 1, Index: 3},
|
||||
}
|
||||
ents := []pb.Entry{
|
||||
{Term: 1, Index: 3},
|
||||
{Term: 1, Index: 4},
|
||||
{Term: 1, Index: 5},
|
||||
{Term: 1, Index: 6},
|
||||
@ -338,15 +349,16 @@ func TestNextEnts(t *testing.T) {
|
||||
applied uint64
|
||||
wents []pb.Entry
|
||||
}{
|
||||
{0, ents[1:3]},
|
||||
{3, ents[1:3]},
|
||||
{4, ents[2:3]},
|
||||
{0, ents[:2]},
|
||||
{3, ents[:2]},
|
||||
{4, ents[1:2]},
|
||||
{5, nil},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
raftLog := newLog()
|
||||
raftLog.restore(snap)
|
||||
raftLog.load(ents)
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(snap)
|
||||
raftLog := newLog(storage)
|
||||
raftLog.append(snap.Metadata.Index, ents...)
|
||||
raftLog.maybeCommit(5, 1)
|
||||
raftLog.appliedTo(tt.applied)
|
||||
|
||||
@ -357,31 +369,37 @@ func TestNextEnts(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// TestUnstableEnts ensures unstableEntries returns the unstable part of the
|
||||
// entries correctly.
|
||||
func TestUnstableEnts(t *testing.T) {
|
||||
previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}
|
||||
tests := []struct {
|
||||
unstable uint64
|
||||
wents []pb.Entry
|
||||
wunstable uint64
|
||||
unstable uint64
|
||||
wents []pb.Entry
|
||||
}{
|
||||
{3, nil, 3},
|
||||
{1, previousEnts, 3},
|
||||
{0, append([]pb.Entry{{}}, previousEnts...), 3},
|
||||
{3, nil},
|
||||
{1, previousEnts},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
raftLog := newLog()
|
||||
raftLog.append(0, previousEnts...)
|
||||
raftLog.unstable = tt.unstable
|
||||
ents := raftLog.unstableEnts()
|
||||
// append stable entries to storage
|
||||
storage := NewMemoryStorage()
|
||||
storage.Append(previousEnts[:tt.unstable-1])
|
||||
|
||||
// append unstable entries to raftlog
|
||||
raftLog := newLog(storage)
|
||||
raftLog.append(raftLog.lastIndex(), previousEnts[tt.unstable-1:]...)
|
||||
|
||||
ents := raftLog.unstableEntries()
|
||||
if l := len(ents); l > 0 {
|
||||
raftLog.stableTo(ents[l-1].Index)
|
||||
}
|
||||
if !reflect.DeepEqual(ents, tt.wents) {
|
||||
t.Errorf("#%d: unstableEnts = %+v, want %+v", i, ents, tt.wents)
|
||||
}
|
||||
if g := raftLog.unstable; g != tt.wunstable {
|
||||
t.Errorf("#%d: unstable = %d, want %d", i, g, tt.wunstable)
|
||||
w := previousEnts[len(previousEnts)-1].Index + 1
|
||||
if g := raftLog.unstable; g != w {
|
||||
t.Errorf("#%d: unstable = %d, want %d", i, g, w)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -407,7 +425,7 @@ func TestCommitTo(t *testing.T) {
|
||||
}
|
||||
}
|
||||
}()
|
||||
raftLog := newLog()
|
||||
raftLog := newLog(NewMemoryStorage())
|
||||
raftLog.append(0, previousEnts...)
|
||||
raftLog.committed = commit
|
||||
raftLog.commitTo(tt.commit)
|
||||
@ -423,12 +441,12 @@ func TestStableTo(t *testing.T) {
|
||||
stable uint64
|
||||
wunstable uint64
|
||||
}{
|
||||
{0, 1},
|
||||
{1, 2},
|
||||
{2, 3},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
raftLog := newLog()
|
||||
raftLog := newLog(NewMemoryStorage())
|
||||
raftLog.append(0, []pb.Entry{{}, {}}...)
|
||||
raftLog.stableTo(tt.stable)
|
||||
if raftLog.unstable != tt.wunstable {
|
||||
t.Errorf("#%d: unstable = %d, want %d", i, raftLog.unstable, tt.wunstable)
|
||||
@ -436,21 +454,19 @@ func TestStableTo(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
//TestCompaction ensures that the number of log entreis is correct after compactions.
|
||||
//TestCompaction ensures that the number of log entries is correct after compactions.
|
||||
func TestCompaction(t *testing.T) {
|
||||
tests := []struct {
|
||||
applied uint64
|
||||
lastIndex uint64
|
||||
compact []uint64
|
||||
wleft []int
|
||||
wallow bool
|
||||
}{
|
||||
// out of upper bound
|
||||
{1000, 1000, []uint64{1001}, []int{-1}, false},
|
||||
{1000, 1000, []uint64{300, 500, 800, 900}, []int{701, 501, 201, 101}, true},
|
||||
{1000, []uint64{1001}, []int{-1}, false},
|
||||
{1000, []uint64{300, 500, 800, 900}, []int{700, 500, 200, 100}, true},
|
||||
// out of lower bound
|
||||
{1000, 1000, []uint64{300, 299}, []int{701, -1}, false},
|
||||
{0, 1000, []uint64{1}, []int{-1}, false},
|
||||
{1000, []uint64{300, 299}, []int{700, -1}, false},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
@ -458,22 +474,29 @@ func TestCompaction(t *testing.T) {
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
if tt.wallow == true {
|
||||
t.Errorf("%d: allow = %v, want %v", i, false, true)
|
||||
t.Errorf("%d: allow = %v, want %v: %v", i, false, true, r)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
raftLog := newLog()
|
||||
for i := uint64(0); i < tt.lastIndex; i++ {
|
||||
raftLog.append(uint64(i), pb.Entry{})
|
||||
storage := NewMemoryStorage()
|
||||
for i := uint64(1); i <= tt.lastIndex; i++ {
|
||||
storage.Append([]pb.Entry{{Index: i}})
|
||||
}
|
||||
raftLog.maybeCommit(tt.applied, 0)
|
||||
raftLog := newLog(storage)
|
||||
raftLog.maybeCommit(tt.lastIndex, 0)
|
||||
raftLog.appliedTo(raftLog.committed)
|
||||
|
||||
for j := 0; j < len(tt.compact); j++ {
|
||||
raftLog.compact(tt.compact[j])
|
||||
if len(raftLog.ents) != tt.wleft[j] {
|
||||
t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.ents), tt.wleft[j])
|
||||
err := storage.Compact(tt.compact[j], nil, nil)
|
||||
if err != nil {
|
||||
if tt.wallow {
|
||||
t.Errorf("#%d.%d allow = %t, want %t", i, j, false, tt.wallow)
|
||||
}
|
||||
continue
|
||||
}
|
||||
if len(raftLog.allEntries()) != tt.wleft[j] {
|
||||
t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.allEntries()), tt.wleft[j])
|
||||
}
|
||||
}
|
||||
}()
|
||||
@ -481,28 +504,24 @@ func TestCompaction(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestLogRestore(t *testing.T) {
|
||||
var i uint64
|
||||
raftLog := newLog()
|
||||
for i = 0; i < 100; i++ {
|
||||
raftLog.append(i, pb.Entry{Term: i + 1})
|
||||
}
|
||||
|
||||
index := uint64(1000)
|
||||
term := uint64(1000)
|
||||
raftLog.restore(pb.Snapshot{Index: index, Term: term})
|
||||
snap := pb.SnapshotMetadata{Index: index, Term: term}
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(pb.Snapshot{Metadata: snap})
|
||||
raftLog := newLog(storage)
|
||||
|
||||
// only has the guard entry
|
||||
if len(raftLog.ents) != 1 {
|
||||
t.Errorf("len = %d, want 0", len(raftLog.ents))
|
||||
if len(raftLog.allEntries()) != 0 {
|
||||
t.Errorf("len = %d, want 0", len(raftLog.allEntries()))
|
||||
}
|
||||
if raftLog.offset != index {
|
||||
t.Errorf("offset = %d, want %d", raftLog.offset, index)
|
||||
if raftLog.firstIndex() != index+1 {
|
||||
t.Errorf("firstIndex = %d, want %d", raftLog.firstIndex(), index+1)
|
||||
}
|
||||
if raftLog.committed != index {
|
||||
t.Errorf("comitted = %d, want %d", raftLog.committed, index)
|
||||
}
|
||||
if raftLog.unstable != index {
|
||||
t.Errorf("unstable = %d, want %d", raftLog.unstable, index)
|
||||
if raftLog.unstable != index+1 {
|
||||
t.Errorf("unstable = %d, want %d", raftLog.unstable, index+1)
|
||||
}
|
||||
if raftLog.term(index) != term {
|
||||
t.Errorf("term = %d, want %d", raftLog.term(index), term)
|
||||
@ -512,17 +531,20 @@ func TestLogRestore(t *testing.T) {
|
||||
func TestIsOutOfBounds(t *testing.T) {
|
||||
offset := uint64(100)
|
||||
num := uint64(100)
|
||||
l := &raftLog{offset: offset, ents: make([]pb.Entry, num)}
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
|
||||
l := newLog(storage)
|
||||
l.append(offset, make([]pb.Entry, num)...)
|
||||
|
||||
tests := []struct {
|
||||
index uint64
|
||||
w bool
|
||||
}{
|
||||
{offset - 1, true},
|
||||
{offset, false},
|
||||
{offset, true},
|
||||
{offset + num/2, false},
|
||||
{offset + num - 1, false},
|
||||
{offset + num, true},
|
||||
{offset + num, false},
|
||||
{offset + num + 1, true},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
@ -533,31 +555,33 @@ func TestIsOutOfBounds(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestAt(t *testing.T) {
|
||||
func TestTerm(t *testing.T) {
|
||||
var i uint64
|
||||
offset := uint64(100)
|
||||
num := uint64(100)
|
||||
|
||||
l := &raftLog{offset: offset}
|
||||
for i = 0; i < num; i++ {
|
||||
l.ents = append(l.ents, pb.Entry{Term: i})
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
|
||||
l := newLog(storage)
|
||||
for i = 1; i < num; i++ {
|
||||
l.append(offset+i-1, pb.Entry{Index: i, Term: i})
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
index uint64
|
||||
w *pb.Entry
|
||||
w uint64
|
||||
}{
|
||||
{offset - 1, nil},
|
||||
{offset, &pb.Entry{Term: 0}},
|
||||
{offset + num/2, &pb.Entry{Term: num / 2}},
|
||||
{offset + num - 1, &pb.Entry{Term: num - 1}},
|
||||
{offset + num, nil},
|
||||
{offset - 1, 0},
|
||||
{offset, 0},
|
||||
{offset + num/2, num / 2},
|
||||
{offset + num - 1, num - 1},
|
||||
{offset + num, 0},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
g := l.at(tt.index)
|
||||
if !reflect.DeepEqual(g, tt.w) {
|
||||
t.Errorf("#%d: at = %v, want %v", i, g, tt.w)
|
||||
term := l.term(tt.index)
|
||||
if !reflect.DeepEqual(term, tt.w) {
|
||||
t.Errorf("#%d: at = %d, want %d", i, term, tt.w)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -567,9 +591,11 @@ func TestSlice(t *testing.T) {
|
||||
offset := uint64(100)
|
||||
num := uint64(100)
|
||||
|
||||
l := &raftLog{offset: offset}
|
||||
for i = 0; i < num; i++ {
|
||||
l.ents = append(l.ents, pb.Entry{Term: i})
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
|
||||
l := newLog(storage)
|
||||
for i = 1; i < num; i++ {
|
||||
l.append(offset+i-1, pb.Entry{Index: i, Term: i})
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
@ -578,9 +604,9 @@ func TestSlice(t *testing.T) {
|
||||
w []pb.Entry
|
||||
}{
|
||||
{offset - 1, offset + 1, nil},
|
||||
{offset, offset + 1, []pb.Entry{{Term: 0}}},
|
||||
{offset + num/2, offset + num/2 + 1, []pb.Entry{{Term: num / 2}}},
|
||||
{offset + num - 1, offset + num, []pb.Entry{{Term: num - 1}}},
|
||||
{offset, offset + 1, nil},
|
||||
{offset + num/2, offset + num/2 + 1, []pb.Entry{{Index: num / 2, Term: num / 2}}},
|
||||
{offset + num - 1, offset + num, []pb.Entry{{Index: num - 1, Term: num - 1}}},
|
||||
{offset + num, offset + num + 1, nil},
|
||||
|
||||
{offset + num/2, offset + num/2, nil},
|
||||
|
130
raft/node.go
130
raft/node.go
@ -75,12 +75,6 @@ type Ready struct {
|
||||
Messages []pb.Message
|
||||
}
|
||||
|
||||
type compact struct {
|
||||
index uint64
|
||||
nodes []uint64
|
||||
data []byte
|
||||
}
|
||||
|
||||
func isHardStateEqual(a, b pb.HardState) bool {
|
||||
return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit
|
||||
}
|
||||
@ -92,12 +86,13 @@ func IsEmptyHardState(st pb.HardState) bool {
|
||||
|
||||
// IsEmptySnap returns true if the given Snapshot is empty.
|
||||
func IsEmptySnap(sp pb.Snapshot) bool {
|
||||
return sp.Index == 0
|
||||
return sp.Metadata.Index == 0
|
||||
}
|
||||
|
||||
func (rd Ready) containsUpdates() bool {
|
||||
return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) || !IsEmptySnap(rd.Snapshot) ||
|
||||
len(rd.Entries) > 0 || len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
|
||||
return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) ||
|
||||
!IsEmptySnap(rd.Snapshot) || len(rd.Entries) > 0 ||
|
||||
len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
|
||||
}
|
||||
|
||||
// Node represents a node in a raft cluster.
|
||||
@ -122,18 +117,12 @@ type Node interface {
|
||||
// It prepares the node to return the next available Ready.
|
||||
Advance()
|
||||
// ApplyConfChange applies config change to the local node.
|
||||
// TODO: reject existing node when add node
|
||||
// TODO: reject non-existant node when remove node
|
||||
ApplyConfChange(cc pb.ConfChange)
|
||||
// Returns an opaque ConfState protobuf which must be recorded
|
||||
// in snapshots. Will never return nil; it returns a pointer only
|
||||
// to match MemoryStorage.Compact.
|
||||
ApplyConfChange(cc pb.ConfChange) *pb.ConfState
|
||||
// Stop performs any necessary termination of the Node
|
||||
Stop()
|
||||
// Compact discards the entrire log up to the given index. It also
|
||||
// generates a raft snapshot containing the given nodes configuration
|
||||
// and the given snapshot data.
|
||||
// It is the caller's responsibility to ensure the given configuration
|
||||
// and snapshot data match the actual point-in-time configuration and snapshot
|
||||
// at the given index.
|
||||
Compact(index uint64, nodes []uint64, d []byte)
|
||||
}
|
||||
|
||||
type Peer struct {
|
||||
@ -144,9 +133,9 @@ type Peer struct {
|
||||
// StartNode returns a new Node given a unique raft id, a list of raft peers, and
|
||||
// the election and heartbeat timeouts in units of ticks.
|
||||
// It appends a ConfChangeAddNode entry for each given peer to the initial log.
|
||||
func StartNode(id uint64, peers []Peer, election, heartbeat int) Node {
|
||||
func StartNode(id uint64, peers []Peer, election, heartbeat int, storage Storage) Node {
|
||||
n := newNode()
|
||||
r := newRaft(id, nil, election, heartbeat)
|
||||
r := newRaft(id, nil, election, heartbeat, storage)
|
||||
|
||||
for _, peer := range peers {
|
||||
cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
|
||||
@ -157,56 +146,49 @@ func StartNode(id uint64, peers []Peer, election, heartbeat int) Node {
|
||||
e := pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: r.raftLog.lastIndex() + 1, Data: d}
|
||||
r.raftLog.append(r.raftLog.lastIndex(), e)
|
||||
}
|
||||
// Mark these initial entries as committed.
|
||||
// TODO(bdarnell): These entries are still unstable; do we need to preserve
|
||||
// the invariant that committed < unstable?
|
||||
r.raftLog.committed = r.raftLog.lastIndex()
|
||||
|
||||
go n.run(r)
|
||||
return &n
|
||||
}
|
||||
|
||||
// RestartNode is identical to StartNode but takes an initial State and a slice
|
||||
// of entries. Generally this is used when restarting from a stable storage
|
||||
// log.
|
||||
func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, ents []pb.Entry) Node {
|
||||
// RestartNode is identical to StartNode but does not take a list of peers.
|
||||
// The current membership of the cluster will be restored from the Storage.
|
||||
func RestartNode(id uint64, election, heartbeat int, storage Storage) Node {
|
||||
n := newNode()
|
||||
r := newRaft(id, nil, election, heartbeat)
|
||||
if snapshot != nil {
|
||||
r.restore(*snapshot)
|
||||
r.raftLog.appliedTo(snapshot.Index)
|
||||
}
|
||||
if !isHardStateEqual(st, emptyState) {
|
||||
r.loadState(st)
|
||||
}
|
||||
if len(ents) != 0 {
|
||||
r.loadEnts(ents)
|
||||
}
|
||||
r := newRaft(id, nil, election, heartbeat, storage)
|
||||
|
||||
go n.run(r)
|
||||
return &n
|
||||
}
|
||||
|
||||
// node is the canonical implementation of the Node interface
|
||||
type node struct {
|
||||
propc chan pb.Message
|
||||
recvc chan pb.Message
|
||||
compactc chan compact
|
||||
confc chan pb.ConfChange
|
||||
readyc chan Ready
|
||||
advancec chan struct{}
|
||||
tickc chan struct{}
|
||||
done chan struct{}
|
||||
stop chan struct{}
|
||||
propc chan pb.Message
|
||||
recvc chan pb.Message
|
||||
confc chan pb.ConfChange
|
||||
confstatec chan pb.ConfState
|
||||
readyc chan Ready
|
||||
advancec chan struct{}
|
||||
tickc chan struct{}
|
||||
done chan struct{}
|
||||
stop chan struct{}
|
||||
}
|
||||
|
||||
func newNode() node {
|
||||
return node{
|
||||
propc: make(chan pb.Message),
|
||||
recvc: make(chan pb.Message),
|
||||
compactc: make(chan compact),
|
||||
confc: make(chan pb.ConfChange),
|
||||
readyc: make(chan Ready),
|
||||
advancec: make(chan struct{}),
|
||||
tickc: make(chan struct{}),
|
||||
done: make(chan struct{}),
|
||||
stop: make(chan struct{}),
|
||||
propc: make(chan pb.Message),
|
||||
recvc: make(chan pb.Message),
|
||||
confc: make(chan pb.ConfChange),
|
||||
confstatec: make(chan pb.ConfState),
|
||||
readyc: make(chan Ready),
|
||||
advancec: make(chan struct{}),
|
||||
tickc: make(chan struct{}),
|
||||
done: make(chan struct{}),
|
||||
stop: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
@ -228,18 +210,18 @@ func (n *node) run(r *raft) {
|
||||
var advancec chan struct{}
|
||||
var prevLastUnstablei uint64
|
||||
var havePrevLastUnstablei bool
|
||||
var prevSnapi uint64
|
||||
var rd Ready
|
||||
|
||||
lead := None
|
||||
prevSoftSt := r.softState()
|
||||
prevHardSt := r.HardState
|
||||
prevSnapi := r.raftLog.snapshot.Index
|
||||
|
||||
for {
|
||||
if advancec != nil {
|
||||
readyc = nil
|
||||
} else {
|
||||
rd = newReady(r, prevSoftSt, prevHardSt, prevSnapi)
|
||||
rd = newReady(r, prevSoftSt, prevHardSt)
|
||||
if rd.containsUpdates() {
|
||||
readyc = n.readyc
|
||||
} else {
|
||||
@ -271,11 +253,13 @@ func (n *node) run(r *raft) {
|
||||
r.Step(m)
|
||||
case m := <-n.recvc:
|
||||
r.Step(m) // raft never returns an error
|
||||
case c := <-n.compactc:
|
||||
r.compact(c.index, c.nodes, c.data)
|
||||
case cc := <-n.confc:
|
||||
if cc.NodeID == None {
|
||||
r.resetPendingConf()
|
||||
select {
|
||||
case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
|
||||
case <-n.done:
|
||||
}
|
||||
break
|
||||
}
|
||||
switch cc.Type {
|
||||
@ -288,6 +272,10 @@ func (n *node) run(r *raft) {
|
||||
default:
|
||||
panic("unexpected conf type")
|
||||
}
|
||||
select {
|
||||
case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
|
||||
case <-n.done:
|
||||
}
|
||||
case <-n.tickc:
|
||||
r.tick()
|
||||
case readyc <- rd:
|
||||
@ -302,11 +290,11 @@ func (n *node) run(r *raft) {
|
||||
prevHardSt = rd.HardState
|
||||
}
|
||||
if !IsEmptySnap(rd.Snapshot) {
|
||||
prevSnapi = rd.Snapshot.Index
|
||||
if prevSnapi > prevLastUnstablei {
|
||||
prevLastUnstablei = prevSnapi
|
||||
if rd.Snapshot.Metadata.Index > prevLastUnstablei {
|
||||
prevLastUnstablei = rd.Snapshot.Metadata.Index
|
||||
havePrevLastUnstablei = true
|
||||
}
|
||||
prevSnapi = rd.Snapshot.Metadata.Index
|
||||
}
|
||||
r.msgs = nil
|
||||
advancec = n.advancec
|
||||
@ -318,6 +306,9 @@ func (n *node) run(r *raft) {
|
||||
r.raftLog.stableTo(prevLastUnstablei)
|
||||
havePrevLastUnstablei = false
|
||||
}
|
||||
if r.raftLog.unstableSnapshot != nil && r.raftLog.unstableSnapshot.Metadata.Index == prevSnapi {
|
||||
r.raftLog.unstableSnapshot = nil
|
||||
}
|
||||
advancec = nil
|
||||
case <-n.stop:
|
||||
close(n.done)
|
||||
@ -389,23 +380,22 @@ func (n *node) Advance() {
|
||||
}
|
||||
}
|
||||
|
||||
func (n *node) ApplyConfChange(cc pb.ConfChange) {
|
||||
func (n *node) ApplyConfChange(cc pb.ConfChange) *pb.ConfState {
|
||||
var cs pb.ConfState
|
||||
select {
|
||||
case n.confc <- cc:
|
||||
case <-n.done:
|
||||
}
|
||||
}
|
||||
|
||||
func (n *node) Compact(index uint64, nodes []uint64, d []byte) {
|
||||
select {
|
||||
case n.compactc <- compact{index, nodes, d}:
|
||||
case cs = <-n.confstatec:
|
||||
case <-n.done:
|
||||
}
|
||||
return &cs
|
||||
}
|
||||
|
||||
func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState, prevSnapi uint64) Ready {
|
||||
func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready {
|
||||
rd := Ready{
|
||||
Entries: r.raftLog.unstableEnts(),
|
||||
Entries: r.raftLog.unstableEntries(),
|
||||
CommittedEntries: r.raftLog.nextEnts(),
|
||||
Messages: r.msgs,
|
||||
}
|
||||
@ -415,8 +405,8 @@ func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState, prevSnapi
|
||||
if !isHardStateEqual(r.HardState, prevHardSt) {
|
||||
rd.HardState = r.HardState
|
||||
}
|
||||
if prevSnapi != r.raftLog.snapshot.Index {
|
||||
rd.Snapshot = r.raftLog.snapshot
|
||||
if r.raftLog.unstableSnapshot != nil {
|
||||
rd.Snapshot = *r.raftLog.unstableSnapshot
|
||||
}
|
||||
return rd
|
||||
}
|
||||
|
@ -27,7 +27,7 @@ func BenchmarkOneNode(b *testing.B) {
|
||||
defer cancel()
|
||||
|
||||
n := newNode()
|
||||
r := newRaft(1, []uint64{1}, 10, 1)
|
||||
r := newRaft(1, []uint64{1}, 10, 1, nil)
|
||||
go n.run(r)
|
||||
|
||||
defer n.Stop()
|
||||
|
@ -115,11 +115,13 @@ func TestNodePropose(t *testing.T) {
|
||||
}
|
||||
|
||||
n := newNode()
|
||||
r := newRaft(1, []uint64{1}, 10, 1)
|
||||
s := NewMemoryStorage()
|
||||
r := newRaft(1, []uint64{1}, 10, 1, s)
|
||||
go n.run(r)
|
||||
n.Campaign(context.TODO())
|
||||
for {
|
||||
rd := <-n.Ready()
|
||||
s.Append(rd.Entries)
|
||||
// change the step function to appendStep until this raft becomes leader
|
||||
if rd.SoftState.Lead == r.id {
|
||||
r.step = appendStep
|
||||
@ -151,11 +153,13 @@ func TestNodeProposeConfig(t *testing.T) {
|
||||
}
|
||||
|
||||
n := newNode()
|
||||
r := newRaft(1, []uint64{1}, 10, 1)
|
||||
s := NewMemoryStorage()
|
||||
r := newRaft(1, []uint64{1}, 10, 1, s)
|
||||
go n.run(r)
|
||||
n.Campaign(context.TODO())
|
||||
for {
|
||||
rd := <-n.Ready()
|
||||
s.Append(rd.Entries)
|
||||
// change the step function to appendStep until this raft becomes leader
|
||||
if rd.SoftState.Lead == r.id {
|
||||
r.step = appendStep
|
||||
@ -188,7 +192,7 @@ func TestNodeProposeConfig(t *testing.T) {
|
||||
// who is the current leader.
|
||||
func TestBlockProposal(t *testing.T) {
|
||||
n := newNode()
|
||||
r := newRaft(1, []uint64{1}, 10, 1)
|
||||
r := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
||||
go n.run(r)
|
||||
defer n.Stop()
|
||||
|
||||
@ -220,7 +224,8 @@ func TestBlockProposal(t *testing.T) {
|
||||
// elapsed of the underlying raft state machine.
|
||||
func TestNodeTick(t *testing.T) {
|
||||
n := newNode()
|
||||
r := newRaft(1, []uint64{1}, 10, 1)
|
||||
s := NewMemoryStorage()
|
||||
r := newRaft(1, []uint64{1}, 10, 1, s)
|
||||
go n.run(r)
|
||||
elapsed := r.elapsed
|
||||
n.Tick()
|
||||
@ -234,7 +239,8 @@ func TestNodeTick(t *testing.T) {
|
||||
// processing, and that it is idempotent
|
||||
func TestNodeStop(t *testing.T) {
|
||||
n := newNode()
|
||||
r := newRaft(1, []uint64{1}, 10, 1)
|
||||
s := NewMemoryStorage()
|
||||
r := newRaft(1, []uint64{1}, 10, 1, s)
|
||||
donec := make(chan struct{})
|
||||
|
||||
go func() {
|
||||
@ -275,7 +281,7 @@ func TestReadyContainUpdates(t *testing.T) {
|
||||
{Ready{Entries: make([]raftpb.Entry, 1, 1)}, true},
|
||||
{Ready{CommittedEntries: make([]raftpb.Entry, 1, 1)}, true},
|
||||
{Ready{Messages: make([]raftpb.Message, 1, 1)}, true},
|
||||
{Ready{Snapshot: raftpb.Snapshot{Index: 1}}, true},
|
||||
{Ready{Snapshot: raftpb.Snapshot{Metadata: raftpb.SnapshotMetadata{Index: 1}}}, true},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
@ -302,7 +308,6 @@ func TestNodeStart(t *testing.T) {
|
||||
SoftState: &SoftState{Lead: 1, Nodes: []uint64{1}, RaftState: StateLeader},
|
||||
HardState: raftpb.HardState{Term: 1, Commit: 2},
|
||||
Entries: []raftpb.Entry{
|
||||
{},
|
||||
{Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata},
|
||||
{Term: 1, Index: 2},
|
||||
},
|
||||
@ -317,12 +322,15 @@ func TestNodeStart(t *testing.T) {
|
||||
CommittedEntries: []raftpb.Entry{{Term: 1, Index: 3, Data: []byte("foo")}},
|
||||
},
|
||||
}
|
||||
n := StartNode(1, []Peer{{ID: 1}}, 10, 1)
|
||||
storage := NewMemoryStorage()
|
||||
n := StartNode(1, []Peer{{ID: 1}}, 10, 1, storage)
|
||||
n.ApplyConfChange(cc)
|
||||
n.Campaign(ctx)
|
||||
if g := <-n.Ready(); !reflect.DeepEqual(g, wants[0]) {
|
||||
t.Errorf("#%d: g = %+v,\n w %+v", 1, g, wants[0])
|
||||
g := <-n.Ready()
|
||||
if !reflect.DeepEqual(g, wants[0]) {
|
||||
t.Fatalf("#%d: g = %+v,\n w %+v", 1, g, wants[0])
|
||||
} else {
|
||||
storage.Append(g.Entries)
|
||||
n.Advance()
|
||||
}
|
||||
|
||||
@ -330,6 +338,7 @@ func TestNodeStart(t *testing.T) {
|
||||
if g := <-n.Ready(); !reflect.DeepEqual(g, wants[1]) {
|
||||
t.Errorf("#%d: g = %+v,\n w %+v", 2, g, wants[1])
|
||||
} else {
|
||||
storage.Append(g.Entries)
|
||||
n.Advance()
|
||||
}
|
||||
|
||||
@ -342,7 +351,6 @@ func TestNodeStart(t *testing.T) {
|
||||
|
||||
func TestNodeRestart(t *testing.T) {
|
||||
entries := []raftpb.Entry{
|
||||
{},
|
||||
{Term: 1, Index: 1},
|
||||
{Term: 1, Index: 2, Data: []byte("foo")},
|
||||
}
|
||||
@ -351,15 +359,17 @@ func TestNodeRestart(t *testing.T) {
|
||||
want := Ready{
|
||||
HardState: emptyState,
|
||||
// commit upto index commit index in st
|
||||
CommittedEntries: entries[1 : st.Commit+1],
|
||||
CommittedEntries: entries[:st.Commit],
|
||||
}
|
||||
|
||||
n := RestartNode(1, 10, 1, nil, st, entries)
|
||||
storage := NewMemoryStorage()
|
||||
storage.SetHardState(st)
|
||||
storage.Append(entries)
|
||||
n := RestartNode(1, 10, 1, storage)
|
||||
if g := <-n.Ready(); !reflect.DeepEqual(g, want) {
|
||||
t.Errorf("g = %+v,\n w %+v", g, want)
|
||||
} else {
|
||||
n.Advance()
|
||||
}
|
||||
n.Advance()
|
||||
|
||||
select {
|
||||
case rd := <-n.Ready():
|
||||
@ -369,14 +379,14 @@ func TestNodeRestart(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestNodeRestartFromSnapshot(t *testing.T) {
|
||||
snap := &raftpb.Snapshot{
|
||||
Data: []byte("some data"),
|
||||
Nodes: []uint64{1, 2},
|
||||
Index: 2,
|
||||
Term: 1,
|
||||
snap := raftpb.Snapshot{
|
||||
Metadata: raftpb.SnapshotMetadata{
|
||||
ConfState: raftpb.ConfState{Nodes: []uint64{1, 2}},
|
||||
Index: 2,
|
||||
Term: 1,
|
||||
},
|
||||
}
|
||||
entries := []raftpb.Entry{
|
||||
{Term: 1, Index: 2},
|
||||
{Term: 1, Index: 3, Data: []byte("foo")},
|
||||
}
|
||||
st := raftpb.HardState{Term: 1, Commit: 3}
|
||||
@ -384,10 +394,14 @@ func TestNodeRestartFromSnapshot(t *testing.T) {
|
||||
want := Ready{
|
||||
HardState: emptyState,
|
||||
// commit upto index commit index in st
|
||||
CommittedEntries: entries[1:],
|
||||
CommittedEntries: entries,
|
||||
}
|
||||
|
||||
n := RestartNode(1, 10, 1, snap, st, entries)
|
||||
s := NewMemoryStorage()
|
||||
s.SetHardState(st)
|
||||
s.ApplySnapshot(snap)
|
||||
s.Append(entries)
|
||||
n := RestartNode(1, 10, 1, s)
|
||||
if g := <-n.Ready(); !reflect.DeepEqual(g, want) {
|
||||
t.Errorf("g = %+v,\n w %+v", g, want)
|
||||
} else {
|
||||
@ -401,72 +415,23 @@ func TestNodeRestartFromSnapshot(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// TestCompacts ensures Node.Compact creates a correct raft snapshot and compacts
|
||||
// the raft log (call raft.compact)
|
||||
func TestNodeCompact(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
n := newNode()
|
||||
r := newRaft(1, []uint64{1}, 10, 1)
|
||||
go n.run(r)
|
||||
|
||||
n.Campaign(ctx)
|
||||
n.Propose(ctx, []byte("foo"))
|
||||
|
||||
w := raftpb.Snapshot{
|
||||
Term: 1,
|
||||
Index: 2, // one nop + one proposal
|
||||
Data: []byte("a snapshot"),
|
||||
Nodes: []uint64{1},
|
||||
}
|
||||
|
||||
testutil.ForceGosched()
|
||||
select {
|
||||
case <-n.Ready():
|
||||
n.Advance()
|
||||
default:
|
||||
t.Fatalf("unexpected proposal failure: unable to commit entry")
|
||||
}
|
||||
|
||||
n.Compact(w.Index, w.Nodes, w.Data)
|
||||
testutil.ForceGosched()
|
||||
select {
|
||||
case rd := <-n.Ready():
|
||||
if !reflect.DeepEqual(rd.Snapshot, w) {
|
||||
t.Errorf("snap = %+v, want %+v", rd.Snapshot, w)
|
||||
}
|
||||
n.Advance()
|
||||
default:
|
||||
t.Fatalf("unexpected compact failure: unable to create a snapshot")
|
||||
}
|
||||
testutil.ForceGosched()
|
||||
// TODO: this test the run updates the snapi correctly... should be tested
|
||||
// separately with other kinds of updates
|
||||
select {
|
||||
case <-n.Ready():
|
||||
t.Fatalf("unexpected more ready")
|
||||
default:
|
||||
}
|
||||
n.Stop()
|
||||
|
||||
if r.raftLog.offset != w.Index {
|
||||
t.Errorf("log.offset = %d, want %d", r.raftLog.offset, w.Index)
|
||||
}
|
||||
}
|
||||
|
||||
func TestNodeAdvance(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
n := StartNode(1, []Peer{{ID: 1}}, 10, 1)
|
||||
storage := NewMemoryStorage()
|
||||
n := StartNode(1, []Peer{{ID: 1}}, 10, 1, storage)
|
||||
n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1})
|
||||
n.Campaign(ctx)
|
||||
<-n.Ready()
|
||||
n.Propose(ctx, []byte("foo"))
|
||||
var rd Ready
|
||||
select {
|
||||
case rd := <-n.Ready():
|
||||
case rd = <-n.Ready():
|
||||
t.Fatalf("unexpected Ready before Advance: %+v", rd)
|
||||
case <-time.After(time.Millisecond):
|
||||
}
|
||||
storage.Append(rd.Entries)
|
||||
n.Advance()
|
||||
select {
|
||||
case <-n.Ready():
|
||||
|
75
raft/raft.go
75
raft/raft.go
@ -100,13 +100,6 @@ func (pr *progress) String() string {
|
||||
return fmt.Sprintf("n=%d m=%d", pr.next, pr.match)
|
||||
}
|
||||
|
||||
// uint64Slice implements sort interface
|
||||
type uint64Slice []uint64
|
||||
|
||||
func (p uint64Slice) Len() int { return len(p) }
|
||||
func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] }
|
||||
func (p uint64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
|
||||
|
||||
type raft struct {
|
||||
pb.HardState
|
||||
|
||||
@ -137,21 +130,38 @@ type raft struct {
|
||||
step stepFunc
|
||||
}
|
||||
|
||||
func newRaft(id uint64, peers []uint64, election, heartbeat int) *raft {
|
||||
func newRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage) *raft {
|
||||
if id == None {
|
||||
panic("cannot use none id")
|
||||
}
|
||||
log := newLog(storage)
|
||||
hs, cs, err := storage.InitialState()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
if len(cs.Nodes) > 0 {
|
||||
if len(peers) > 0 {
|
||||
// TODO(bdarnell): the peers argument is always nil except in
|
||||
// tests; the argument should be removed and these tests should be
|
||||
// updated to specify their nodes through a snapshot.
|
||||
panic("cannot specify both newRaft(peers) and ConfState.Nodes)")
|
||||
}
|
||||
peers = cs.Nodes
|
||||
}
|
||||
r := &raft{
|
||||
id: id,
|
||||
lead: None,
|
||||
raftLog: newLog(),
|
||||
raftLog: log,
|
||||
prs: make(map[uint64]*progress),
|
||||
electionTimeout: election,
|
||||
heartbeatTimeout: heartbeat,
|
||||
}
|
||||
r.rand = rand.New(rand.NewSource(int64(id)))
|
||||
for _, p := range peers {
|
||||
r.prs[p] = &progress{}
|
||||
r.prs[p] = &progress{next: 1}
|
||||
}
|
||||
if !isHardStateEqual(hs, emptyState) {
|
||||
r.loadState(hs)
|
||||
}
|
||||
r.becomeFollower(0, None)
|
||||
return r
|
||||
@ -207,12 +217,19 @@ func (r *raft) sendAppend(to uint64) {
|
||||
pr := r.prs[to]
|
||||
m := pb.Message{}
|
||||
m.To = to
|
||||
m.Index = pr.next - 1
|
||||
if r.needSnapshot(m.Index) {
|
||||
if r.needSnapshot(pr.next) {
|
||||
m.Type = pb.MsgSnap
|
||||
m.Snapshot = r.raftLog.snapshot
|
||||
snapshot, err := r.raftLog.snapshot()
|
||||
if err != nil {
|
||||
panic(err) // TODO(bdarnell)
|
||||
}
|
||||
if IsEmptySnap(snapshot) {
|
||||
panic("need non-empty snapshot")
|
||||
}
|
||||
m.Snapshot = snapshot
|
||||
} else {
|
||||
m.Type = pb.MsgApp
|
||||
m.Index = pr.next - 1
|
||||
m.LogTerm = r.raftLog.term(pr.next - 1)
|
||||
m.Entries = r.raftLog.entries(pr.next)
|
||||
m.Commit = r.raftLog.committed
|
||||
@ -424,9 +441,7 @@ func (r *raft) handleSnapshot(m pb.Message) {
|
||||
}
|
||||
}
|
||||
|
||||
func (r *raft) resetPendingConf() {
|
||||
r.pendingConf = false
|
||||
}
|
||||
func (r *raft) resetPendingConf() { r.pendingConf = false }
|
||||
|
||||
func (r *raft) addNode(id uint64) {
|
||||
r.setProgress(id, 0, r.raftLog.lastIndex()+1)
|
||||
@ -530,28 +545,16 @@ func stepFollower(r *raft, m pb.Message) {
|
||||
}
|
||||
}
|
||||
|
||||
func (r *raft) compact(index uint64, nodes []uint64, d []byte) {
|
||||
if index > r.raftLog.applied {
|
||||
panic(fmt.Sprintf("raft: compact index (%d) exceeds applied index (%d)", index, r.raftLog.applied))
|
||||
}
|
||||
if index < r.raftLog.offset {
|
||||
//TODO: return an error?
|
||||
return
|
||||
}
|
||||
r.raftLog.snap(d, index, r.raftLog.term(index), nodes)
|
||||
r.raftLog.compact(index)
|
||||
}
|
||||
|
||||
// restore recovers the statemachine from a snapshot. It restores the log and the
|
||||
// configuration of statemachine.
|
||||
func (r *raft) restore(s pb.Snapshot) bool {
|
||||
if s.Index <= r.raftLog.committed {
|
||||
if s.Metadata.Index <= r.raftLog.committed {
|
||||
return false
|
||||
}
|
||||
|
||||
r.raftLog.restore(s)
|
||||
r.prs = make(map[uint64]*progress)
|
||||
for _, n := range s.Nodes {
|
||||
for _, n := range s.Metadata.ConfState.Nodes {
|
||||
if n == r.id {
|
||||
r.setProgress(n, r.raftLog.lastIndex(), r.raftLog.lastIndex()+1)
|
||||
} else {
|
||||
@ -562,13 +565,7 @@ func (r *raft) restore(s pb.Snapshot) bool {
|
||||
}
|
||||
|
||||
func (r *raft) needSnapshot(i uint64) bool {
|
||||
if i < r.raftLog.offset {
|
||||
if r.raftLog.snapshot.Term == 0 {
|
||||
panic("need non-empty snapshot")
|
||||
}
|
||||
return true
|
||||
}
|
||||
return false
|
||||
return i < r.raftLog.firstIndex()
|
||||
}
|
||||
|
||||
func (r *raft) nodes() []uint64 {
|
||||
@ -595,10 +592,6 @@ func (r *raft) promotable() bool {
|
||||
return ok
|
||||
}
|
||||
|
||||
func (r *raft) loadEnts(ents []pb.Entry) {
|
||||
r.raftLog.load(ents)
|
||||
}
|
||||
|
||||
func (r *raft) loadState(state pb.HardState) {
|
||||
r.raftLog.committed = state.Commit
|
||||
r.Term = state.Term
|
||||
|
@ -52,7 +52,7 @@ func TestLeaderUpdateTermFromMessage(t *testing.T) {
|
||||
// it immediately reverts to follower state.
|
||||
// Reference: section 5.1
|
||||
func testUpdateTermFromMessage(t *testing.T, state StateType) {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
switch state {
|
||||
case StateFollower:
|
||||
r.becomeFollower(1, 2)
|
||||
@ -82,7 +82,7 @@ func TestRejectStaleTermMessage(t *testing.T) {
|
||||
fakeStep := func(r *raft, m pb.Message) {
|
||||
called = true
|
||||
}
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
r.step = fakeStep
|
||||
r.loadState(pb.HardState{Term: 2})
|
||||
|
||||
@ -96,7 +96,7 @@ func TestRejectStaleTermMessage(t *testing.T) {
|
||||
// TestStartAsFollower tests that when servers start up, they begin as followers.
|
||||
// Reference: section 5.2
|
||||
func TestStartAsFollower(t *testing.T) {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
if r.state != StateFollower {
|
||||
t.Errorf("state = %s, want %s", r.state, StateFollower)
|
||||
}
|
||||
@ -109,11 +109,11 @@ func TestStartAsFollower(t *testing.T) {
|
||||
func TestLeaderBcastBeat(t *testing.T) {
|
||||
// heartbeat interval
|
||||
hi := 1
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, hi)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, hi, NewMemoryStorage())
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
for i := 0; i < 10; i++ {
|
||||
r.appendEntry(pb.Entry{})
|
||||
r.appendEntry(pb.Entry{Index: uint64(i) + 1})
|
||||
}
|
||||
|
||||
for i := 0; i <= hi; i++ {
|
||||
@ -151,7 +151,7 @@ func TestCandidateStartNewElection(t *testing.T) {
|
||||
func testNonleaderStartElection(t *testing.T, state StateType) {
|
||||
// election timeout
|
||||
et := 10
|
||||
r := newRaft(1, []uint64{1, 2, 3}, et, 1)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, et, 1, NewMemoryStorage())
|
||||
switch state {
|
||||
case StateFollower:
|
||||
r.becomeFollower(1, 2)
|
||||
@ -215,7 +215,7 @@ func TestLeaderElectionInOneRoundRPC(t *testing.T) {
|
||||
{5, map[uint64]bool{}, StateCandidate},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, idsBySize(tt.size), 10, 1)
|
||||
r := newRaft(1, idsBySize(tt.size), 10, 1, NewMemoryStorage())
|
||||
|
||||
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
|
||||
for id, vote := range tt.votes {
|
||||
@ -248,7 +248,7 @@ func TestFollowerVote(t *testing.T) {
|
||||
{2, 1, true},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
r.loadState(pb.HardState{Term: 1, Vote: tt.vote})
|
||||
|
||||
r.Step(pb.Message{From: tt.nvote, To: 1, Term: 1, Type: pb.MsgVote})
|
||||
@ -274,7 +274,7 @@ func TestCandidateFallback(t *testing.T) {
|
||||
{From: 2, To: 1, Term: 2, Type: pb.MsgApp},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
|
||||
if r.state != StateCandidate {
|
||||
t.Fatalf("unexpected state = %s, want %s", r.state, StateCandidate)
|
||||
@ -303,7 +303,7 @@ func TestCandidateElectionTimeoutRandomized(t *testing.T) {
|
||||
// Reference: section 5.2
|
||||
func testNonleaderElectionTimeoutRandomized(t *testing.T, state StateType) {
|
||||
et := 10
|
||||
r := newRaft(1, []uint64{1, 2, 3}, et, 1)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, et, 1, NewMemoryStorage())
|
||||
timeouts := make(map[int]bool)
|
||||
for round := 0; round < 50*et; round++ {
|
||||
switch state {
|
||||
@ -345,7 +345,7 @@ func testNonleadersElectionTimeoutNonconflict(t *testing.T, state StateType) {
|
||||
rs := make([]*raft, size)
|
||||
ids := idsBySize(size)
|
||||
for k := range rs {
|
||||
rs[k] = newRaft(ids[k], ids, et, 1)
|
||||
rs[k] = newRaft(ids[k], ids, et, 1, NewMemoryStorage())
|
||||
}
|
||||
conflicts := 0
|
||||
for round := 0; round < 1000; round++ {
|
||||
@ -387,10 +387,11 @@ func testNonleadersElectionTimeoutNonconflict(t *testing.T, state StateType) {
|
||||
// Also, it writes the new entry into stable storage.
|
||||
// Reference: section 5.3
|
||||
func TestLeaderStartReplication(t *testing.T) {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
s := NewMemoryStorage()
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, s)
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
commitNoopEntry(r)
|
||||
commitNoopEntry(r, s)
|
||||
li := r.raftLog.lastIndex()
|
||||
|
||||
ents := []pb.Entry{{Data: []byte("some data")}}
|
||||
@ -412,7 +413,7 @@ func TestLeaderStartReplication(t *testing.T) {
|
||||
if !reflect.DeepEqual(msgs, wmsgs) {
|
||||
t.Errorf("msgs = %+v, want %+v", msgs, wmsgs)
|
||||
}
|
||||
if g := r.raftLog.unstableEnts(); !reflect.DeepEqual(g, wents) {
|
||||
if g := r.raftLog.unstableEntries(); !reflect.DeepEqual(g, wents) {
|
||||
t.Errorf("ents = %+v, want %+v", g, wents)
|
||||
}
|
||||
}
|
||||
@ -425,10 +426,11 @@ func TestLeaderStartReplication(t *testing.T) {
|
||||
// servers eventually find out.
|
||||
// Reference: section 5.3
|
||||
func TestLeaderCommitEntry(t *testing.T) {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
s := NewMemoryStorage()
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, s)
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
commitNoopEntry(r)
|
||||
commitNoopEntry(r, s)
|
||||
li := r.raftLog.lastIndex()
|
||||
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
|
||||
|
||||
@ -478,10 +480,11 @@ func TestLeaderAcknowledgeCommit(t *testing.T) {
|
||||
{5, map[uint64]bool{2: true, 3: true, 4: true, 5: true}, true},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, idsBySize(tt.size), 10, 1)
|
||||
s := NewMemoryStorage()
|
||||
r := newRaft(1, idsBySize(tt.size), 10, 1, s)
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
commitNoopEntry(r)
|
||||
commitNoopEntry(r, s)
|
||||
li := r.raftLog.lastIndex()
|
||||
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
|
||||
|
||||
@ -510,8 +513,9 @@ func TestLeaderCommitPrecedingEntries(t *testing.T) {
|
||||
{{Term: 1, Index: 1}},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r.loadEnts(append([]pb.Entry{{}}, tt...))
|
||||
storage := NewMemoryStorage()
|
||||
storage.Append(tt)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage)
|
||||
r.loadState(pb.HardState{Term: 2})
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
@ -566,7 +570,7 @@ func TestFollowerCommitEntry(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
r.becomeFollower(1, 2)
|
||||
|
||||
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 1, Entries: tt.ents, Commit: tt.commit})
|
||||
@ -587,21 +591,22 @@ func TestFollowerCommitEntry(t *testing.T) {
|
||||
// append entries.
|
||||
// Reference: section 5.3
|
||||
func TestFollowerCheckMsgApp(t *testing.T) {
|
||||
ents := []pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}}
|
||||
ents := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}
|
||||
tests := []struct {
|
||||
term uint64
|
||||
index uint64
|
||||
wreject bool
|
||||
}{
|
||||
{ents[0].Term, ents[0].Index, false},
|
||||
{ents[0].Term, ents[0].Index + 1, true},
|
||||
{ents[0].Term + 1, ents[0].Index, true},
|
||||
{ents[1].Term, ents[1].Index, false},
|
||||
{ents[2].Term, ents[2].Index, false},
|
||||
{ents[1].Term, ents[1].Index + 1, true},
|
||||
{ents[1].Term + 1, ents[1].Index, true},
|
||||
{3, 3, true},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r.loadEnts(ents)
|
||||
storage := NewMemoryStorage()
|
||||
storage.Append(ents)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage)
|
||||
r.loadState(pb.HardState{Commit: 2})
|
||||
r.becomeFollower(2, 2)
|
||||
|
||||
@ -632,39 +637,40 @@ func TestFollowerAppendEntries(t *testing.T) {
|
||||
{
|
||||
2, 2,
|
||||
[]pb.Entry{{Term: 3, Index: 3}},
|
||||
[]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}, {Term: 3, Index: 3}},
|
||||
[]pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}, {Term: 3, Index: 3}},
|
||||
[]pb.Entry{{Term: 3, Index: 3}},
|
||||
},
|
||||
{
|
||||
1, 1,
|
||||
[]pb.Entry{{Term: 3, Index: 3}, {Term: 4, Index: 4}},
|
||||
[]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 3, Index: 3}, {Term: 4, Index: 4}},
|
||||
[]pb.Entry{{Term: 1, Index: 1}, {Term: 3, Index: 3}, {Term: 4, Index: 4}},
|
||||
[]pb.Entry{{Term: 3, Index: 3}, {Term: 4, Index: 4}},
|
||||
},
|
||||
{
|
||||
0, 0,
|
||||
[]pb.Entry{{Term: 1, Index: 1}},
|
||||
[]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}},
|
||||
[]pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}},
|
||||
nil,
|
||||
},
|
||||
{
|
||||
0, 0,
|
||||
[]pb.Entry{{Term: 3, Index: 3}},
|
||||
[]pb.Entry{{}, {Term: 3, Index: 3}},
|
||||
[]pb.Entry{{Term: 3, Index: 3}},
|
||||
[]pb.Entry{{Term: 3, Index: 3}},
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r.loadEnts([]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}})
|
||||
storage := NewMemoryStorage()
|
||||
storage.Append([]pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}})
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, storage)
|
||||
r.becomeFollower(2, 2)
|
||||
|
||||
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 2, LogTerm: tt.term, Index: tt.index, Entries: tt.ents})
|
||||
|
||||
if g := r.raftLog.ents; !reflect.DeepEqual(g, tt.wents) {
|
||||
if g := r.raftLog.allEntries(); !reflect.DeepEqual(g, tt.wents) {
|
||||
t.Errorf("#%d: ents = %+v, want %+v", i, g, tt.wents)
|
||||
}
|
||||
if g := r.raftLog.unstableEnts(); !reflect.DeepEqual(g, tt.wunstable) {
|
||||
if g := r.raftLog.unstableEntries(); !reflect.DeepEqual(g, tt.wunstable) {
|
||||
t.Errorf("#%d: unstableEnts = %+v, want %+v", i, g, tt.wunstable)
|
||||
}
|
||||
}
|
||||
@ -723,11 +729,13 @@ func TestLeaderSyncFollowerLog(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
lead := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
lead.loadEnts(ents)
|
||||
leadStorage := NewMemoryStorage()
|
||||
leadStorage.Append(ents)
|
||||
lead := newRaft(1, []uint64{1, 2, 3}, 10, 1, leadStorage)
|
||||
lead.loadState(pb.HardState{Commit: lead.raftLog.lastIndex(), Term: term})
|
||||
follower := newRaft(2, []uint64{1, 2, 3}, 10, 1)
|
||||
follower.loadEnts(tt)
|
||||
followerStorage := NewMemoryStorage()
|
||||
followerStorage.Append(tt)
|
||||
follower := newRaft(2, []uint64{1, 2, 3}, 10, 1, followerStorage)
|
||||
follower.loadState(pb.HardState{Term: term - 1})
|
||||
// It is necessary to have a three-node cluster.
|
||||
// The second may have more up-to-date log than the first one, so the
|
||||
@ -756,7 +764,7 @@ func TestVoteRequest(t *testing.T) {
|
||||
{[]pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}, 3},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
r.Step(pb.Message{
|
||||
From: 2, To: 1, Type: pb.MsgApp, Term: tt.wterm - 1, LogTerm: 0, Index: 0, Entries: tt.ents,
|
||||
})
|
||||
@ -804,21 +812,22 @@ func TestVoter(t *testing.T) {
|
||||
wreject bool
|
||||
}{
|
||||
// same logterm
|
||||
{[]pb.Entry{{}, {Term: 1, Index: 1}}, 1, 1, false},
|
||||
{[]pb.Entry{{}, {Term: 1, Index: 1}}, 1, 2, false},
|
||||
{[]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true},
|
||||
{[]pb.Entry{{Term: 1, Index: 1}}, 1, 1, false},
|
||||
{[]pb.Entry{{Term: 1, Index: 1}}, 1, 2, false},
|
||||
{[]pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true},
|
||||
// candidate higher logterm
|
||||
{[]pb.Entry{{}, {Term: 1, Index: 1}}, 2, 1, false},
|
||||
{[]pb.Entry{{}, {Term: 1, Index: 1}}, 2, 2, false},
|
||||
{[]pb.Entry{{}, {Term: 1, Index: 1}, {Term: 1, Index: 2}}, 2, 1, false},
|
||||
{[]pb.Entry{{Term: 1, Index: 1}}, 2, 1, false},
|
||||
{[]pb.Entry{{Term: 1, Index: 1}}, 2, 2, false},
|
||||
{[]pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}}, 2, 1, false},
|
||||
// voter higher logterm
|
||||
{[]pb.Entry{{}, {Term: 2, Index: 1}}, 1, 1, true},
|
||||
{[]pb.Entry{{}, {Term: 2, Index: 1}}, 1, 2, true},
|
||||
{[]pb.Entry{{}, {Term: 2, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true},
|
||||
{[]pb.Entry{{Term: 2, Index: 1}}, 1, 1, true},
|
||||
{[]pb.Entry{{Term: 2, Index: 1}}, 1, 2, true},
|
||||
{[]pb.Entry{{Term: 2, Index: 1}, {Term: 1, Index: 2}}, 1, 1, true},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1)
|
||||
r.loadEnts(tt.ents)
|
||||
storage := NewMemoryStorage()
|
||||
storage.Append(tt.ents)
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1, storage)
|
||||
|
||||
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgVote, Term: 3, LogTerm: tt.logterm, Index: tt.index})
|
||||
|
||||
@ -840,7 +849,7 @@ func TestVoter(t *testing.T) {
|
||||
// current term are committed by counting replicas.
|
||||
// Reference: section 5.4.2
|
||||
func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) {
|
||||
ents := []pb.Entry{{}, {Term: 1, Index: 1}, {Term: 2, Index: 2}}
|
||||
ents := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}
|
||||
tests := []struct {
|
||||
index uint64
|
||||
wcommit uint64
|
||||
@ -852,8 +861,9 @@ func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) {
|
||||
{3, 3},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1)
|
||||
r.loadEnts(ents)
|
||||
storage := NewMemoryStorage()
|
||||
storage.Append(ents)
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1, storage)
|
||||
r.loadState(pb.HardState{Term: 2})
|
||||
// become leader at term 3
|
||||
r.becomeCandidate()
|
||||
@ -875,7 +885,7 @@ func (s messageSlice) Len() int { return len(s) }
|
||||
func (s messageSlice) Less(i, j int) bool { return fmt.Sprint(s[i]) < fmt.Sprint(s[j]) }
|
||||
func (s messageSlice) Swap(i, j int) { s[i], s[j] = s[j], s[i] }
|
||||
|
||||
func commitNoopEntry(r *raft) {
|
||||
func commitNoopEntry(r *raft, s *MemoryStorage) {
|
||||
if r.state != StateLeader {
|
||||
panic("it should only be used when it is the leader")
|
||||
}
|
||||
@ -890,6 +900,7 @@ func commitNoopEntry(r *raft) {
|
||||
}
|
||||
// ignore further messages to refresh followers' commmit index
|
||||
r.readMessages()
|
||||
s.Append(r.raftLog.unstableEntries())
|
||||
r.raftLog.appliedTo(r.raftLog.committed)
|
||||
r.raftLog.stableTo(r.raftLog.lastIndex())
|
||||
}
|
||||
|
@ -22,14 +22,17 @@ import (
|
||||
"math"
|
||||
"math/rand"
|
||||
"reflect"
|
||||
"sort"
|
||||
"testing"
|
||||
|
||||
pb "github.com/coreos/etcd/raft/raftpb"
|
||||
)
|
||||
|
||||
// nextEnts returns the appliable entries and updates the applied index
|
||||
func nextEnts(r *raft) (ents []pb.Entry) {
|
||||
func nextEnts(r *raft, s *MemoryStorage) (ents []pb.Entry) {
|
||||
// Transfer all unstable entries to "stable" storage.
|
||||
s.Append(r.raftLog.unstableEntries())
|
||||
r.raftLog.stableTo(r.raftLog.lastIndex())
|
||||
|
||||
ents = r.raftLog.nextEnts()
|
||||
r.raftLog.appliedTo(r.raftLog.committed)
|
||||
return ents
|
||||
@ -209,7 +212,7 @@ func TestLogReplication(t *testing.T) {
|
||||
}
|
||||
|
||||
ents := []pb.Entry{}
|
||||
for _, e := range nextEnts(sm) {
|
||||
for _, e := range nextEnts(sm, tt.network.storage[j]) {
|
||||
if e.Data != nil {
|
||||
ents = append(ents, e)
|
||||
}
|
||||
@ -318,9 +321,9 @@ func TestCommitWithoutNewTermEntry(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestDuelingCandidates(t *testing.T) {
|
||||
a := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
b := newRaft(2, []uint64{1, 2, 3}, 10, 1)
|
||||
c := newRaft(3, []uint64{1, 2, 3}, 10, 1)
|
||||
a := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
b := newRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
c := newRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
|
||||
nt := newNetwork(a, b, c)
|
||||
nt.cut(1, 3)
|
||||
@ -331,7 +334,11 @@ func TestDuelingCandidates(t *testing.T) {
|
||||
nt.recover()
|
||||
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
|
||||
|
||||
wlog := &raftLog{ents: []pb.Entry{{}, pb.Entry{Data: nil, Term: 1, Index: 1}}, committed: 1}
|
||||
wlog := &raftLog{
|
||||
storage: &MemoryStorage{ents: []pb.Entry{{}, pb.Entry{Data: nil, Term: 1, Index: 1}}},
|
||||
committed: 1,
|
||||
unstable: 2,
|
||||
}
|
||||
tests := []struct {
|
||||
sm *raft
|
||||
state StateType
|
||||
@ -340,7 +347,7 @@ func TestDuelingCandidates(t *testing.T) {
|
||||
}{
|
||||
{a, StateFollower, 2, wlog},
|
||||
{b, StateFollower, 2, wlog},
|
||||
{c, StateFollower, 2, newLog()},
|
||||
{c, StateFollower, 2, newLog(NewMemoryStorage())},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
@ -383,7 +390,13 @@ func TestCandidateConcede(t *testing.T) {
|
||||
if g := a.Term; g != 1 {
|
||||
t.Errorf("term = %d, want %d", g, 1)
|
||||
}
|
||||
wantLog := ltoa(&raftLog{ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}}, committed: 2})
|
||||
wantLog := ltoa(&raftLog{
|
||||
storage: &MemoryStorage{
|
||||
ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}},
|
||||
},
|
||||
unstable: 3,
|
||||
committed: 2,
|
||||
})
|
||||
for i, p := range tt.peers {
|
||||
if sm, ok := p.(*raft); ok {
|
||||
l := ltoa(sm.raftLog)
|
||||
@ -416,10 +429,13 @@ func TestOldMessages(t *testing.T) {
|
||||
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgApp, Term: 1, Entries: []pb.Entry{{Term: 1}}})
|
||||
|
||||
l := &raftLog{
|
||||
ents: []pb.Entry{
|
||||
{}, {Data: nil, Term: 1, Index: 1},
|
||||
{Data: nil, Term: 2, Index: 2}, {Data: nil, Term: 3, Index: 3},
|
||||
storage: &MemoryStorage{
|
||||
ents: []pb.Entry{
|
||||
{}, {Data: nil, Term: 1, Index: 1},
|
||||
{Data: nil, Term: 2, Index: 2}, {Data: nil, Term: 3, Index: 3},
|
||||
},
|
||||
},
|
||||
unstable: 4,
|
||||
committed: 3,
|
||||
}
|
||||
base := ltoa(l)
|
||||
@ -470,9 +486,14 @@ func TestProposal(t *testing.T) {
|
||||
send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
|
||||
send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}})
|
||||
|
||||
wantLog := newLog()
|
||||
wantLog := newLog(NewMemoryStorage())
|
||||
if tt.success {
|
||||
wantLog = &raftLog{ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}}, committed: 2}
|
||||
wantLog = &raftLog{
|
||||
storage: &MemoryStorage{
|
||||
ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}},
|
||||
},
|
||||
unstable: 3,
|
||||
committed: 2}
|
||||
}
|
||||
base := ltoa(wantLog)
|
||||
for i, p := range tt.peers {
|
||||
@ -506,7 +527,12 @@ func TestProposalByProxy(t *testing.T) {
|
||||
// propose via follower
|
||||
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
|
||||
|
||||
wantLog := &raftLog{ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Data: data, Index: 2}}, committed: 2}
|
||||
wantLog := &raftLog{
|
||||
storage: &MemoryStorage{
|
||||
ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Data: data, Index: 2}},
|
||||
},
|
||||
unstable: 3,
|
||||
committed: 2}
|
||||
base := ltoa(wantLog)
|
||||
for i, p := range tt.peers {
|
||||
if sm, ok := p.(*raft); ok {
|
||||
@ -525,50 +551,6 @@ func TestProposalByProxy(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestCompact(t *testing.T) {
|
||||
tests := []struct {
|
||||
compacti uint64
|
||||
nodes []uint64
|
||||
snapd []byte
|
||||
wpanic bool
|
||||
}{
|
||||
{1, []uint64{1, 2, 3}, []byte("some data"), false},
|
||||
{2, []uint64{1, 2, 3}, []byte("some data"), false},
|
||||
{4, []uint64{1, 2, 3}, []byte("some data"), true}, // compact out of range
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
func() {
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
if tt.wpanic != true {
|
||||
t.Errorf("%d: panic = %v, want %v", i, true, tt.wpanic)
|
||||
}
|
||||
}
|
||||
}()
|
||||
sm := &raft{
|
||||
state: StateLeader,
|
||||
raftLog: &raftLog{
|
||||
committed: 2,
|
||||
applied: 2,
|
||||
ents: []pb.Entry{{}, {Term: 1}, {Term: 1}, {Term: 1}},
|
||||
},
|
||||
}
|
||||
sm.compact(tt.compacti, tt.nodes, tt.snapd)
|
||||
sort.Sort(uint64Slice(sm.raftLog.snapshot.Nodes))
|
||||
if sm.raftLog.offset != tt.compacti {
|
||||
t.Errorf("%d: log.offset = %d, want %d", i, sm.raftLog.offset, tt.compacti)
|
||||
}
|
||||
if !reflect.DeepEqual(sm.raftLog.snapshot.Nodes, tt.nodes) {
|
||||
t.Errorf("%d: snap.nodes = %v, want %v", i, sm.raftLog.snapshot.Nodes, tt.nodes)
|
||||
}
|
||||
if !reflect.DeepEqual(sm.raftLog.snapshot.Data, tt.snapd) {
|
||||
t.Errorf("%d: snap.data = %v, want %v", i, sm.raftLog.snapshot.Data, tt.snapd)
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
func TestCommit(t *testing.T) {
|
||||
tests := []struct {
|
||||
matches []uint64
|
||||
@ -602,7 +584,11 @@ func TestCommit(t *testing.T) {
|
||||
for j := 0; j < len(tt.matches); j++ {
|
||||
prs[uint64(j)] = &progress{tt.matches[j], tt.matches[j] + 1}
|
||||
}
|
||||
sm := &raft{raftLog: &raftLog{ents: tt.logs}, prs: prs, HardState: pb.HardState{Term: tt.smTerm}}
|
||||
sm := &raft{
|
||||
raftLog: &raftLog{storage: &MemoryStorage{ents: tt.logs}, unstable: uint64(len(tt.logs))},
|
||||
prs: prs,
|
||||
HardState: pb.HardState{Term: tt.smTerm},
|
||||
}
|
||||
sm.maybeCommit()
|
||||
if g := sm.raftLog.committed; g != tt.w {
|
||||
t.Errorf("#%d: committed = %d, want %d", i, g, tt.w)
|
||||
@ -624,7 +610,7 @@ func TestIsElectionTimeout(t *testing.T) {
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
sm := newRaft(1, []uint64{1}, 10, 1)
|
||||
sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
||||
sm.elapsed = tt.elapse
|
||||
c := 0
|
||||
for j := 0; j < 10000; j++ {
|
||||
@ -649,7 +635,7 @@ func TestStepIgnoreOldTermMsg(t *testing.T) {
|
||||
fakeStep := func(r *raft, m pb.Message) {
|
||||
called = true
|
||||
}
|
||||
sm := newRaft(1, []uint64{1}, 10, 1)
|
||||
sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
||||
sm.step = fakeStep
|
||||
sm.Term = 2
|
||||
sm.Step(pb.Message{Type: pb.MsgApp, Term: sm.Term - 1})
|
||||
@ -692,7 +678,11 @@ func TestHandleMsgApp(t *testing.T) {
|
||||
sm := &raft{
|
||||
state: StateFollower,
|
||||
HardState: pb.HardState{Term: 2},
|
||||
raftLog: &raftLog{committed: 0, ents: []pb.Entry{{}, {Term: 1}, {Term: 2}}},
|
||||
raftLog: &raftLog{
|
||||
committed: 0,
|
||||
storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 1}, {Term: 2}}},
|
||||
unstable: 3,
|
||||
},
|
||||
}
|
||||
|
||||
sm.handleAppendEntries(tt.m)
|
||||
@ -724,10 +714,12 @@ func TestHandleHeartbeat(t *testing.T) {
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
storage := NewMemoryStorage()
|
||||
storage.Append([]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}})
|
||||
sm := &raft{
|
||||
state: StateFollower,
|
||||
HardState: pb.HardState{Term: 2},
|
||||
raftLog: &raftLog{committed: 0, ents: []pb.Entry{{}, {Term: 1}, {Term: 2}, {Term: 3}}},
|
||||
raftLog: newLog(storage),
|
||||
}
|
||||
sm.raftLog.commitTo(commit)
|
||||
sm.handleHeartbeat(tt.m)
|
||||
@ -776,7 +768,7 @@ func TestRecvMsgVote(t *testing.T) {
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
sm := newRaft(1, []uint64{1}, 10, 1)
|
||||
sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
||||
sm.state = tt.state
|
||||
switch tt.state {
|
||||
case StateFollower:
|
||||
@ -787,7 +779,10 @@ func TestRecvMsgVote(t *testing.T) {
|
||||
sm.step = stepLeader
|
||||
}
|
||||
sm.HardState = pb.HardState{Vote: tt.voteFor}
|
||||
sm.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 2}, {Term: 2}}}
|
||||
sm.raftLog = &raftLog{
|
||||
storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 2}, {Index: 2, Term: 2}}},
|
||||
unstable: 3,
|
||||
}
|
||||
|
||||
sm.Step(pb.Message{Type: pb.MsgVote, From: 2, Index: tt.i, LogTerm: tt.term})
|
||||
|
||||
@ -833,7 +828,7 @@ func TestStateTransition(t *testing.T) {
|
||||
}
|
||||
}()
|
||||
|
||||
sm := newRaft(1, []uint64{1}, 10, 1)
|
||||
sm := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
||||
sm.state = tt.from
|
||||
|
||||
switch tt.to {
|
||||
@ -863,16 +858,16 @@ func TestAllServerStepdown(t *testing.T) {
|
||||
wterm uint64
|
||||
windex uint64
|
||||
}{
|
||||
{StateFollower, StateFollower, 3, 1},
|
||||
{StateCandidate, StateFollower, 3, 1},
|
||||
{StateLeader, StateFollower, 3, 2},
|
||||
{StateFollower, StateFollower, 3, 0},
|
||||
{StateCandidate, StateFollower, 3, 0},
|
||||
{StateLeader, StateFollower, 3, 1},
|
||||
}
|
||||
|
||||
tmsgTypes := [...]pb.MessageType{pb.MsgVote, pb.MsgApp}
|
||||
tterm := uint64(3)
|
||||
|
||||
for i, tt := range tests {
|
||||
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
switch tt.state {
|
||||
case StateFollower:
|
||||
sm.becomeFollower(1, None)
|
||||
@ -892,8 +887,11 @@ func TestAllServerStepdown(t *testing.T) {
|
||||
if sm.Term != tt.wterm {
|
||||
t.Errorf("#%d.%d term = %v , want %v", i, j, sm.Term, tt.wterm)
|
||||
}
|
||||
if uint64(len(sm.raftLog.ents)) != tt.windex {
|
||||
t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.ents), tt.windex)
|
||||
if uint64(sm.raftLog.lastIndex()) != tt.windex {
|
||||
t.Errorf("#%d.%d index = %v , want %v", i, j, sm.raftLog.lastIndex(), tt.windex)
|
||||
}
|
||||
if uint64(len(sm.raftLog.allEntries())) != tt.windex {
|
||||
t.Errorf("#%d.%d len(ents) = %v , want %v", i, j, len(sm.raftLog.allEntries()), tt.windex)
|
||||
}
|
||||
wlead := uint64(2)
|
||||
if msgType == pb.MsgVote {
|
||||
@ -928,8 +926,11 @@ func TestLeaderAppResp(t *testing.T) {
|
||||
for i, tt := range tests {
|
||||
// sm term is 1 after it becomes the leader.
|
||||
// thus the last log term must be 1 to be committed.
|
||||
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
sm.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}
|
||||
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
sm.raftLog = &raftLog{
|
||||
storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}},
|
||||
unstable: 3,
|
||||
}
|
||||
sm.becomeCandidate()
|
||||
sm.becomeLeader()
|
||||
sm.readMessages()
|
||||
@ -965,18 +966,21 @@ func TestBcastBeat(t *testing.T) {
|
||||
offset := uint64(1000)
|
||||
// make a state machine with log.offset = 1000
|
||||
s := pb.Snapshot{
|
||||
Index: offset,
|
||||
Term: 1,
|
||||
Nodes: []uint64{1, 2, 3},
|
||||
Metadata: pb.SnapshotMetadata{
|
||||
Index: offset,
|
||||
Term: 1,
|
||||
ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
|
||||
},
|
||||
}
|
||||
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
storage := NewMemoryStorage()
|
||||
storage.ApplySnapshot(s)
|
||||
sm := newRaft(1, nil, 10, 1, storage)
|
||||
sm.Term = 1
|
||||
sm.restore(s)
|
||||
|
||||
sm.becomeCandidate()
|
||||
sm.becomeLeader()
|
||||
for i := 0; i < 10; i++ {
|
||||
sm.appendEntry(pb.Entry{})
|
||||
sm.appendEntry(pb.Entry{Index: uint64(i) + 1})
|
||||
}
|
||||
// slow follower
|
||||
sm.prs[2].match, sm.prs[2].next = 5, 6
|
||||
@ -1029,8 +1033,8 @@ func TestRecvMsgBeat(t *testing.T) {
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1)
|
||||
sm.raftLog = &raftLog{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}
|
||||
sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}}}
|
||||
sm.Term = 1
|
||||
sm.state = tt.state
|
||||
switch tt.state {
|
||||
@ -1072,7 +1076,7 @@ func TestLeaderIncreaseNext(t *testing.T) {
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
sm := newRaft(1, []uint64{1, 2}, 10, 1)
|
||||
sm := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
||||
sm.raftLog.append(0, previousEnts...)
|
||||
sm.becomeCandidate()
|
||||
sm.becomeLeader()
|
||||
@ -1088,28 +1092,28 @@ func TestLeaderIncreaseNext(t *testing.T) {
|
||||
|
||||
func TestRestore(t *testing.T) {
|
||||
s := pb.Snapshot{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
Nodes: []uint64{1, 2, 3},
|
||||
Metadata: pb.SnapshotMetadata{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
|
||||
},
|
||||
}
|
||||
|
||||
sm := newRaft(1, []uint64{1, 2}, 10, 1)
|
||||
storage := NewMemoryStorage()
|
||||
sm := newRaft(1, []uint64{1, 2}, 10, 1, storage)
|
||||
if ok := sm.restore(s); !ok {
|
||||
t.Fatal("restore fail, want succeed")
|
||||
}
|
||||
|
||||
if sm.raftLog.lastIndex() != s.Index {
|
||||
t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Index)
|
||||
if sm.raftLog.lastIndex() != s.Metadata.Index {
|
||||
t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Metadata.Index)
|
||||
}
|
||||
if sm.raftLog.term(s.Index) != s.Term {
|
||||
t.Errorf("log.lastTerm = %d, want %d", sm.raftLog.term(s.Index), s.Term)
|
||||
if sm.raftLog.term(s.Metadata.Index) != s.Metadata.Term {
|
||||
t.Errorf("log.lastTerm = %d, want %d", sm.raftLog.term(s.Metadata.Index), s.Metadata.Term)
|
||||
}
|
||||
sg := sm.nodes()
|
||||
if !reflect.DeepEqual(sg, s.Nodes) {
|
||||
t.Errorf("sm.Nodes = %+v, want %+v", sg, s.Nodes)
|
||||
}
|
||||
if !reflect.DeepEqual(sm.raftLog.snapshot, s) {
|
||||
t.Errorf("snapshot = %+v, want %+v", sm.raftLog.snapshot, s)
|
||||
if !reflect.DeepEqual(sg, s.Metadata.ConfState.Nodes) {
|
||||
t.Errorf("sm.Nodes = %+v, want %+v", sg, s.Metadata.ConfState.Nodes)
|
||||
}
|
||||
|
||||
if ok := sm.restore(s); ok {
|
||||
@ -1118,14 +1122,17 @@ func TestRestore(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestProvideSnap(t *testing.T) {
|
||||
s := pb.Snapshot{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
Nodes: []uint64{1, 2},
|
||||
}
|
||||
sm := newRaft(1, []uint64{1}, 10, 1)
|
||||
// restore the statemachin from a snapshot
|
||||
// so it has a compacted log and a snapshot
|
||||
s := pb.Snapshot{
|
||||
Metadata: pb.SnapshotMetadata{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
|
||||
},
|
||||
}
|
||||
storage := NewMemoryStorage()
|
||||
sm := newRaft(1, []uint64{1}, 10, 1, storage)
|
||||
sm.restore(s)
|
||||
|
||||
sm.becomeCandidate()
|
||||
@ -1133,7 +1140,7 @@ func TestProvideSnap(t *testing.T) {
|
||||
|
||||
// force set the next of node 1, so that
|
||||
// node 1 needs a snapshot
|
||||
sm.prs[2].next = sm.raftLog.offset
|
||||
sm.prs[2].next = sm.raftLog.firstIndex()
|
||||
|
||||
sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs[2].next - 1, Reject: true})
|
||||
msgs := sm.readMessages()
|
||||
@ -1148,18 +1155,18 @@ func TestProvideSnap(t *testing.T) {
|
||||
|
||||
func TestRestoreFromSnapMsg(t *testing.T) {
|
||||
s := pb.Snapshot{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
Nodes: []uint64{1, 2},
|
||||
Metadata: pb.SnapshotMetadata{
|
||||
Index: 11, // magic number
|
||||
Term: 11, // magic number
|
||||
ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
|
||||
},
|
||||
}
|
||||
m := pb.Message{Type: pb.MsgSnap, From: 1, Term: 2, Snapshot: s}
|
||||
|
||||
sm := newRaft(2, []uint64{1, 2}, 10, 1)
|
||||
sm := newRaft(2, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
||||
sm.Step(m)
|
||||
|
||||
if !reflect.DeepEqual(sm.raftLog.snapshot, s) {
|
||||
t.Errorf("snapshot = %+v, want %+v", sm.raftLog.snapshot, s)
|
||||
}
|
||||
// TODO(bdarnell): what should this test?
|
||||
}
|
||||
|
||||
func TestSlowNodeRestore(t *testing.T) {
|
||||
@ -1171,16 +1178,13 @@ func TestSlowNodeRestore(t *testing.T) {
|
||||
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
|
||||
}
|
||||
lead := nt.peers[1].(*raft)
|
||||
nextEnts(lead)
|
||||
lead.compact(lead.raftLog.applied, lead.nodes(), nil)
|
||||
nextEnts(lead, nt.storage[1])
|
||||
nt.storage[1].Compact(lead.raftLog.applied, &pb.ConfState{Nodes: lead.nodes()}, nil)
|
||||
|
||||
nt.recover()
|
||||
// trigger a snapshot
|
||||
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
|
||||
follower := nt.peers[3].(*raft)
|
||||
if !reflect.DeepEqual(follower.raftLog.snapshot, lead.raftLog.snapshot) {
|
||||
t.Errorf("follower.snap = %+v, want %+v", follower.raftLog.snapshot, lead.raftLog.snapshot)
|
||||
}
|
||||
|
||||
// trigger a commit
|
||||
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
|
||||
@ -1193,7 +1197,7 @@ func TestSlowNodeRestore(t *testing.T) {
|
||||
// it appends the entry to log and sets pendingConf to be true.
|
||||
func TestStepConfig(t *testing.T) {
|
||||
// a raft that cannot make progress
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
index := r.raftLog.lastIndex()
|
||||
@ -1211,7 +1215,7 @@ func TestStepConfig(t *testing.T) {
|
||||
// the proposal and keep its original state.
|
||||
func TestStepIgnoreConfig(t *testing.T) {
|
||||
// a raft that cannot make progress
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange}}})
|
||||
@ -1237,7 +1241,7 @@ func TestRecoverPendingConfig(t *testing.T) {
|
||||
{pb.EntryConfChange, true},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
||||
r.appendEntry(pb.Entry{Type: tt.entType})
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
@ -1256,7 +1260,7 @@ func TestRecoverDoublePendingConfig(t *testing.T) {
|
||||
t.Errorf("expect panic, but nothing happens")
|
||||
}
|
||||
}()
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
||||
r.appendEntry(pb.Entry{Type: pb.EntryConfChange})
|
||||
r.appendEntry(pb.Entry{Type: pb.EntryConfChange})
|
||||
r.becomeCandidate()
|
||||
@ -1266,7 +1270,7 @@ func TestRecoverDoublePendingConfig(t *testing.T) {
|
||||
|
||||
// TestAddNode tests that addNode could update pendingConf and nodes correctly.
|
||||
func TestAddNode(t *testing.T) {
|
||||
r := newRaft(1, []uint64{1}, 10, 1)
|
||||
r := newRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
||||
r.pendingConf = true
|
||||
r.addNode(2)
|
||||
if r.pendingConf != false {
|
||||
@ -1282,7 +1286,7 @@ func TestAddNode(t *testing.T) {
|
||||
// TestRemoveNode tests that removeNode could update pendingConf, nodes and
|
||||
// and removed list correctly.
|
||||
func TestRemoveNode(t *testing.T) {
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1)
|
||||
r := newRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
||||
r.pendingConf = true
|
||||
r.removeNode(2)
|
||||
if r.pendingConf != false {
|
||||
@ -1331,7 +1335,7 @@ func TestRaftNodes(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
r := newRaft(1, tt.ids, 10, 1)
|
||||
r := newRaft(1, tt.ids, 10, 1, NewMemoryStorage())
|
||||
if !reflect.DeepEqual(r.nodes(), tt.wids) {
|
||||
t.Errorf("#%d: nodes = %+v, want %+v", i, r.nodes(), tt.wids)
|
||||
}
|
||||
@ -1340,17 +1344,23 @@ func TestRaftNodes(t *testing.T) {
|
||||
|
||||
func ents(terms ...uint64) *raft {
|
||||
ents := []pb.Entry{{}}
|
||||
for _, term := range terms {
|
||||
ents = append(ents, pb.Entry{Term: term})
|
||||
for i, term := range terms {
|
||||
ents = append(ents, pb.Entry{Index: uint64(i), Term: term})
|
||||
}
|
||||
|
||||
sm := &raft{raftLog: &raftLog{ents: ents}}
|
||||
sm := &raft{
|
||||
raftLog: &raftLog{
|
||||
storage: &MemoryStorage{ents: ents},
|
||||
unstable: uint64(len(ents)),
|
||||
},
|
||||
}
|
||||
sm.reset(0)
|
||||
return sm
|
||||
}
|
||||
|
||||
type network struct {
|
||||
peers map[uint64]Interface
|
||||
storage map[uint64]*MemoryStorage
|
||||
dropm map[connem]float64
|
||||
ignorem map[pb.MessageType]bool
|
||||
}
|
||||
@ -1364,12 +1374,14 @@ func newNetwork(peers ...Interface) *network {
|
||||
peerAddrs := idsBySize(size)
|
||||
|
||||
npeers := make(map[uint64]Interface, size)
|
||||
nstorage := make(map[uint64]*MemoryStorage, size)
|
||||
|
||||
for i, p := range peers {
|
||||
id := peerAddrs[i]
|
||||
switch v := p.(type) {
|
||||
case nil:
|
||||
sm := newRaft(id, peerAddrs, 10, 1)
|
||||
nstorage[id] = NewMemoryStorage()
|
||||
sm := newRaft(id, peerAddrs, 10, 1, nstorage[id])
|
||||
npeers[id] = sm
|
||||
case *raft:
|
||||
v.id = id
|
||||
@ -1387,6 +1399,7 @@ func newNetwork(peers ...Interface) *network {
|
||||
}
|
||||
return &network{
|
||||
peers: npeers,
|
||||
storage: nstorage,
|
||||
dropm: make(map[connem]float64),
|
||||
ignorem: make(map[pb.MessageType]bool),
|
||||
}
|
||||
|
@ -10,9 +10,11 @@
|
||||
|
||||
It has these top-level messages:
|
||||
Entry
|
||||
SnapshotMetadata
|
||||
Snapshot
|
||||
Message
|
||||
HardState
|
||||
ConfState
|
||||
ConfChange
|
||||
*/
|
||||
package raftpb
|
||||
@ -163,12 +165,21 @@ func (m *Entry) Reset() { *m = Entry{} }
|
||||
func (m *Entry) String() string { return proto.CompactTextString(m) }
|
||||
func (*Entry) ProtoMessage() {}
|
||||
|
||||
type SnapshotMetadata struct {
|
||||
ConfState ConfState `protobuf:"bytes,1,req,name=conf_state" json:"conf_state"`
|
||||
Index uint64 `protobuf:"varint,2,req,name=index" json:"index"`
|
||||
Term uint64 `protobuf:"varint,3,req,name=term" json:"term"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
}
|
||||
|
||||
func (m *SnapshotMetadata) Reset() { *m = SnapshotMetadata{} }
|
||||
func (m *SnapshotMetadata) String() string { return proto.CompactTextString(m) }
|
||||
func (*SnapshotMetadata) ProtoMessage() {}
|
||||
|
||||
type Snapshot struct {
|
||||
Data []byte `protobuf:"bytes,1,req,name=data" json:"data"`
|
||||
Nodes []uint64 `protobuf:"varint,2,rep,name=nodes" json:"nodes"`
|
||||
Index uint64 `protobuf:"varint,3,req,name=index" json:"index"`
|
||||
Term uint64 `protobuf:"varint,4,req,name=term" json:"term"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
Data []byte `protobuf:"bytes,1,opt,name=data" json:"data"`
|
||||
Metadata SnapshotMetadata `protobuf:"bytes,2,req,name=metadata" json:"metadata"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Snapshot) Reset() { *m = Snapshot{} }
|
||||
@ -204,6 +215,15 @@ func (m *HardState) Reset() { *m = HardState{} }
|
||||
func (m *HardState) String() string { return proto.CompactTextString(m) }
|
||||
func (*HardState) ProtoMessage() {}
|
||||
|
||||
type ConfState struct {
|
||||
Nodes []uint64 `protobuf:"varint,1,rep,name=nodes" json:"nodes"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
}
|
||||
|
||||
func (m *ConfState) Reset() { *m = ConfState{} }
|
||||
func (m *ConfState) String() string { return proto.CompactTextString(m) }
|
||||
func (*ConfState) ProtoMessage() {}
|
||||
|
||||
type ConfChange struct {
|
||||
ID uint64 `protobuf:"varint,1,req" json:"ID"`
|
||||
Type ConfChangeType `protobuf:"varint,2,req,enum=raftpb.ConfChangeType" json:"Type"`
|
||||
@ -330,6 +350,102 @@ func (m *Entry) Unmarshal(data []byte) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *SnapshotMetadata) Unmarshal(data []byte) error {
|
||||
l := len(data)
|
||||
index := 0
|
||||
for index < l {
|
||||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
wire |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
fieldNum := int32(wire >> 3)
|
||||
wireType := int(wire & 0x7)
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
msglen |= (int(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
postIndex := index + msglen
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
if err := m.ConfState.Unmarshal(data[index:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
index = postIndex
|
||||
case 2:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
m.Index |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
case 3:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
m.Term |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
default:
|
||||
var sizeOfWire int
|
||||
for {
|
||||
sizeOfWire++
|
||||
wire >>= 7
|
||||
if wire == 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
index -= sizeOfWire
|
||||
skippy, err := code_google_com_p_gogoprotobuf_proto.Skip(data[index:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (index + skippy) > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
|
||||
index += skippy
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *Snapshot) Unmarshal(data []byte) error {
|
||||
l := len(data)
|
||||
index := 0
|
||||
@ -372,52 +488,29 @@ func (m *Snapshot) Unmarshal(data []byte) error {
|
||||
m.Data = append(m.Data, data[index:postIndex]...)
|
||||
index = postIndex
|
||||
case 2:
|
||||
if wireType != 0 {
|
||||
if wireType != 2 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
var v uint64
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
v |= (uint64(b) & 0x7F) << shift
|
||||
msglen |= (int(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.Nodes = append(m.Nodes, v)
|
||||
case 3:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
postIndex := index + msglen
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
m.Index |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
case 4:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
m.Term |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
if err := m.Metadata.Unmarshal(data[index:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
index = postIndex
|
||||
default:
|
||||
var sizeOfWire int
|
||||
for {
|
||||
@ -739,6 +832,65 @@ func (m *HardState) Unmarshal(data []byte) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *ConfState) Unmarshal(data []byte) error {
|
||||
l := len(data)
|
||||
index := 0
|
||||
for index < l {
|
||||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
wire |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
fieldNum := int32(wire >> 3)
|
||||
wireType := int(wire & 0x7)
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 0 {
|
||||
return code_google_com_p_gogoprotobuf_proto.ErrWrongType
|
||||
}
|
||||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
v |= (uint64(b) & 0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.Nodes = append(m.Nodes, v)
|
||||
default:
|
||||
var sizeOfWire int
|
||||
for {
|
||||
sizeOfWire++
|
||||
wire >>= 7
|
||||
if wire == 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
index -= sizeOfWire
|
||||
skippy, err := code_google_com_p_gogoprotobuf_proto.Skip(data[index:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (index + skippy) > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
|
||||
index += skippy
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *ConfChange) Unmarshal(data []byte) error {
|
||||
l := len(data)
|
||||
index := 0
|
||||
@ -861,18 +1013,25 @@ func (m *Entry) Size() (n int) {
|
||||
}
|
||||
return n
|
||||
}
|
||||
func (m *SnapshotMetadata) Size() (n int) {
|
||||
var l int
|
||||
_ = l
|
||||
l = m.ConfState.Size()
|
||||
n += 1 + l + sovRaft(uint64(l))
|
||||
n += 1 + sovRaft(uint64(m.Index))
|
||||
n += 1 + sovRaft(uint64(m.Term))
|
||||
if m.XXX_unrecognized != nil {
|
||||
n += len(m.XXX_unrecognized)
|
||||
}
|
||||
return n
|
||||
}
|
||||
func (m *Snapshot) Size() (n int) {
|
||||
var l int
|
||||
_ = l
|
||||
l = len(m.Data)
|
||||
n += 1 + l + sovRaft(uint64(l))
|
||||
if len(m.Nodes) > 0 {
|
||||
for _, e := range m.Nodes {
|
||||
n += 1 + sovRaft(uint64(e))
|
||||
}
|
||||
}
|
||||
n += 1 + sovRaft(uint64(m.Index))
|
||||
n += 1 + sovRaft(uint64(m.Term))
|
||||
l = m.Metadata.Size()
|
||||
n += 1 + l + sovRaft(uint64(l))
|
||||
if m.XXX_unrecognized != nil {
|
||||
n += len(m.XXX_unrecognized)
|
||||
}
|
||||
@ -913,6 +1072,19 @@ func (m *HardState) Size() (n int) {
|
||||
}
|
||||
return n
|
||||
}
|
||||
func (m *ConfState) Size() (n int) {
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.Nodes) > 0 {
|
||||
for _, e := range m.Nodes {
|
||||
n += 1 + sovRaft(uint64(e))
|
||||
}
|
||||
}
|
||||
if m.XXX_unrecognized != nil {
|
||||
n += len(m.XXX_unrecognized)
|
||||
}
|
||||
return n
|
||||
}
|
||||
func (m *ConfChange) Size() (n int) {
|
||||
var l int
|
||||
_ = l
|
||||
@ -973,6 +1145,40 @@ func (m *Entry) MarshalTo(data []byte) (n int, err error) {
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
func (m *SnapshotMetadata) Marshal() (data []byte, err error) {
|
||||
size := m.Size()
|
||||
data = make([]byte, size)
|
||||
n, err := m.MarshalTo(data)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return data[:n], nil
|
||||
}
|
||||
|
||||
func (m *SnapshotMetadata) MarshalTo(data []byte) (n int, err error) {
|
||||
var i int
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
data[i] = 0xa
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.ConfState.Size()))
|
||||
n1, err := m.ConfState.MarshalTo(data[i:])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i += n1
|
||||
data[i] = 0x10
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Index))
|
||||
data[i] = 0x18
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Term))
|
||||
if m.XXX_unrecognized != nil {
|
||||
i += copy(data[i:], m.XXX_unrecognized)
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
func (m *Snapshot) Marshal() (data []byte, err error) {
|
||||
size := m.Size()
|
||||
data = make([]byte, size)
|
||||
@ -992,25 +1198,14 @@ func (m *Snapshot) MarshalTo(data []byte) (n int, err error) {
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(len(m.Data)))
|
||||
i += copy(data[i:], m.Data)
|
||||
if len(m.Nodes) > 0 {
|
||||
for _, num := range m.Nodes {
|
||||
data[i] = 0x10
|
||||
i++
|
||||
for num >= 1<<7 {
|
||||
data[i] = uint8(uint64(num)&0x7f | 0x80)
|
||||
num >>= 7
|
||||
i++
|
||||
}
|
||||
data[i] = uint8(num)
|
||||
i++
|
||||
}
|
||||
data[i] = 0x12
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Metadata.Size()))
|
||||
n2, err := m.Metadata.MarshalTo(data[i:])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
data[i] = 0x18
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Index))
|
||||
data[i] = 0x20
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Term))
|
||||
i += n2
|
||||
if m.XXX_unrecognized != nil {
|
||||
i += copy(data[i:], m.XXX_unrecognized)
|
||||
}
|
||||
@ -1067,11 +1262,11 @@ func (m *Message) MarshalTo(data []byte) (n int, err error) {
|
||||
data[i] = 0x4a
|
||||
i++
|
||||
i = encodeVarintRaft(data, i, uint64(m.Snapshot.Size()))
|
||||
n1, err := m.Snapshot.MarshalTo(data[i:])
|
||||
n3, err := m.Snapshot.MarshalTo(data[i:])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i += n1
|
||||
i += n3
|
||||
data[i] = 0x50
|
||||
i++
|
||||
if m.Reject {
|
||||
@ -1114,6 +1309,39 @@ func (m *HardState) MarshalTo(data []byte) (n int, err error) {
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
func (m *ConfState) Marshal() (data []byte, err error) {
|
||||
size := m.Size()
|
||||
data = make([]byte, size)
|
||||
n, err := m.MarshalTo(data)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return data[:n], nil
|
||||
}
|
||||
|
||||
func (m *ConfState) MarshalTo(data []byte) (n int, err error) {
|
||||
var i int
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.Nodes) > 0 {
|
||||
for _, num := range m.Nodes {
|
||||
data[i] = 0x8
|
||||
i++
|
||||
for num >= 1<<7 {
|
||||
data[i] = uint8(uint64(num)&0x7f | 0x80)
|
||||
num >>= 7
|
||||
i++
|
||||
}
|
||||
data[i] = uint8(num)
|
||||
i++
|
||||
}
|
||||
}
|
||||
if m.XXX_unrecognized != nil {
|
||||
i += copy(data[i:], m.XXX_unrecognized)
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
func (m *ConfChange) Marshal() (data []byte, err error) {
|
||||
size := m.Size()
|
||||
data = make([]byte, size)
|
||||
|
@ -20,11 +20,15 @@ message Entry {
|
||||
optional bytes Data = 4 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
message SnapshotMetadata {
|
||||
required ConfState conf_state = 1 [(gogoproto.nullable) = false];
|
||||
required uint64 index = 2 [(gogoproto.nullable) = false];
|
||||
required uint64 term = 3 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
message Snapshot {
|
||||
required bytes data = 1 [(gogoproto.nullable) = false];
|
||||
repeated uint64 nodes = 2 [(gogoproto.nullable) = false];
|
||||
required uint64 index = 3 [(gogoproto.nullable) = false];
|
||||
required uint64 term = 4 [(gogoproto.nullable) = false];
|
||||
optional bytes data = 1 [(gogoproto.nullable) = false];
|
||||
required SnapshotMetadata metadata = 2 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
enum MessageType {
|
||||
@ -57,6 +61,10 @@ message HardState {
|
||||
required uint64 commit = 3 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
message ConfState {
|
||||
repeated uint64 nodes = 1 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
enum ConfChangeType {
|
||||
ConfChangeAddNode = 0;
|
||||
ConfChangeRemoveNode = 1;
|
||||
|
191
raft/storage.go
Normal file
191
raft/storage.go
Normal file
@ -0,0 +1,191 @@
|
||||
/*
|
||||
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"
|
||||
"log"
|
||||
"sync"
|
||||
|
||||
pb "github.com/coreos/etcd/raft/raftpb"
|
||||
)
|
||||
|
||||
// ErrCompacted is returned by Storage.Entries/Compact when a requested
|
||||
// index is unavailable because it predates the last snapshot.
|
||||
var ErrCompacted = errors.New("requested index is unavailable due to compaction")
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
||||
// 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 overwrites the contents of this Storage object with
|
||||
// those of the given snapshot.
|
||||
func (ms *MemoryStorage) ApplySnapshot(snap pb.Snapshot) error {
|
||||
ms.Lock()
|
||||
defer ms.Unlock()
|
||||
|
||||
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 {
|
||||
return ErrCompacted
|
||||
}
|
||||
if i > offset+uint64(len(ms.ents))-1 {
|
||||
log.Panicf("compact %d is out of bound lastindex(%d)", i, offset+uint64(len(ms.ents))-1)
|
||||
}
|
||||
i -= offset
|
||||
ents := make([]pb.Entry, 1, 1+uint64(len(ms.ents))-i)
|
||||
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
|
||||
// do not append out of date entries
|
||||
if offset < 0 {
|
||||
return
|
||||
}
|
||||
if uint64(len(ms.ents)) >= offset {
|
||||
ms.ents = ms.ents[:offset]
|
||||
}
|
||||
ms.ents = append(ms.ents, entries...)
|
||||
}
|
75
raft/util.go
Normal file
75
raft/util.go
Normal file
@ -0,0 +1,75 @@
|
||||
/*
|
||||
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"
|
||||
)
|
||||
|
||||
// uint64Slice implements sort interface
|
||||
type uint64Slice []uint64
|
||||
|
||||
func (p uint64Slice) Len() int { return len(p) }
|
||||
func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] }
|
||||
func (p uint64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
|
||||
|
||||
func min(a, b uint64) uint64 {
|
||||
if a > b {
|
||||
return b
|
||||
}
|
||||
return a
|
||||
}
|
||||
|
||||
func max(a, b uint64) uint64 {
|
||||
if a > b {
|
||||
return a
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
// DescribeMessage returns a concise human-readable description of a
|
||||
// Message for debugging.
|
||||
func DescribeMessage(m pb.Message) string {
|
||||
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))
|
||||
}
|
@ -61,7 +61,7 @@ func (s *Snapshotter) SaveSnap(snapshot raftpb.Snapshot) error {
|
||||
}
|
||||
|
||||
func (s *Snapshotter) save(snapshot *raftpb.Snapshot) error {
|
||||
fname := fmt.Sprintf("%016x-%016x%s", snapshot.Term, snapshot.Index, snapSuffix)
|
||||
fname := fmt.Sprintf("%016x-%016x%s", snapshot.Metadata.Term, snapshot.Metadata.Index, snapSuffix)
|
||||
b := pbutil.MustMarshal(snapshot)
|
||||
crc := crc32.Update(0, crcTable, b)
|
||||
snap := snappb.Snapshot{Crc: crc, Data: b}
|
||||
|
@ -29,10 +29,14 @@ import (
|
||||
)
|
||||
|
||||
var testSnap = &raftpb.Snapshot{
|
||||
Data: []byte("some snapshot"),
|
||||
Nodes: []uint64{1, 2, 3},
|
||||
Index: 1,
|
||||
Term: 1,
|
||||
Data: []byte("some snapshot"),
|
||||
Metadata: raftpb.SnapshotMetadata{
|
||||
ConfState: raftpb.ConfState{
|
||||
Nodes: []uint64{1, 2, 3},
|
||||
},
|
||||
Index: 1,
|
||||
Term: 1,
|
||||
},
|
||||
}
|
||||
|
||||
func TestSaveAndLoad(t *testing.T) {
|
||||
@ -156,7 +160,7 @@ func TestLoadNewestSnap(t *testing.T) {
|
||||
}
|
||||
|
||||
newSnap := *testSnap
|
||||
newSnap.Index = 5
|
||||
newSnap.Metadata.Index = 5
|
||||
err = ss.save(&newSnap)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
|
Loading…
x
Reference in New Issue
Block a user