etcd/raft/raft_test.go

1046 lines
25 KiB
Go
Raw Normal View History

2014-05-06 10:28:14 +04:00
package raft
import (
"bytes"
2014-07-01 23:10:43 +04:00
"fmt"
"math/rand"
2014-07-01 23:10:43 +04:00
"reflect"
"sort"
2014-05-06 10:28:14 +04:00
"testing"
)
func TestLeaderElection(t *testing.T) {
tests := []struct {
2014-05-16 21:11:21 +04:00
*network
2014-05-06 10:28:14 +04:00
state stateType
}{
{newNetwork(nil, nil, nil), stateLeader},
{newNetwork(nil, nil, nopStepper), stateLeader},
{newNetwork(nil, nopStepper, nopStepper), stateCandidate},
{newNetwork(nil, nopStepper, nopStepper, nil), stateCandidate},
{newNetwork(nil, nopStepper, nopStepper, nil, nil), stateLeader},
2014-05-19 03:11:45 +04:00
// three logs further along than 0
{newNetwork(nil, ents(1), ents(2), ents(1, 3), nil), stateFollower},
2014-05-19 05:05:03 +04:00
// logs converge
{newNetwork(ents(1), nil, ents(2), ents(1), nil), stateLeader},
2014-05-06 10:28:14 +04:00
}
for i, tt := range tests {
tt.send(Message{To: 0, Type: msgHup})
sm := tt.network.peers[0].(*stateMachine)
2014-05-06 10:28:14 +04:00
if sm.state != tt.state {
t.Errorf("#%d: state = %s, want %s", i, sm.state, tt.state)
}
if g := sm.term; g != 1 {
t.Errorf("#%d: term = %d, want %d", i, g, 1)
}
}
}
func TestLogReplication(t *testing.T) {
tests := []struct {
*network
msgs []Message
wcommitted int
}{
{
newNetwork(nil, nil, nil),
[]Message{
{To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("somedata")}}},
},
2,
},
{
newNetwork(nil, nil, nil),
[]Message{
{To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("somedata")}}},
{To: 1, Type: msgHup},
{To: 1, Type: msgProp, Entries: []Entry{{Data: []byte("somedata")}}},
},
4,
},
}
for i, tt := range tests {
tt.send(Message{To: 0, Type: msgHup})
for _, m := range tt.msgs {
tt.send(m)
}
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)
}
ents := make([]Entry, 0)
for _, e := range sm.nextEnts() {
if e.Data != nil {
ents = append(ents, e)
}
}
props := make([]Message, 0)
for _, m := range tt.msgs {
if m.Type == msgProp {
props = append(props, m)
}
}
for k, m := range props {
if !bytes.Equal(ents[k].Data, m.Entries[0].Data) {
t.Errorf("#%d.%d: data = %d, want %d", i, j, ents[k].Data, m.Entries[0].Data)
}
}
}
}
}
func TestSingleNodeCommit(t *testing.T) {
tt := newNetwork(nil)
tt.send(Message{To: 0, Type: msgHup})
tt.send(Message{To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
tt.send(Message{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)
}
}
// TestCannotCommitWithoutNewTermEntry tests the entries cannot be committed
// when leader changes, no new proposal comes in and ChangeTerm proposal is
// filtered.
2014-06-06 21:26:44 +04:00
func TestCannotCommitWithoutNewTermEntry(t *testing.T) {
tt := newNetwork(nil, nil, nil, nil, nil)
tt.send(Message{To: 0, Type: msgHup})
// 0 cannot reach 2,3,4
2014-06-06 22:01:55 +04:00
tt.cut(0, 2)
tt.cut(0, 3)
tt.cut(0, 4)
2014-06-06 21:26:44 +04:00
tt.send(Message{To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
tt.send(Message{To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
2014-06-06 21:26:44 +04:00
sm := tt.peers[0].(*stateMachine)
if sm.log.committed != 1 {
t.Errorf("committed = %d, want %d", sm.log.committed, 1)
2014-06-06 21:26:44 +04:00
}
// network recovery
tt.recover()
// avoid committing ChangeTerm proposal
tt.ignore(msgApp)
2014-06-06 21:26:44 +04:00
// elect 1 as the new leader with term 2
tt.send(Message{To: 1, Type: msgHup})
// 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)
2014-06-06 21:26:44 +04:00
}
tt.recover()
// send out a heartbeat
// after append a ChangeTerm entry from the current term, all entries
2014-06-06 21:26:44 +04:00
// should be committed
tt.send(Message{To: 1, Type: msgBeat})
if sm.log.committed != 4 {
t.Errorf("committed = %d, want %d", sm.log.committed, 4)
}
// still be able to append a entry
tt.send(Message{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)
}
}
// TestCommitWithoutNewTermEntry tests the entries could be committed
// when leader changes, no new proposal comes in.
func TestCommitWithoutNewTermEntry(t *testing.T) {
tt := newNetwork(nil, nil, nil, nil, nil)
tt.send(Message{To: 0, Type: msgHup})
// 0 cannot reach 2,3,4
tt.cut(0, 2)
tt.cut(0, 3)
tt.cut(0, 4)
tt.send(Message{To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
tt.send(Message{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)
}
// network recovery
tt.recover()
// elect 1 as the new leader with term 2
// after append a ChangeTerm entry from the current term, all entries
// should be committed
tt.send(Message{To: 1, Type: msgHup})
if sm.log.committed != 4 {
t.Errorf("committed = %d, want %d", sm.log.committed, 4)
2014-06-06 21:26:44 +04:00
}
}
func TestDuelingCandidates(t *testing.T) {
2014-06-14 01:48:36 +04:00
a := newStateMachine(0, nil) // k, id are set later
b := newStateMachine(0, nil)
2014-06-10 03:45:42 +04:00
c := newStateMachine(0, nil)
2014-05-19 00:59:10 +04:00
nt := newNetwork(a, b, c)
nt.cut(0, 2)
2014-05-19 00:59:10 +04:00
nt.send(Message{To: 0, Type: msgHup})
nt.send(Message{To: 2, Type: msgHup})
2014-05-16 21:11:21 +04:00
nt.recover()
nt.send(Message{To: 2, Type: msgHup})
2014-05-16 21:53:56 +04:00
wlog := &log{ents: []Entry{{}, Entry{Type: Normal, Data: nil, Term: 1}}, committed: 1}
2014-05-16 21:53:56 +04:00
tests := []struct {
sm *stateMachine
2014-05-16 21:53:56 +04:00
state stateType
2014-05-18 01:09:52 +04:00
term int
log *log
2014-05-16 21:53:56 +04:00
}{
{a, stateFollower, 2, wlog},
{b, stateFollower, 2, wlog},
{c, stateFollower, 2, newLog()},
2014-05-16 21:53:56 +04:00
}
for i, tt := range tests {
if g := tt.sm.state; g != tt.state {
t.Errorf("#%d: state = %s, want %s", i, g, tt.state)
}
if g := tt.sm.term; g != tt.term {
t.Errorf("#%d: term = %d, want %d", i, g, tt.term)
}
base := ltoa(tt.log)
2014-07-09 22:53:27 +04:00
if sm, ok := nt.peers[int64(i)].(*stateMachine); ok {
l := ltoa(sm.log)
if g := diffu(base, l); g != "" {
t.Errorf("#%d: diff:\n%s", i, g)
}
} else {
t.Logf("#%d: empty log", i)
2014-05-19 04:51:45 +04:00
}
}
}
func TestCandidateConcede(t *testing.T) {
tt := newNetwork(nil, nil, nil)
tt.isolate(0)
2014-05-19 04:51:45 +04:00
tt.send(Message{To: 0, Type: msgHup})
tt.send(Message{To: 2, Type: msgHup})
2014-05-19 04:51:45 +04:00
// heal the partition
tt.recover()
2014-05-19 04:51:45 +04:00
data := []byte("force follower")
// send a proposal to 2 to flush out a msgApp to 0
tt.send(Message{To: 2, Type: msgProp, Entries: []Entry{{Data: data}}})
2014-05-19 04:51:45 +04:00
a := tt.peers[0].(*stateMachine)
2014-05-19 04:51:45 +04:00
if g := a.state; g != stateFollower {
t.Errorf("state = %s, want %s", g, stateFollower)
}
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})
for i, p := range tt.peers {
if sm, ok := p.(*stateMachine); ok {
l := ltoa(sm.log)
if g := diffu(wantLog, l); g != "" {
t.Errorf("#%d: diff:\n%s", i, g)
}
} else {
t.Logf("#%d: empty log", i)
2014-05-16 21:53:56 +04:00
}
2014-05-16 21:11:21 +04:00
}
2014-05-16 21:53:56 +04:00
}
2014-05-28 22:59:01 +04:00
func TestSingleNodeCandidate(t *testing.T) {
tt := newNetwork(nil)
tt.send(Message{To: 0, Type: msgHup})
2014-05-28 22:59:01 +04:00
sm := tt.peers[0].(*stateMachine)
2014-05-28 22:59:01 +04:00
if sm.state != stateLeader {
t.Errorf("state = %d, want %d", sm.state, stateLeader)
}
}
2014-05-16 21:53:56 +04:00
func TestOldMessages(t *testing.T) {
tt := newNetwork(nil, nil, nil)
// make 0 leader @ term 3
tt.send(Message{To: 0, Type: msgHup})
tt.send(Message{To: 1, Type: msgHup})
tt.send(Message{To: 0, Type: msgHup})
2014-05-16 21:53:56 +04:00
// pretend we're an old leader trying to make progress
tt.send(Message{To: 0, Type: msgApp, Term: 1, Entries: []Entry{{Term: 1}}})
l := &log{
ents: []Entry{
{}, {Type: Normal, Data: nil, Term: 1},
{Type: Normal, Data: nil, Term: 2}, {Type: Normal, Data: nil, Term: 3},
},
committed: 3,
}
base := ltoa(l)
for i, p := range tt.peers {
if sm, ok := p.(*stateMachine); ok {
l := ltoa(sm.log)
if g := diffu(base, l); g != "" {
t.Errorf("#%d: diff:\n%s", i, g)
}
} else {
t.Logf("#%d: empty log", i)
2014-05-16 21:11:21 +04:00
}
}
}
2014-05-18 04:18:58 +04:00
// TestOldMessagesReply - optimization - reply with new term.
2014-05-06 10:28:14 +04:00
func TestProposal(t *testing.T) {
tests := []struct {
2014-05-16 21:11:21 +04:00
*network
2014-05-18 01:09:52 +04:00
success bool
2014-05-06 10:28:14 +04:00
}{
2014-05-18 01:09:52 +04:00
{newNetwork(nil, nil, nil), true},
{newNetwork(nil, nil, nopStepper), true},
{newNetwork(nil, nopStepper, nopStepper), false},
{newNetwork(nil, nopStepper, nopStepper, nil), false},
{newNetwork(nil, nopStepper, nopStepper, nil, nil), true},
2014-05-06 10:28:14 +04:00
}
for i, tt := range tests {
send := func(m Message) {
2014-05-06 10:28:14 +04:00
defer func() {
2014-05-18 01:09:52 +04:00
// only recover is we expect it to panic so
// panics we don't expect go up.
if !tt.success {
2014-05-06 10:28:14 +04:00
e := recover()
if e != nil {
t.Logf("#%d: err: %s", i, e)
}
}
}()
tt.send(m)
}
2014-05-06 10:28:14 +04:00
2014-05-18 01:09:52 +04:00
data := []byte("somedata")
2014-05-06 10:28:14 +04:00
// promote 0 the leader
send(Message{To: 0, Type: msgHup})
send(Message{To: 0, Type: msgProp, Entries: []Entry{{Data: data}}})
2014-05-06 10:28:14 +04:00
wantLog := newLog()
2014-05-18 01:09:52 +04:00
if tt.success {
wantLog = &log{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1}, {Term: 1, Data: data}}, committed: 2}
2014-05-18 01:09:52 +04:00
}
base := ltoa(wantLog)
for i, p := range tt.peers {
if sm, ok := p.(*stateMachine); ok {
l := ltoa(sm.log)
if g := diffu(base, l); g != "" {
t.Errorf("#%d: diff:\n%s", i, g)
}
} else {
t.Logf("#%d: empty log", i)
2014-05-06 10:28:14 +04:00
}
}
sm := tt.network.peers[0].(*stateMachine)
2014-05-06 10:28:14 +04:00
if g := sm.term; g != 1 {
t.Errorf("#%d: term = %d, want %d", i, g, 1)
}
}
}
func TestProposalByProxy(t *testing.T) {
2014-05-16 11:55:17 +04:00
data := []byte("somedata")
2014-05-18 01:09:52 +04:00
tests := []*network{
newNetwork(nil, nil, nil),
newNetwork(nil, nil, nopStepper),
2014-05-06 10:28:14 +04:00
}
for i, tt := range tests {
// promote 0 the leader
tt.send(Message{To: 0, Type: msgHup})
2014-05-06 10:28:14 +04:00
// propose via follower
tt.send(Message{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}
base := ltoa(wantLog)
for i, p := range tt.peers {
if sm, ok := p.(*stateMachine); ok {
l := ltoa(sm.log)
if g := diffu(base, l); g != "" {
t.Errorf("#%d: diff:\n%s", i, g)
}
} else {
t.Logf("#%d: empty log", i)
2014-05-06 10:28:14 +04:00
}
}
sm := tt.peers[0].(*stateMachine)
2014-05-06 10:28:14 +04:00
if g := sm.term; g != 1 {
t.Errorf("#%d: term = %d, want %d", i, g, 1)
}
}
}
func TestCommit(t *testing.T) {
tests := []struct {
matches []int
logs []Entry
smTerm int
w int
}{
2014-06-06 22:21:26 +04:00
// single
{[]int{1}, []Entry{{}, {Term: 1}}, 1, 1},
{[]int{1}, []Entry{{}, {Term: 1}}, 2, 0},
{[]int{2}, []Entry{{}, {Term: 1}, {Term: 2}}, 2, 2},
{[]int{1}, []Entry{{}, {Term: 2}}, 2, 1},
// odd
{[]int{2, 1, 1}, []Entry{{}, {Term: 1}, {Term: 2}}, 1, 1},
{[]int{2, 1, 1}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
{[]int{2, 1, 2}, []Entry{{}, {Term: 1}, {Term: 2}}, 2, 2},
{[]int{2, 1, 2}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
// even
{[]int{2, 1, 1, 1}, []Entry{{}, {Term: 1}, {Term: 2}}, 1, 1},
{[]int{2, 1, 1, 1}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
{[]int{2, 1, 1, 2}, []Entry{{}, {Term: 1}, {Term: 2}}, 1, 1},
{[]int{2, 1, 1, 2}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
{[]int{2, 1, 2, 2}, []Entry{{}, {Term: 1}, {Term: 2}}, 2, 2},
{[]int{2, 1, 2, 2}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
}
for i, tt := range tests {
2014-07-09 22:53:27 +04:00
ins := make(map[int64]*index)
2014-06-10 03:45:42 +04:00
for j := 0; j < len(tt.matches); j++ {
2014-07-09 22:53:27 +04:00
ins[int64(j)] = &index{tt.matches[j], tt.matches[j] + 1}
}
2014-06-10 03:45:42 +04:00
sm := &stateMachine{log: &log{ents: tt.logs}, ins: ins, term: tt.smTerm}
sm.maybeCommit()
if g := sm.log.committed; g != tt.w {
t.Errorf("#%d: committed = %d, want %d", i, g, tt.w)
}
}
}
2014-06-11 19:58:26 +04:00
func TestRecvMsgVote(t *testing.T) {
2014-05-18 11:17:46 +04:00
tests := []struct {
state stateType
2014-05-18 11:17:46 +04:00
i, term int
2014-07-09 22:53:27 +04:00
voteFor int64
2014-05-18 11:17:46 +04:00
w int
}{
{stateFollower, 0, 0, none, -1},
{stateFollower, 0, 1, none, -1},
{stateFollower, 0, 2, none, -1},
{stateFollower, 0, 3, none, 2},
{stateFollower, 1, 0, none, -1},
{stateFollower, 1, 1, none, -1},
{stateFollower, 1, 2, none, -1},
{stateFollower, 1, 3, none, 2},
{stateFollower, 2, 0, none, -1},
{stateFollower, 2, 1, none, -1},
{stateFollower, 2, 2, none, 2},
{stateFollower, 2, 3, none, 2},
{stateFollower, 3, 0, none, -1},
{stateFollower, 3, 1, none, -1},
{stateFollower, 3, 2, none, 2},
{stateFollower, 3, 3, none, 2},
{stateFollower, 3, 2, 1, 2},
{stateFollower, 3, 2, 0, -1},
{stateLeader, 3, 3, 0, -1},
{stateCandidate, 3, 3, 0, -1},
2014-05-18 11:17:46 +04:00
}
for i, tt := range tests {
sm := &stateMachine{
state: tt.state,
vote: tt.voteFor,
log: &log{ents: []Entry{{}, {Term: 2}, {Term: 2}}},
}
2014-05-25 08:08:06 +04:00
sm.Step(Message{Type: msgVote, From: 1, Index: tt.i, LogTerm: tt.term})
2014-06-11 19:58:26 +04:00
msgs := sm.Msgs()
2014-06-17 22:19:56 +04:00
if g := len(msgs); g != 1 {
t.Errorf("#%d: len(msgs) = %d, want 1", i, g)
2014-06-11 19:58:26 +04:00
continue
}
2014-06-17 22:19:56 +04:00
if g := msgs[0].Index; g != tt.w {
2014-06-11 19:58:26 +04:00
t.Errorf("#%d, m.Index = %d, want %d", i, g, tt.w)
2014-05-18 11:17:46 +04:00
}
}
}
2014-06-07 03:27:29 +04:00
func TestStateTransition(t *testing.T) {
tests := []struct {
from stateType
to stateType
wallow bool
wterm int
2014-07-09 22:53:27 +04:00
wlead int64
2014-06-07 03:27:29 +04:00
}{
{stateFollower, stateFollower, true, 1, none},
{stateFollower, stateCandidate, true, 1, none},
{stateFollower, stateLeader, false, -1, none},
{stateCandidate, stateFollower, true, 0, none},
{stateCandidate, stateCandidate, true, 1, none},
{stateCandidate, stateLeader, true, 0, 0},
{stateLeader, stateFollower, true, 1, none},
{stateLeader, stateCandidate, false, 1, none},
{stateLeader, stateLeader, true, 0, 0},
}
for i, tt := range tests {
func() {
defer func() {
if r := recover(); r != nil {
if tt.wallow == true {
t.Errorf("%d: allow = %v, want %v", i, false, true)
}
}
}()
2014-07-09 22:53:27 +04:00
sm := newStateMachine(0, []int64{0})
2014-06-07 03:27:29 +04:00
sm.state = tt.from
switch tt.to {
case stateFollower:
sm.becomeFollower(tt.wterm, tt.wlead)
case stateCandidate:
sm.becomeCandidate()
case stateLeader:
sm.becomeLeader()
}
if sm.term != tt.wterm {
t.Errorf("%d: term = %d, want %d", i, sm.term, tt.wterm)
}
2014-07-11 09:12:55 +04:00
if sm.lead.Get() != tt.wlead {
2014-06-07 03:27:29 +04:00
t.Errorf("%d: lead = %d, want %d", i, sm.lead, tt.wlead)
}
}()
}
}
func TestConf(t *testing.T) {
2014-07-09 22:53:27 +04:00
sm := newStateMachine(0, []int64{0})
sm.becomeCandidate()
sm.becomeLeader()
sm.Step(Message{Type: msgProp, Entries: []Entry{{Type: AddNode}}})
if sm.log.lastIndex() != 2 {
t.Errorf("lastindex = %d, want %d", sm.log.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)
}
// deny the second configuration change request if there is a pending one
sm.Step(Message{Type: msgProp, Entries: []Entry{{Type: AddNode}}})
if sm.log.lastIndex() != 2 {
t.Errorf("lastindex = %d, want %d", sm.log.lastIndex(), 1)
}
}
// Ensures that the new leader sets the pendingConf flag correctly according to
// the uncommitted log entries
func TestConfChangeLeader(t *testing.T) {
tests := []struct {
et int
wPending bool
}{
2014-06-20 01:08:35 +04:00
{Normal, false},
{AddNode, true},
{RemoveNode, true},
}
for i, tt := range tests {
2014-07-09 22:53:27 +04:00
sm := newStateMachine(0, []int64{0})
sm.log = &log{ents: []Entry{{}, {Type: tt.et}}}
sm.becomeCandidate()
sm.becomeLeader()
if sm.pendingConf != tt.wPending {
t.Errorf("#%d: pendingConf = %v, want %v", i, sm.pendingConf, tt.wPending)
}
}
}
2014-05-24 00:30:04 +04:00
func TestAllServerStepdown(t *testing.T) {
tests := []struct {
2014-05-24 00:30:04 +04:00
state stateType
wstate stateType
wterm int
windex int
}{
{stateFollower, stateFollower, 3, 1},
{stateCandidate, stateFollower, 3, 1},
{stateLeader, stateFollower, 3, 2},
}
2014-05-24 00:30:04 +04:00
tmsgTypes := [...]messageType{msgVote, msgApp}
tterm := 3
for i, tt := range tests {
2014-07-09 22:53:27 +04:00
sm := newStateMachine(0, []int64{0, 1, 2})
switch tt.state {
2014-05-24 00:30:04 +04:00
case stateFollower:
sm.becomeFollower(1, 0)
case stateCandidate:
sm.becomeCandidate()
case stateLeader:
sm.becomeCandidate()
sm.becomeLeader()
}
for j, msgType := range tmsgTypes {
sm.Step(Message{Type: msgType, Term: tterm, LogTerm: tterm})
if sm.state != tt.wstate {
t.Errorf("#%d.%d state = %v , want %v", i, j, sm.state, tt.wstate)
2014-05-24 00:30:04 +04:00
}
if sm.term != tt.wterm {
t.Errorf("#%d.%d term = %v , want %v", i, j, sm.term, tt.wterm)
2014-05-24 00:30:04 +04:00
}
if len(sm.log.ents) != tt.windex {
t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.log.ents), tt.windex)
2014-05-24 00:30:04 +04:00
}
}
}
}
2014-05-25 10:03:13 +04:00
func TestLeaderAppResp(t *testing.T) {
tests := []struct {
index int
wmsgNum int
windex int
wcommitted int
}{
{-1, 1, 1, 0}, // bad resp; leader does not commit; reply with log entries
{2, 2, 2, 2}, // good resp; leader commits; broadcast with commit index
}
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.
2014-07-09 22:53:27 +04:00
sm := newStateMachine(0, []int64{0, 1, 2})
2014-06-10 03:45:42 +04:00
sm.log = &log{ents: []Entry{{}, {Term: 0}, {Term: 1}}}
2014-05-25 10:03:13 +04:00
sm.becomeCandidate()
sm.becomeLeader()
sm.Msgs()
2014-05-25 10:03:13 +04:00
sm.Step(Message{From: 1, Type: msgAppResp, Index: tt.index, Term: sm.term})
msgs := sm.Msgs()
if len(msgs) != tt.wmsgNum {
t.Errorf("#%d msgNum = %d, want %d", i, len(msgs), tt.wmsgNum)
}
for j, msg := range msgs {
if msg.Index != tt.windex {
t.Errorf("#%d.%d index = %d, want %d", i, j, msg.Index, tt.windex)
}
if msg.Commit != tt.wcommitted {
t.Errorf("#%d.%d commit = %d, want %d", i, j, msg.Commit, tt.wcommitted)
}
}
}
}
2014-06-07 00:40:24 +04:00
// tests the output of the statemachine when receiving msgBeat
func TestRecvMsgBeat(t *testing.T) {
tests := []struct {
state stateType
wMsg int
}{
{stateLeader, 2},
// candidate and follower should ignore msgBeat
{stateCandidate, 0},
{stateFollower, 0},
}
for i, tt := range tests {
2014-07-09 22:53:27 +04:00
sm := newStateMachine(0, []int64{0, 1, 2})
2014-06-07 00:40:24 +04:00
sm.log = &log{ents: []Entry{{}, {Term: 0}, {Term: 1}}}
sm.term = 1
sm.state = tt.state
sm.Step(Message{Type: msgBeat})
msgs := sm.Msgs()
if len(msgs) != tt.wMsg {
t.Errorf("%d: len(msgs) = %d, want %d", i, len(msgs), tt.wMsg)
}
for _, m := range msgs {
if m.Type != msgApp {
t.Errorf("%d: msg.type = %v, want %v", m.Type, msgApp)
}
}
}
}
2014-07-01 23:10:43 +04:00
func TestMaybeCompact(t *testing.T) {
tests := []struct {
snapshoter Snapshoter
applied int
wCompact bool
}{
{nil, defaultCompactThreshold + 1, false},
{new(logSnapshoter), defaultCompactThreshold - 1, false},
{new(logSnapshoter), defaultCompactThreshold + 1, true},
}
for i, tt := range tests {
2014-07-09 22:53:27 +04:00
sm := newStateMachine(0, []int64{0, 1, 2})
2014-07-01 23:10:43 +04:00
sm.setSnapshoter(tt.snapshoter)
for i := 0; i < defaultCompactThreshold*2; i++ {
sm.log.append(i, Entry{Term: i + 1})
}
sm.log.applied = tt.applied
sm.log.committed = tt.applied
if g := sm.maybeCompact(); g != tt.wCompact {
t.Errorf("#%d: compact = %v, want %v", i, g, tt.wCompact)
}
if tt.wCompact {
s := sm.snapshoter.GetSnap()
if s.Index != tt.applied {
t.Errorf("#%d: snap.Index = %v, want %v", i, s.Index, tt.applied)
}
if s.Term != tt.applied {
t.Errorf("#%d: snap.Term = %v, want %v", i, s.Index, tt.applied)
}
w := sm.nodes()
2014-07-09 22:53:27 +04:00
sw := int64Slice(w)
sg := int64Slice(s.Nodes)
sort.Sort(sw)
sort.Sort(sg)
if !reflect.DeepEqual(sg, sw) {
t.Errorf("#%d: snap.Nodes = %+v, want %+v", i, sg, sw)
2014-07-01 23:10:43 +04:00
}
}
}
}
func TestRestore(t *testing.T) {
s := Snapshot{
Index: defaultCompactThreshold + 1,
Term: defaultCompactThreshold + 1,
2014-07-09 22:53:27 +04:00
Nodes: []int64{0, 1, 2},
2014-07-01 23:10:43 +04:00
}
tests := []struct {
snapshoter Snapshoter
wallow bool
}{
{nil, false},
{new(logSnapshoter), true},
}
for i, tt := range tests {
func() {
defer func() {
if r := recover(); r != nil {
if tt.wallow == true {
t.Errorf("%d: allow = %v, want %v", i, false, true)
}
}
}()
2014-07-09 22:53:27 +04:00
sm := newStateMachine(0, []int64{0, 1})
2014-07-01 23:10:43 +04:00
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.log.term(s.Index) != s.Term {
t.Errorf("#%d: log.lastTerm = %d, want %d", i, sm.log.term(s.Index), s.Term)
}
2014-07-09 22:53:27 +04:00
sg := int64Slice(sm.nodes())
sw := int64Slice(s.Nodes)
sort.Sort(sg)
sort.Sort(sw)
if !reflect.DeepEqual(sg, sw) {
t.Errorf("#%d: sm.Nodes = %+v, want %+v", i, sg, sw)
2014-07-01 23:10:43 +04:00
}
if !reflect.DeepEqual(sm.snapshoter.GetSnap(), s) {
t.Errorf("%d: snapshoter.getSnap = %+v, want %+v", sm.snapshoter.GetSnap(), s)
}
}()
}
}
2014-07-02 23:49:58 +04:00
func TestProvideSnap(t *testing.T) {
s := Snapshot{
Index: defaultCompactThreshold + 1,
Term: defaultCompactThreshold + 1,
2014-07-09 22:53:27 +04:00
Nodes: []int64{0, 1},
2014-07-02 23:49:58 +04:00
}
2014-07-09 22:53:27 +04:00
sm := newStateMachine(0, []int64{0})
2014-07-02 23:49:58 +04:00
sm.setSnapshoter(new(logSnapshoter))
// restore the statemachin from a snapshot
// so it has a compacted log and a snapshot
sm.restore(s)
sm.becomeCandidate()
sm.becomeLeader()
sm.Step(Message{Type: msgBeat})
msgs := sm.Msgs()
if len(msgs) != 1 {
t.Errorf("len(msgs) = %d, want 1", len(msgs))
}
m := msgs[0]
if m.Type != msgApp {
t.Errorf("m.Type = %v, want %v", m.Type, msgApp)
}
// force set the next of node 1, so that
// node 1 needs a snapshot
sm.ins[1].next = sm.log.offset
sm.Step(Message{Type: msgBeat})
msgs = sm.Msgs()
if len(msgs) != 1 {
t.Errorf("len(msgs) = %d, want 1", len(msgs))
}
m = msgs[0]
if m.Type != msgSnap {
t.Errorf("m.Type = %v, want %v", m.Type, msgSnap)
}
}
func TestRestoreFromSnapMsg(t *testing.T) {
s := Snapshot{
Index: defaultCompactThreshold + 1,
Term: defaultCompactThreshold + 1,
2014-07-09 22:53:27 +04:00
Nodes: []int64{0, 1},
2014-07-02 23:49:58 +04:00
}
m := Message{Type: msgSnap, From: 0, Term: 1, Snapshot: s}
2014-07-09 22:53:27 +04:00
sm := newStateMachine(1, []int64{0, 1})
2014-07-02 23:49:58 +04:00
sm.setSnapshoter(new(logSnapshoter))
sm.Step(m)
if !reflect.DeepEqual(sm.snapshoter.GetSnap(), s) {
t.Errorf("snapshot = %+v, want %+v", sm.snapshoter.GetSnap(), s)
}
}
func TestSlowNodeRestore(t *testing.T) {
nt := newNetwork(nil, nil, nil)
nt.send(Message{To: 0, Type: msgHup})
nt.isolate(2)
for j := 0; j < defaultCompactThreshold+1; j++ {
nt.send(Message{To: 0, Type: msgProp, Entries: []Entry{{}}})
}
lead := nt.peers[0].(*stateMachine)
lead.nextEnts()
if !lead.maybeCompact() {
t.Errorf("compacted = false, want true")
}
nt.recover()
nt.send(Message{To: 0, Type: msgBeat})
follower := nt.peers[2].(*stateMachine)
if !reflect.DeepEqual(follower.snapshoter.GetSnap(), lead.snapshoter.GetSnap()) {
t.Errorf("follower.snap = %+v, want %+v", follower.snapshoter.GetSnap(), lead.snapshoter.GetSnap())
}
committed := follower.log.lastIndex()
nt.send(Message{To: 0, Type: msgProp, Entries: []Entry{{}}})
if follower.log.committed != committed+1 {
t.Errorf("follower.comitted = %d, want %d", follower.log.committed, committed+1)
}
}
func ents(terms ...int) *stateMachine {
ents := []Entry{{}}
for _, term := range terms {
ents = append(ents, Entry{Term: term})
2014-05-18 01:09:52 +04:00
}
sm := &stateMachine{log: &log{ents: ents}}
2014-06-16 05:52:10 +04:00
sm.reset(0)
return sm
2014-05-18 01:09:52 +04:00
}
2014-05-16 21:11:21 +04:00
type network struct {
2014-07-09 22:53:27 +04:00
peers map[int64]Interface
dropm map[connem]float64
ignorem map[messageType]bool
2014-05-16 21:11:21 +04:00
}
2014-05-06 10:28:14 +04:00
// newNetwork initializes a network from peers.
// A nil node will be replaced with a new *stateMachine.
// A *stateMachine will get its k, id.
// When using stateMachine, the address list is always [0, n).
func newNetwork(peers ...Interface) *network {
size := len(peers)
2014-07-09 22:53:27 +04:00
defaultPeerAddrs := make([]int64, size)
for i := 0; i < size; i++ {
2014-07-09 22:53:27 +04:00
defaultPeerAddrs[i] = int64(i)
2014-06-10 03:45:42 +04:00
}
2014-07-09 22:53:27 +04:00
npeers := make(map[int64]Interface, size)
2014-06-14 01:48:36 +04:00
for id, p := range peers {
2014-07-09 22:53:27 +04:00
nid := int64(id)
switch v := p.(type) {
2014-05-06 10:28:14 +04:00
case nil:
2014-07-09 22:53:27 +04:00
sm := newStateMachine(nid, defaultPeerAddrs)
2014-07-02 23:49:58 +04:00
sm.setSnapshoter(new(logSnapshoter))
2014-07-09 22:53:27 +04:00
npeers[nid] = sm
case *stateMachine:
2014-07-09 22:53:27 +04:00
v.id = nid
v.ins = make(map[int64]*index)
for i := 0; i < size; i++ {
2014-07-09 22:53:27 +04:00
v.ins[int64(i)] = &index{}
2014-06-10 03:45:42 +04:00
}
2014-06-16 05:52:10 +04:00
v.reset(0)
2014-07-09 22:53:27 +04:00
npeers[nid] = v
case *Node:
npeers[v.sm.id] = v
default:
2014-07-09 22:53:27 +04:00
npeers[nid] = v
2014-05-06 10:28:14 +04:00
}
}
return &network{
peers: npeers,
dropm: make(map[connem]float64),
ignorem: make(map[messageType]bool),
}
2014-05-06 10:28:14 +04:00
}
func (nw *network) send(msgs ...Message) {
for len(msgs) > 0 {
m := msgs[0]
p := nw.peers[m.To]
p.Step(m)
msgs = append(msgs[1:], nw.filter(p.Msgs())...)
2014-05-16 21:11:21 +04:00
}
}
2014-07-09 22:53:27 +04:00
func (nw *network) drop(from, to int64, perc float64) {
nw.dropm[connem{from, to}] = perc
}
2014-07-09 22:53:27 +04:00
func (nw *network) cut(one, other int64) {
2014-06-06 22:01:55 +04:00
nw.drop(one, other, 1)
nw.drop(other, one, 1)
}
2014-07-09 22:53:27 +04:00
func (nw *network) isolate(id int64) {
for i := 0; i < len(nw.peers); i++ {
2014-07-09 22:53:27 +04:00
nid := int64(i)
if nid != id {
nw.drop(id, nid, 1.0)
nw.drop(nid, id, 1.0)
2014-05-06 10:28:14 +04:00
}
}
}
func (nw *network) ignore(t messageType) {
nw.ignorem[t] = true
}
func (nw *network) recover() {
nw.dropm = make(map[connem]float64)
nw.ignorem = make(map[messageType]bool)
2014-05-06 10:28:14 +04:00
}
func (nw *network) filter(msgs []Message) []Message {
mm := make([]Message, 0)
for _, m := range msgs {
if nw.ignorem[m.Type] {
continue
}
switch m.Type {
case msgHup:
// hups never go over the network, so don't drop them but panic
panic("unexpected msgHup")
2014-05-06 10:28:14 +04:00
default:
perc := nw.dropm[connem{m.From, m.To}]
if n := rand.Float64(); n < perc {
2014-05-06 10:28:14 +04:00
continue
}
}
mm = append(mm, m)
2014-05-06 10:28:14 +04:00
}
return mm
2014-05-06 10:28:14 +04:00
}
type connem struct {
2014-07-09 22:53:27 +04:00
from, to int64
2014-05-24 00:30:04 +04:00
}
type blackHole struct{}
2014-05-27 23:24:49 +04:00
2014-06-14 03:07:27 +04:00
func (blackHole) Step(Message) bool { return true }
func (blackHole) Msgs() []Message { return nil }
var nopStepper = &blackHole{}
2014-07-01 23:10:43 +04:00
type logSnapshoter struct {
snapshot Snapshot
}
2014-07-09 22:53:27 +04:00
func (s *logSnapshoter) Snap(index, term int, nodes []int64) {
2014-07-01 23:10:43 +04:00
s.snapshot = Snapshot{
Index: index,
Term: term,
Nodes: nodes,
Data: []byte(fmt.Sprintf("%d:%d", term, index)),
}
}
func (s *logSnapshoter) Restore(ss Snapshot) {
s.snapshot = ss
}
func (s *logSnapshoter) GetSnap() Snapshot {
return s.snapshot
}
2014-07-09 22:53:27 +04:00
// int64Slice implements sort interface
type int64Slice []int64
func (p int64Slice) Len() int { return len(p) }
func (p int64Slice) Less(i, j int) bool { return p[i] < p[j] }
func (p int64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] }