|
|
@@ -74,8 +74,8 @@ func TestLogReplication(t *testing.T) {
|
|
|
for j, x := range tt.network.peers {
|
|
|
sm := x.(*stateMachine)
|
|
|
|
|
|
- if sm.log.committed != tt.wcommitted {
|
|
|
- t.Errorf("#%d.%d: committed = %d, want %d", i, j, sm.log.committed, tt.wcommitted)
|
|
|
+ if sm.raftLog.committed != tt.wcommitted {
|
|
|
+ t.Errorf("#%d.%d: committed = %d, want %d", i, j, sm.raftLog.committed, tt.wcommitted)
|
|
|
}
|
|
|
|
|
|
ents := make([]Entry, 0)
|
|
|
@@ -106,8 +106,8 @@ func TestSingleNodeCommit(t *testing.T) {
|
|
|
tt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
|
|
|
|
|
|
sm := tt.peers[0].(*stateMachine)
|
|
|
- if sm.log.committed != 3 {
|
|
|
- t.Errorf("committed = %d, want %d", sm.log.committed, 3)
|
|
|
+ if sm.raftLog.committed != 3 {
|
|
|
+ t.Errorf("committed = %d, want %d", sm.raftLog.committed, 3)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -127,8 +127,8 @@ func TestCannotCommitWithoutNewTermEntry(t *testing.T) {
|
|
|
tt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
|
|
|
|
|
|
sm := tt.peers[0].(*stateMachine)
|
|
|
- if sm.log.committed != 1 {
|
|
|
- t.Errorf("committed = %d, want %d", sm.log.committed, 1)
|
|
|
+ if sm.raftLog.committed != 1 {
|
|
|
+ t.Errorf("committed = %d, want %d", sm.raftLog.committed, 1)
|
|
|
}
|
|
|
|
|
|
// network recovery
|
|
|
@@ -141,8 +141,8 @@ func TestCannotCommitWithoutNewTermEntry(t *testing.T) {
|
|
|
|
|
|
// no log entries from previous term should be committed
|
|
|
sm = tt.peers[1].(*stateMachine)
|
|
|
- if sm.log.committed != 1 {
|
|
|
- t.Errorf("committed = %d, want %d", sm.log.committed, 1)
|
|
|
+ if sm.raftLog.committed != 1 {
|
|
|
+ t.Errorf("committed = %d, want %d", sm.raftLog.committed, 1)
|
|
|
}
|
|
|
|
|
|
tt.recover()
|
|
|
@@ -152,15 +152,15 @@ func TestCannotCommitWithoutNewTermEntry(t *testing.T) {
|
|
|
// should be committed
|
|
|
tt.send(Message{From: 1, To: 1, Type: msgBeat})
|
|
|
|
|
|
- if sm.log.committed != 4 {
|
|
|
- t.Errorf("committed = %d, want %d", sm.log.committed, 4)
|
|
|
+ if sm.raftLog.committed != 4 {
|
|
|
+ t.Errorf("committed = %d, want %d", sm.raftLog.committed, 4)
|
|
|
}
|
|
|
|
|
|
// still be able to append a entry
|
|
|
tt.send(Message{From: 1, To: 1, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
|
|
|
|
|
|
- if sm.log.committed != 5 {
|
|
|
- t.Errorf("committed = %d, want %d", sm.log.committed, 5)
|
|
|
+ if sm.raftLog.committed != 5 {
|
|
|
+ t.Errorf("committed = %d, want %d", sm.raftLog.committed, 5)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -179,8 +179,8 @@ func TestCommitWithoutNewTermEntry(t *testing.T) {
|
|
|
tt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
|
|
|
|
|
|
sm := tt.peers[0].(*stateMachine)
|
|
|
- if sm.log.committed != 1 {
|
|
|
- t.Errorf("committed = %d, want %d", sm.log.committed, 1)
|
|
|
+ if sm.raftLog.committed != 1 {
|
|
|
+ t.Errorf("committed = %d, want %d", sm.raftLog.committed, 1)
|
|
|
}
|
|
|
|
|
|
// network recovery
|
|
|
@@ -191,8 +191,8 @@ func TestCommitWithoutNewTermEntry(t *testing.T) {
|
|
|
// should be committed
|
|
|
tt.send(Message{From: 1, To: 1, Type: msgHup})
|
|
|
|
|
|
- if sm.log.committed != 4 {
|
|
|
- t.Errorf("committed = %d, want %d", sm.log.committed, 4)
|
|
|
+ if sm.raftLog.committed != 4 {
|
|
|
+ t.Errorf("committed = %d, want %d", sm.raftLog.committed, 4)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -210,12 +210,12 @@ func TestDuelingCandidates(t *testing.T) {
|
|
|
nt.recover()
|
|
|
nt.send(Message{From: 2, To: 2, Type: msgHup})
|
|
|
|
|
|
- wlog := &log{ents: []Entry{{}, Entry{Type: Normal, Data: nil, Term: 1}}, committed: 1}
|
|
|
+ wlog := &raftLog{ents: []Entry{{}, Entry{Type: Normal, Data: nil, Term: 1}}, committed: 1}
|
|
|
tests := []struct {
|
|
|
- sm *stateMachine
|
|
|
- state stateType
|
|
|
- term int64
|
|
|
- log *log
|
|
|
+ sm *stateMachine
|
|
|
+ state stateType
|
|
|
+ term int64
|
|
|
+ raftLog *raftLog
|
|
|
}{
|
|
|
{a, stateFollower, 2, wlog},
|
|
|
{b, stateFollower, 2, wlog},
|
|
|
@@ -229,9 +229,9 @@ func TestDuelingCandidates(t *testing.T) {
|
|
|
if g := tt.sm.term.Get(); g != tt.term {
|
|
|
t.Errorf("#%d: term = %d, want %d", i, g, tt.term)
|
|
|
}
|
|
|
- base := ltoa(tt.log)
|
|
|
+ base := ltoa(tt.raftLog)
|
|
|
if sm, ok := nt.peers[int64(i)].(*stateMachine); ok {
|
|
|
- l := ltoa(sm.log)
|
|
|
+ l := ltoa(sm.raftLog)
|
|
|
if g := diffu(base, l); g != "" {
|
|
|
t.Errorf("#%d: diff:\n%s", i, g)
|
|
|
}
|
|
|
@@ -262,10 +262,10 @@ func TestCandidateConcede(t *testing.T) {
|
|
|
if g := a.term; g != 1 {
|
|
|
t.Errorf("term = %d, want %d", g, 1)
|
|
|
}
|
|
|
- wantLog := ltoa(&log{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1}, {Term: 1, Data: data}}, committed: 2})
|
|
|
+ wantLog := ltoa(&raftLog{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1}, {Term: 1, Data: data}}, committed: 2})
|
|
|
for i, p := range tt.peers {
|
|
|
if sm, ok := p.(*stateMachine); ok {
|
|
|
- l := ltoa(sm.log)
|
|
|
+ l := ltoa(sm.raftLog)
|
|
|
if g := diffu(wantLog, l); g != "" {
|
|
|
t.Errorf("#%d: diff:\n%s", i, g)
|
|
|
}
|
|
|
@@ -294,7 +294,7 @@ func TestOldMessages(t *testing.T) {
|
|
|
// pretend we're an old leader trying to make progress
|
|
|
tt.send(Message{From: 0, To: 0, Type: msgApp, Term: 1, Entries: []Entry{{Term: 1}}})
|
|
|
|
|
|
- l := &log{
|
|
|
+ l := &raftLog{
|
|
|
ents: []Entry{
|
|
|
{}, {Type: Normal, Data: nil, Term: 1},
|
|
|
{Type: Normal, Data: nil, Term: 2}, {Type: Normal, Data: nil, Term: 3},
|
|
|
@@ -304,7 +304,7 @@ func TestOldMessages(t *testing.T) {
|
|
|
base := ltoa(l)
|
|
|
for i, p := range tt.peers {
|
|
|
if sm, ok := p.(*stateMachine); ok {
|
|
|
- l := ltoa(sm.log)
|
|
|
+ l := ltoa(sm.raftLog)
|
|
|
if g := diffu(base, l); g != "" {
|
|
|
t.Errorf("#%d: diff:\n%s", i, g)
|
|
|
}
|
|
|
@@ -351,12 +351,12 @@ func TestProposal(t *testing.T) {
|
|
|
|
|
|
wantLog := newLog()
|
|
|
if tt.success {
|
|
|
- wantLog = &log{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1}, {Term: 1, Data: data}}, committed: 2}
|
|
|
+ wantLog = &raftLog{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1}, {Term: 1, Data: data}}, committed: 2}
|
|
|
}
|
|
|
base := ltoa(wantLog)
|
|
|
for i, p := range tt.peers {
|
|
|
if sm, ok := p.(*stateMachine); ok {
|
|
|
- l := ltoa(sm.log)
|
|
|
+ l := ltoa(sm.raftLog)
|
|
|
if g := diffu(base, l); g != "" {
|
|
|
t.Errorf("#%d: diff:\n%s", i, g)
|
|
|
}
|
|
|
@@ -385,11 +385,11 @@ func TestProposalByProxy(t *testing.T) {
|
|
|
// propose via follower
|
|
|
tt.send(Message{From: 1, To: 1, Type: msgProp, Entries: []Entry{{Data: []byte("somedata")}}})
|
|
|
|
|
|
- wantLog := &log{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1}, {Term: 1, Data: data}}, committed: 2}
|
|
|
+ wantLog := &raftLog{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1}, {Term: 1, Data: data}}, committed: 2}
|
|
|
base := ltoa(wantLog)
|
|
|
for i, p := range tt.peers {
|
|
|
if sm, ok := p.(*stateMachine); ok {
|
|
|
- l := ltoa(sm.log)
|
|
|
+ l := ltoa(sm.raftLog)
|
|
|
if g := diffu(base, l); g != "" {
|
|
|
t.Errorf("#%d: diff:\n%s", i, g)
|
|
|
}
|
|
|
@@ -437,9 +437,9 @@ func TestCommit(t *testing.T) {
|
|
|
for j := 0; j < len(tt.matches); j++ {
|
|
|
ins[int64(j)] = &index{tt.matches[j], tt.matches[j] + 1}
|
|
|
}
|
|
|
- sm := &stateMachine{log: &log{ents: tt.logs}, ins: ins, term: atomicInt(tt.smTerm)}
|
|
|
+ sm := &stateMachine{raftLog: &raftLog{ents: tt.logs}, ins: ins, term: atomicInt(tt.smTerm)}
|
|
|
sm.maybeCommit()
|
|
|
- if g := sm.log.committed; g != tt.w {
|
|
|
+ if g := sm.raftLog.committed; g != tt.w {
|
|
|
t.Errorf("#%d: committed = %d, want %d", i, g, tt.w)
|
|
|
}
|
|
|
}
|
|
|
@@ -475,17 +475,17 @@ func TestHandleMsgApp(t *testing.T) {
|
|
|
|
|
|
for i, tt := range tests {
|
|
|
sm := &stateMachine{
|
|
|
- state: stateFollower,
|
|
|
- term: 2,
|
|
|
- log: &log{committed: 0, ents: []Entry{{}, {Term: 1}, {Term: 2}}},
|
|
|
+ state: stateFollower,
|
|
|
+ term: 2,
|
|
|
+ raftLog: &raftLog{committed: 0, ents: []Entry{{}, {Term: 1}, {Term: 2}}},
|
|
|
}
|
|
|
|
|
|
sm.handleAppendEntries(tt.m)
|
|
|
- if sm.log.lastIndex() != tt.wIndex {
|
|
|
- t.Errorf("#%d: lastIndex = %d, want %d", i, sm.log.lastIndex(), tt.wIndex)
|
|
|
+ if sm.raftLog.lastIndex() != tt.wIndex {
|
|
|
+ t.Errorf("#%d: lastIndex = %d, want %d", i, sm.raftLog.lastIndex(), tt.wIndex)
|
|
|
}
|
|
|
- if sm.log.committed != tt.wCommit {
|
|
|
- t.Errorf("#%d: committed = %d, want %d", i, sm.log.committed, tt.wCommit)
|
|
|
+ if sm.raftLog.committed != tt.wCommit {
|
|
|
+ t.Errorf("#%d: committed = %d, want %d", i, sm.raftLog.committed, tt.wCommit)
|
|
|
}
|
|
|
m := sm.Msgs()
|
|
|
if len(m) != 1 {
|
|
|
@@ -537,9 +537,9 @@ func TestRecvMsgVote(t *testing.T) {
|
|
|
|
|
|
for i, tt := range tests {
|
|
|
sm := &stateMachine{
|
|
|
- state: tt.state,
|
|
|
- vote: tt.voteFor,
|
|
|
- log: &log{ents: []Entry{{}, {Term: 2}, {Term: 2}}},
|
|
|
+ state: tt.state,
|
|
|
+ vote: tt.voteFor,
|
|
|
+ raftLog: &raftLog{ents: []Entry{{}, {Term: 2}, {Term: 2}}},
|
|
|
}
|
|
|
|
|
|
sm.Step(Message{Type: msgVote, From: 1, Index: tt.i, LogTerm: tt.term})
|
|
|
@@ -614,20 +614,20 @@ func TestConf(t *testing.T) {
|
|
|
sm.becomeLeader()
|
|
|
|
|
|
sm.Step(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Type: AddNode}}})
|
|
|
- if sm.log.lastIndex() != 2 {
|
|
|
- t.Errorf("lastindex = %d, want %d", sm.log.lastIndex(), 1)
|
|
|
+ if sm.raftLog.lastIndex() != 2 {
|
|
|
+ t.Errorf("lastindex = %d, want %d", sm.raftLog.lastIndex(), 1)
|
|
|
}
|
|
|
if !sm.pendingConf {
|
|
|
t.Errorf("pendingConf = %v, want %v", sm.pendingConf, true)
|
|
|
}
|
|
|
- if sm.log.ents[2].Type != AddNode {
|
|
|
- t.Errorf("type = %d, want %d", sm.log.ents[1].Type, AddNode)
|
|
|
+ if sm.raftLog.ents[2].Type != AddNode {
|
|
|
+ t.Errorf("type = %d, want %d", sm.raftLog.ents[1].Type, AddNode)
|
|
|
}
|
|
|
|
|
|
// deny the second configuration change request if there is a pending one
|
|
|
sm.Step(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Type: AddNode}}})
|
|
|
- if sm.log.lastIndex() != 2 {
|
|
|
- t.Errorf("lastindex = %d, want %d", sm.log.lastIndex(), 1)
|
|
|
+ if sm.raftLog.lastIndex() != 2 {
|
|
|
+ t.Errorf("lastindex = %d, want %d", sm.raftLog.lastIndex(), 1)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -645,7 +645,7 @@ func TestConfChangeLeader(t *testing.T) {
|
|
|
|
|
|
for i, tt := range tests {
|
|
|
sm := newStateMachine(0, []int64{0})
|
|
|
- sm.log = &log{ents: []Entry{{}, {Type: tt.et}}}
|
|
|
+ sm.raftLog = &raftLog{ents: []Entry{{}, {Type: tt.et}}}
|
|
|
|
|
|
sm.becomeCandidate()
|
|
|
sm.becomeLeader()
|
|
|
@@ -693,8 +693,8 @@ func TestAllServerStepdown(t *testing.T) {
|
|
|
if sm.term.Get() != tt.wterm {
|
|
|
t.Errorf("#%d.%d term = %v , want %v", i, j, sm.term.Get(), tt.wterm)
|
|
|
}
|
|
|
- if int64(len(sm.log.ents)) != tt.windex {
|
|
|
- t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.log.ents), tt.windex)
|
|
|
+ if int64(len(sm.raftLog.ents)) != tt.windex {
|
|
|
+ t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.ents), tt.windex)
|
|
|
}
|
|
|
wlead := int64(1)
|
|
|
if msgType == msgVote {
|
|
|
@@ -722,7 +722,7 @@ func TestLeaderAppResp(t *testing.T) {
|
|
|
// sm term is 1 after it becomes the leader.
|
|
|
// thus the last log term must be 1 to be committed.
|
|
|
sm := newStateMachine(0, []int64{0, 1, 2})
|
|
|
- sm.log = &log{ents: []Entry{{}, {Term: 0}, {Term: 1}}}
|
|
|
+ sm.raftLog = &raftLog{ents: []Entry{{}, {Term: 0}, {Term: 1}}}
|
|
|
sm.becomeCandidate()
|
|
|
sm.becomeLeader()
|
|
|
sm.Msgs()
|
|
|
@@ -757,7 +757,7 @@ func TestRecvMsgBeat(t *testing.T) {
|
|
|
|
|
|
for i, tt := range tests {
|
|
|
sm := newStateMachine(0, []int64{0, 1, 2})
|
|
|
- sm.log = &log{ents: []Entry{{}, {Term: 0}, {Term: 1}}}
|
|
|
+ sm.raftLog = &raftLog{ents: []Entry{{}, {Term: 0}, {Term: 1}}}
|
|
|
sm.term.Set(1)
|
|
|
sm.state = tt.state
|
|
|
sm.Step(Message{From: 0, To: 0, Type: msgBeat})
|
|
|
@@ -789,10 +789,10 @@ func TestMaybeCompact(t *testing.T) {
|
|
|
sm := newStateMachine(0, []int64{0, 1, 2})
|
|
|
sm.setSnapshoter(tt.snapshoter)
|
|
|
for i := 0; i < defaultCompactThreshold*2; i++ {
|
|
|
- sm.log.append(int64(i), Entry{Term: int64(i + 1)})
|
|
|
+ sm.raftLog.append(int64(i), Entry{Term: int64(i + 1)})
|
|
|
}
|
|
|
- sm.log.applied = tt.applied
|
|
|
- sm.log.committed = tt.applied
|
|
|
+ sm.raftLog.applied = tt.applied
|
|
|
+ sm.raftLog.committed = tt.applied
|
|
|
|
|
|
if g := sm.maybeCompact(); g != tt.wCompact {
|
|
|
t.Errorf("#%d: compact = %v, want %v", i, g, tt.wCompact)
|
|
|
@@ -848,11 +848,11 @@ func TestRestore(t *testing.T) {
|
|
|
sm.setSnapshoter(tt.snapshoter)
|
|
|
sm.restore(s)
|
|
|
|
|
|
- if sm.log.lastIndex() != s.Index {
|
|
|
- t.Errorf("#%d: log.lastIndex = %d, want %d", i, sm.log.lastIndex(), s.Index)
|
|
|
+ if sm.raftLog.lastIndex() != s.Index {
|
|
|
+ t.Errorf("#%d: log.lastIndex = %d, want %d", i, sm.raftLog.lastIndex(), s.Index)
|
|
|
}
|
|
|
- if sm.log.term(s.Index) != s.Term {
|
|
|
- t.Errorf("#%d: log.lastTerm = %d, want %d", i, sm.log.term(s.Index), s.Term)
|
|
|
+ if sm.raftLog.term(s.Index) != s.Term {
|
|
|
+ t.Errorf("#%d: log.lastTerm = %d, want %d", i, sm.raftLog.term(s.Index), s.Term)
|
|
|
}
|
|
|
sg := int64Slice(sm.nodes())
|
|
|
sw := int64Slice(s.Nodes)
|
|
|
@@ -895,7 +895,7 @@ func TestProvideSnap(t *testing.T) {
|
|
|
|
|
|
// force set the next of node 1, so that
|
|
|
// node 1 needs a snapshot
|
|
|
- sm.ins[1].next = sm.log.offset
|
|
|
+ sm.ins[1].next = sm.raftLog.offset
|
|
|
|
|
|
sm.Step(Message{From: 1, To: 0, Type: msgAppResp, Index: -1})
|
|
|
msgs = sm.Msgs()
|
|
|
@@ -947,10 +947,10 @@ func TestSlowNodeRestore(t *testing.T) {
|
|
|
t.Errorf("follower.snap = %+v, want %+v", follower.snapshoter.GetSnap(), lead.snapshoter.GetSnap())
|
|
|
}
|
|
|
|
|
|
- committed := follower.log.lastIndex()
|
|
|
+ committed := follower.raftLog.lastIndex()
|
|
|
nt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{}}})
|
|
|
- if follower.log.committed != committed+1 {
|
|
|
- t.Errorf("follower.comitted = %d, want %d", follower.log.committed, committed+1)
|
|
|
+ if follower.raftLog.committed != committed+1 {
|
|
|
+ t.Errorf("follower.comitted = %d, want %d", follower.raftLog.committed, committed+1)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -960,7 +960,7 @@ func ents(terms ...int64) *stateMachine {
|
|
|
ents = append(ents, Entry{Term: term})
|
|
|
}
|
|
|
|
|
|
- sm := &stateMachine{log: &log{ents: ents}}
|
|
|
+ sm := &stateMachine{raftLog: &raftLog{ents: ents}}
|
|
|
sm.reset(0)
|
|
|
return sm
|
|
|
}
|