raft_test.go 26 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023
  1. package raft
  2. import (
  3. "bytes"
  4. "math/rand"
  5. "reflect"
  6. "sort"
  7. "testing"
  8. )
  9. type Interface interface {
  10. Step(m Message) error
  11. ReadMessages() []Message
  12. }
  13. func TestLeaderElection(t *testing.T) {
  14. tests := []struct {
  15. *network
  16. state stateType
  17. }{
  18. {newNetwork(nil, nil, nil), stateLeader},
  19. {newNetwork(nil, nil, nopStepper), stateLeader},
  20. {newNetwork(nil, nopStepper, nopStepper), stateCandidate},
  21. {newNetwork(nil, nopStepper, nopStepper, nil), stateCandidate},
  22. {newNetwork(nil, nopStepper, nopStepper, nil, nil), stateLeader},
  23. // three logs further along than 0
  24. {newNetwork(nil, ents(1), ents(2), ents(1, 3), nil), stateFollower},
  25. // logs converge
  26. {newNetwork(ents(1), nil, ents(2), ents(1), nil), stateLeader},
  27. }
  28. for i, tt := range tests {
  29. tt.send(Message{From: 0, To: 0, Type: msgHup})
  30. sm := tt.network.peers[0].(*raft)
  31. if sm.state != tt.state {
  32. t.Errorf("#%d: state = %s, want %s", i, sm.state, tt.state)
  33. }
  34. if g := sm.Term; g != 1 {
  35. t.Errorf("#%d: term = %d, want %d", i, g, 1)
  36. }
  37. }
  38. }
  39. func TestLogReplication(t *testing.T) {
  40. tests := []struct {
  41. *network
  42. msgs []Message
  43. wcommitted int64
  44. }{
  45. {
  46. newNetwork(nil, nil, nil),
  47. []Message{
  48. {From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("somedata")}}},
  49. },
  50. 2,
  51. },
  52. {
  53. newNetwork(nil, nil, nil),
  54. []Message{
  55. {From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("somedata")}}},
  56. {From: 0, To: 1, Type: msgHup},
  57. {From: 0, To: 1, Type: msgProp, Entries: []Entry{{Data: []byte("somedata")}}},
  58. },
  59. 4,
  60. },
  61. }
  62. for i, tt := range tests {
  63. tt.send(Message{From: 0, To: 0, Type: msgHup})
  64. for _, m := range tt.msgs {
  65. tt.send(m)
  66. }
  67. for j, x := range tt.network.peers {
  68. sm := x.(*raft)
  69. if sm.raftLog.committed != tt.wcommitted {
  70. t.Errorf("#%d.%d: committed = %d, want %d", i, j, sm.raftLog.committed, tt.wcommitted)
  71. }
  72. ents := make([]Entry, 0)
  73. for _, e := range sm.nextEnts() {
  74. if e.Data != nil {
  75. ents = append(ents, e)
  76. }
  77. }
  78. props := make([]Message, 0)
  79. for _, m := range tt.msgs {
  80. if m.Type == msgProp {
  81. props = append(props, m)
  82. }
  83. }
  84. for k, m := range props {
  85. if !bytes.Equal(ents[k].Data, m.Entries[0].Data) {
  86. t.Errorf("#%d.%d: data = %d, want %d", i, j, ents[k].Data, m.Entries[0].Data)
  87. }
  88. }
  89. }
  90. }
  91. }
  92. func TestSingleNodeCommit(t *testing.T) {
  93. tt := newNetwork(nil)
  94. tt.send(Message{From: 0, To: 0, Type: msgHup})
  95. tt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
  96. tt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
  97. sm := tt.peers[0].(*raft)
  98. if sm.raftLog.committed != 3 {
  99. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 3)
  100. }
  101. }
  102. // TestCannotCommitWithoutNewTermEntry tests the entries cannot be committed
  103. // when leader changes, no new proposal comes in and ChangeTerm proposal is
  104. // filtered.
  105. func TestCannotCommitWithoutNewTermEntry(t *testing.T) {
  106. tt := newNetwork(nil, nil, nil, nil, nil)
  107. tt.send(Message{From: 0, To: 0, Type: msgHup})
  108. // 0 cannot reach 2,3,4
  109. tt.cut(0, 2)
  110. tt.cut(0, 3)
  111. tt.cut(0, 4)
  112. tt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
  113. tt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
  114. sm := tt.peers[0].(*raft)
  115. if sm.raftLog.committed != 1 {
  116. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 1)
  117. }
  118. // network recovery
  119. tt.recover()
  120. // avoid committing ChangeTerm proposal
  121. tt.ignore(msgApp)
  122. // elect 1 as the new leader with term 2
  123. tt.send(Message{From: 1, To: 1, Type: msgHup})
  124. // no log entries from previous term should be committed
  125. sm = tt.peers[1].(*raft)
  126. if sm.raftLog.committed != 1 {
  127. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 1)
  128. }
  129. tt.recover()
  130. // send out a heartbeat
  131. // after append a ChangeTerm entry from the current term, all entries
  132. // should be committed
  133. tt.send(Message{From: 1, To: 1, Type: msgBeat})
  134. if sm.raftLog.committed != 4 {
  135. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 4)
  136. }
  137. // still be able to append a entry
  138. tt.send(Message{From: 1, To: 1, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
  139. if sm.raftLog.committed != 5 {
  140. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 5)
  141. }
  142. }
  143. // TestCommitWithoutNewTermEntry tests the entries could be committed
  144. // when leader changes, no new proposal comes in.
  145. func TestCommitWithoutNewTermEntry(t *testing.T) {
  146. tt := newNetwork(nil, nil, nil, nil, nil)
  147. tt.send(Message{From: 0, To: 0, Type: msgHup})
  148. // 0 cannot reach 2,3,4
  149. tt.cut(0, 2)
  150. tt.cut(0, 3)
  151. tt.cut(0, 4)
  152. tt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
  153. tt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: []byte("some data")}}})
  154. sm := tt.peers[0].(*raft)
  155. if sm.raftLog.committed != 1 {
  156. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 1)
  157. }
  158. // network recovery
  159. tt.recover()
  160. // elect 1 as the new leader with term 2
  161. // after append a ChangeTerm entry from the current term, all entries
  162. // should be committed
  163. tt.send(Message{From: 1, To: 1, Type: msgHup})
  164. if sm.raftLog.committed != 4 {
  165. t.Errorf("committed = %d, want %d", sm.raftLog.committed, 4)
  166. }
  167. }
  168. func TestDuelingCandidates(t *testing.T) {
  169. a := newRaft(0, nil) // k, id are set later
  170. b := newRaft(0, nil)
  171. c := newRaft(0, nil)
  172. nt := newNetwork(a, b, c)
  173. nt.cut(0, 2)
  174. nt.send(Message{From: 0, To: 0, Type: msgHup})
  175. nt.send(Message{From: 2, To: 2, Type: msgHup})
  176. nt.recover()
  177. nt.send(Message{From: 2, To: 2, Type: msgHup})
  178. wlog := &raftLog{ents: []Entry{{}, Entry{Type: Normal, Data: nil, Term: 1, Index: 1}}, committed: 1}
  179. tests := []struct {
  180. sm *raft
  181. state stateType
  182. term int64
  183. raftLog *raftLog
  184. }{
  185. {a, stateFollower, 2, wlog},
  186. {b, stateFollower, 2, wlog},
  187. {c, stateFollower, 2, newLog()},
  188. }
  189. for i, tt := range tests {
  190. if g := tt.sm.state; g != tt.state {
  191. t.Errorf("#%d: state = %s, want %s", i, g, tt.state)
  192. }
  193. if g := tt.sm.Term; g != tt.term {
  194. t.Errorf("#%d: term = %d, want %d", i, g, tt.term)
  195. }
  196. base := ltoa(tt.raftLog)
  197. if sm, ok := nt.peers[int64(i)].(*raft); ok {
  198. l := ltoa(sm.raftLog)
  199. if g := diffu(base, l); g != "" {
  200. t.Errorf("#%d: diff:\n%s", i, g)
  201. }
  202. } else {
  203. t.Logf("#%d: empty log", i)
  204. }
  205. }
  206. }
  207. func TestCandidateConcede(t *testing.T) {
  208. tt := newNetwork(nil, nil, nil)
  209. tt.isolate(0)
  210. tt.send(Message{From: 0, To: 0, Type: msgHup})
  211. tt.send(Message{From: 2, To: 2, Type: msgHup})
  212. // heal the partition
  213. tt.recover()
  214. data := []byte("force follower")
  215. // send a proposal to 2 to flush out a msgApp to 0
  216. tt.send(Message{From: 2, To: 2, Type: msgProp, Entries: []Entry{{Data: data}}})
  217. a := tt.peers[0].(*raft)
  218. if g := a.state; g != stateFollower {
  219. t.Errorf("state = %s, want %s", g, stateFollower)
  220. }
  221. if g := a.Term; g != 1 {
  222. t.Errorf("term = %d, want %d", g, 1)
  223. }
  224. wantLog := ltoa(&raftLog{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}}, committed: 2})
  225. for i, p := range tt.peers {
  226. if sm, ok := p.(*raft); ok {
  227. l := ltoa(sm.raftLog)
  228. if g := diffu(wantLog, l); g != "" {
  229. t.Errorf("#%d: diff:\n%s", i, g)
  230. }
  231. } else {
  232. t.Logf("#%d: empty log", i)
  233. }
  234. }
  235. }
  236. func TestSingleNodeCandidate(t *testing.T) {
  237. tt := newNetwork(nil)
  238. tt.send(Message{From: 0, To: 0, Type: msgHup})
  239. sm := tt.peers[0].(*raft)
  240. if sm.state != stateLeader {
  241. t.Errorf("state = %d, want %d", sm.state, stateLeader)
  242. }
  243. }
  244. func TestOldMessages(t *testing.T) {
  245. tt := newNetwork(nil, nil, nil)
  246. // make 0 leader @ term 3
  247. tt.send(Message{From: 0, To: 0, Type: msgHup})
  248. tt.send(Message{From: 1, To: 1, Type: msgHup})
  249. tt.send(Message{From: 0, To: 0, Type: msgHup})
  250. // pretend we're an old leader trying to make progress
  251. tt.send(Message{From: 0, To: 0, Type: msgApp, Term: 1, Entries: []Entry{{Term: 1}}})
  252. l := &raftLog{
  253. ents: []Entry{
  254. {}, {Type: Normal, Data: nil, Term: 1, Index: 1},
  255. {Type: Normal, Data: nil, Term: 2, Index: 2}, {Type: Normal, Data: nil, Term: 3, Index: 3},
  256. },
  257. committed: 3,
  258. }
  259. base := ltoa(l)
  260. for i, p := range tt.peers {
  261. if sm, ok := p.(*raft); ok {
  262. l := ltoa(sm.raftLog)
  263. if g := diffu(base, l); g != "" {
  264. t.Errorf("#%d: diff:\n%s", i, g)
  265. }
  266. } else {
  267. t.Logf("#%d: empty log", i)
  268. }
  269. }
  270. }
  271. // TestOldMessagesReply - optimization - reply with new term.
  272. func TestProposal(t *testing.T) {
  273. tests := []struct {
  274. *network
  275. success bool
  276. }{
  277. {newNetwork(nil, nil, nil), true},
  278. {newNetwork(nil, nil, nopStepper), true},
  279. {newNetwork(nil, nopStepper, nopStepper), false},
  280. {newNetwork(nil, nopStepper, nopStepper, nil), false},
  281. {newNetwork(nil, nopStepper, nopStepper, nil, nil), true},
  282. }
  283. for i, tt := range tests {
  284. send := func(m Message) {
  285. defer func() {
  286. // only recover is we expect it to panic so
  287. // panics we don't expect go up.
  288. if !tt.success {
  289. e := recover()
  290. if e != nil {
  291. t.Logf("#%d: err: %s", i, e)
  292. }
  293. }
  294. }()
  295. tt.send(m)
  296. }
  297. data := []byte("somedata")
  298. // promote 0 the leader
  299. send(Message{From: 0, To: 0, Type: msgHup})
  300. send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Data: data}}})
  301. wantLog := newLog()
  302. if tt.success {
  303. wantLog = &raftLog{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}}, committed: 2}
  304. }
  305. base := ltoa(wantLog)
  306. for i, p := range tt.peers {
  307. if sm, ok := p.(*raft); ok {
  308. l := ltoa(sm.raftLog)
  309. if g := diffu(base, l); g != "" {
  310. t.Errorf("#%d: diff:\n%s", i, g)
  311. }
  312. } else {
  313. t.Logf("#%d: empty log", i)
  314. }
  315. }
  316. sm := tt.network.peers[0].(*raft)
  317. if g := sm.Term; g != 1 {
  318. t.Errorf("#%d: term = %d, want %d", i, g, 1)
  319. }
  320. }
  321. }
  322. func TestProposalByProxy(t *testing.T) {
  323. data := []byte("somedata")
  324. tests := []*network{
  325. newNetwork(nil, nil, nil),
  326. newNetwork(nil, nil, nopStepper),
  327. }
  328. for i, tt := range tests {
  329. // promote 0 the leader
  330. tt.send(Message{From: 0, To: 0, Type: msgHup})
  331. // propose via follower
  332. tt.send(Message{From: 1, To: 1, Type: msgProp, Entries: []Entry{{Data: []byte("somedata")}}})
  333. wantLog := &raftLog{ents: []Entry{{}, {Type: Normal, Data: nil, Term: 1, Index: 1}, {Term: 1, Data: data, Index: 2}}, committed: 2}
  334. base := ltoa(wantLog)
  335. for i, p := range tt.peers {
  336. if sm, ok := p.(*raft); ok {
  337. l := ltoa(sm.raftLog)
  338. if g := diffu(base, l); g != "" {
  339. t.Errorf("#%d: diff:\n%s", i, g)
  340. }
  341. } else {
  342. t.Logf("#%d: empty log", i)
  343. }
  344. }
  345. sm := tt.peers[0].(*raft)
  346. if g := sm.Term; g != 1 {
  347. t.Errorf("#%d: term = %d, want %d", i, g, 1)
  348. }
  349. }
  350. }
  351. func TestCommit(t *testing.T) {
  352. tests := []struct {
  353. matches []int64
  354. logs []Entry
  355. smTerm int64
  356. w int64
  357. }{
  358. // single
  359. {[]int64{1}, []Entry{{}, {Term: 1}}, 1, 1},
  360. {[]int64{1}, []Entry{{}, {Term: 1}}, 2, 0},
  361. {[]int64{2}, []Entry{{}, {Term: 1}, {Term: 2}}, 2, 2},
  362. {[]int64{1}, []Entry{{}, {Term: 2}}, 2, 1},
  363. // odd
  364. {[]int64{2, 1, 1}, []Entry{{}, {Term: 1}, {Term: 2}}, 1, 1},
  365. {[]int64{2, 1, 1}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
  366. {[]int64{2, 1, 2}, []Entry{{}, {Term: 1}, {Term: 2}}, 2, 2},
  367. {[]int64{2, 1, 2}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
  368. // even
  369. {[]int64{2, 1, 1, 1}, []Entry{{}, {Term: 1}, {Term: 2}}, 1, 1},
  370. {[]int64{2, 1, 1, 1}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
  371. {[]int64{2, 1, 1, 2}, []Entry{{}, {Term: 1}, {Term: 2}}, 1, 1},
  372. {[]int64{2, 1, 1, 2}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
  373. {[]int64{2, 1, 2, 2}, []Entry{{}, {Term: 1}, {Term: 2}}, 2, 2},
  374. {[]int64{2, 1, 2, 2}, []Entry{{}, {Term: 1}, {Term: 1}}, 2, 0},
  375. }
  376. for i, tt := range tests {
  377. prs := make(map[int64]*progress)
  378. for j := 0; j < len(tt.matches); j++ {
  379. prs[int64(j)] = &progress{tt.matches[j], tt.matches[j] + 1}
  380. }
  381. sm := &raft{raftLog: &raftLog{ents: tt.logs}, prs: prs, State: State{Term: tt.smTerm}}
  382. sm.maybeCommit()
  383. if g := sm.raftLog.committed; g != tt.w {
  384. t.Errorf("#%d: committed = %d, want %d", i, g, tt.w)
  385. }
  386. }
  387. }
  388. // TestHandleMsgApp ensures:
  389. // 1. Reply false if log doesn’t contain an entry at prevLogIndex whose term matches prevLogTerm.
  390. // 2. If an existing entry conflicts with a new one (same index but different terms),
  391. // delete the existing entry and all that follow it; append any new entries not already in the log.
  392. // 3. If leaderCommit > commitIndex, set commitIndex = min(leaderCommit, index of last new entry).
  393. func TestHandleMsgApp(t *testing.T) {
  394. tests := []struct {
  395. m Message
  396. wIndex int64
  397. wCommit int64
  398. wAccept bool
  399. }{
  400. // Ensure 1
  401. {Message{Type: msgApp, Term: 2, LogTerm: 3, Index: 2, Commit: 3}, 2, 0, false}, // previous log mismatch
  402. {Message{Type: msgApp, Term: 2, LogTerm: 3, Index: 3, Commit: 3}, 2, 0, false}, // previous log non-exist
  403. // Ensure 2
  404. {Message{Type: msgApp, Term: 2, LogTerm: 1, Index: 1, Commit: 1}, 2, 1, true},
  405. {Message{Type: msgApp, Term: 2, LogTerm: 0, Index: 0, Commit: 1, Entries: []Entry{{Term: 2}}}, 1, 1, true},
  406. {Message{Type: msgApp, Term: 2, LogTerm: 2, Index: 2, Commit: 3, Entries: []Entry{{Term: 2}, {Term: 2}}}, 4, 3, true},
  407. {Message{Type: msgApp, Term: 2, LogTerm: 2, Index: 2, Commit: 4, Entries: []Entry{{Term: 2}}}, 3, 3, true},
  408. {Message{Type: msgApp, Term: 2, LogTerm: 1, Index: 1, Commit: 4, Entries: []Entry{{Term: 2}}}, 2, 2, true},
  409. // Ensure 3
  410. {Message{Type: msgApp, Term: 2, LogTerm: 2, Index: 2, Commit: 2}, 2, 2, true},
  411. {Message{Type: msgApp, Term: 2, LogTerm: 2, Index: 2, Commit: 4}, 2, 2, true}, // commit upto min(commit, last)
  412. }
  413. for i, tt := range tests {
  414. sm := &raft{
  415. state: stateFollower,
  416. State: State{Term: 2},
  417. raftLog: &raftLog{committed: 0, ents: []Entry{{}, {Term: 1}, {Term: 2}}},
  418. }
  419. sm.handleAppendEntries(tt.m)
  420. if sm.raftLog.lastIndex() != tt.wIndex {
  421. t.Errorf("#%d: lastIndex = %d, want %d", i, sm.raftLog.lastIndex(), tt.wIndex)
  422. }
  423. if sm.raftLog.committed != tt.wCommit {
  424. t.Errorf("#%d: committed = %d, want %d", i, sm.raftLog.committed, tt.wCommit)
  425. }
  426. m := sm.ReadMessages()
  427. if len(m) != 1 {
  428. t.Errorf("#%d: msg = nil, want 1")
  429. }
  430. gaccept := true
  431. if m[0].Index == -1 {
  432. gaccept = false
  433. }
  434. if gaccept != tt.wAccept {
  435. t.Errorf("#%d: accept = %v, want %v", gaccept, tt.wAccept)
  436. }
  437. }
  438. }
  439. func TestRecvMsgVote(t *testing.T) {
  440. tests := []struct {
  441. state stateType
  442. i, term int64
  443. voteFor int64
  444. w int64
  445. }{
  446. {stateFollower, 0, 0, none, -1},
  447. {stateFollower, 0, 1, none, -1},
  448. {stateFollower, 0, 2, none, -1},
  449. {stateFollower, 0, 3, none, 2},
  450. {stateFollower, 1, 0, none, -1},
  451. {stateFollower, 1, 1, none, -1},
  452. {stateFollower, 1, 2, none, -1},
  453. {stateFollower, 1, 3, none, 2},
  454. {stateFollower, 2, 0, none, -1},
  455. {stateFollower, 2, 1, none, -1},
  456. {stateFollower, 2, 2, none, 2},
  457. {stateFollower, 2, 3, none, 2},
  458. {stateFollower, 3, 0, none, -1},
  459. {stateFollower, 3, 1, none, -1},
  460. {stateFollower, 3, 2, none, 2},
  461. {stateFollower, 3, 3, none, 2},
  462. {stateFollower, 3, 2, 1, 2},
  463. {stateFollower, 3, 2, 0, -1},
  464. {stateLeader, 3, 3, 0, -1},
  465. {stateCandidate, 3, 3, 0, -1},
  466. }
  467. for i, tt := range tests {
  468. sm := &raft{
  469. state: tt.state,
  470. State: State{Vote: tt.voteFor},
  471. raftLog: &raftLog{ents: []Entry{{}, {Term: 2}, {Term: 2}}},
  472. }
  473. sm.Step(Message{Type: msgVote, From: 1, Index: tt.i, LogTerm: tt.term})
  474. msgs := sm.ReadMessages()
  475. if g := len(msgs); g != 1 {
  476. t.Errorf("#%d: len(msgs) = %d, want 1", i, g)
  477. continue
  478. }
  479. if g := msgs[0].Index; g != tt.w {
  480. t.Errorf("#%d, m.Index = %d, want %d", i, g, tt.w)
  481. }
  482. }
  483. }
  484. func TestStateTransition(t *testing.T) {
  485. tests := []struct {
  486. from stateType
  487. to stateType
  488. wallow bool
  489. wterm int64
  490. wlead int64
  491. }{
  492. {stateFollower, stateFollower, true, 1, none},
  493. {stateFollower, stateCandidate, true, 1, none},
  494. {stateFollower, stateLeader, false, -1, none},
  495. {stateCandidate, stateFollower, true, 0, none},
  496. {stateCandidate, stateCandidate, true, 1, none},
  497. {stateCandidate, stateLeader, true, 0, 0},
  498. {stateLeader, stateFollower, true, 1, none},
  499. {stateLeader, stateCandidate, false, 1, none},
  500. {stateLeader, stateLeader, true, 0, 0},
  501. }
  502. for i, tt := range tests {
  503. func() {
  504. defer func() {
  505. if r := recover(); r != nil {
  506. if tt.wallow == true {
  507. t.Errorf("%d: allow = %v, want %v", i, false, true)
  508. }
  509. }
  510. }()
  511. sm := newRaft(0, []int64{0})
  512. sm.state = tt.from
  513. switch tt.to {
  514. case stateFollower:
  515. sm.becomeFollower(tt.wterm, tt.wlead)
  516. case stateCandidate:
  517. sm.becomeCandidate()
  518. case stateLeader:
  519. sm.becomeLeader()
  520. }
  521. if sm.Term != tt.wterm {
  522. t.Errorf("%d: term = %d, want %d", i, sm.Term, tt.wterm)
  523. }
  524. if sm.lead.Get() != tt.wlead {
  525. t.Errorf("%d: lead = %d, want %d", i, sm.lead, tt.wlead)
  526. }
  527. }()
  528. }
  529. }
  530. func TestConf(t *testing.T) {
  531. sm := newRaft(0, []int64{0})
  532. sm.becomeCandidate()
  533. sm.becomeLeader()
  534. sm.Step(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Type: AddNode}}})
  535. if sm.raftLog.lastIndex() != 2 {
  536. t.Errorf("lastindex = %d, want %d", sm.raftLog.lastIndex(), 1)
  537. }
  538. if !sm.pendingConf {
  539. t.Errorf("pendingConf = %v, want %v", sm.pendingConf, true)
  540. }
  541. if sm.raftLog.ents[2].Type != AddNode {
  542. t.Errorf("type = %d, want %d", sm.raftLog.ents[1].Type, AddNode)
  543. }
  544. // deny the second configuration change request if there is a pending one
  545. sm.Step(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{Type: AddNode}}})
  546. if sm.raftLog.lastIndex() != 2 {
  547. t.Errorf("lastindex = %d, want %d", sm.raftLog.lastIndex(), 1)
  548. }
  549. }
  550. // Ensures that the new leader sets the pendingConf flag correctly according to
  551. // the uncommitted log entries
  552. func TestConfChangeLeader(t *testing.T) {
  553. tests := []struct {
  554. et int64
  555. wPending bool
  556. }{
  557. {Normal, false},
  558. {AddNode, true},
  559. {RemoveNode, true},
  560. }
  561. for i, tt := range tests {
  562. sm := newRaft(0, []int64{0})
  563. sm.raftLog = &raftLog{ents: []Entry{{}, {Type: tt.et}}}
  564. sm.becomeCandidate()
  565. sm.becomeLeader()
  566. if sm.pendingConf != tt.wPending {
  567. t.Errorf("#%d: pendingConf = %v, want %v", i, sm.pendingConf, tt.wPending)
  568. }
  569. }
  570. }
  571. func TestAllServerStepdown(t *testing.T) {
  572. tests := []struct {
  573. state stateType
  574. wstate stateType
  575. wterm int64
  576. windex int64
  577. }{
  578. {stateFollower, stateFollower, 3, 1},
  579. {stateCandidate, stateFollower, 3, 1},
  580. {stateLeader, stateFollower, 3, 2},
  581. }
  582. tmsgTypes := [...]messageType{msgVote, msgApp}
  583. tterm := int64(3)
  584. for i, tt := range tests {
  585. sm := newRaft(0, []int64{0, 1, 2})
  586. switch tt.state {
  587. case stateFollower:
  588. sm.becomeFollower(1, 0)
  589. case stateCandidate:
  590. sm.becomeCandidate()
  591. case stateLeader:
  592. sm.becomeCandidate()
  593. sm.becomeLeader()
  594. }
  595. for j, msgType := range tmsgTypes {
  596. sm.Step(Message{From: 1, Type: msgType, Term: tterm, LogTerm: tterm})
  597. if sm.state != tt.wstate {
  598. t.Errorf("#%d.%d state = %v , want %v", i, j, sm.state, tt.wstate)
  599. }
  600. if sm.Term != tt.wterm {
  601. t.Errorf("#%d.%d term = %v , want %v", i, j, sm.Term, tt.wterm)
  602. }
  603. if int64(len(sm.raftLog.ents)) != tt.windex {
  604. t.Errorf("#%d.%d index = %v , want %v", i, j, len(sm.raftLog.ents), tt.windex)
  605. }
  606. wlead := int64(1)
  607. if msgType == msgVote {
  608. wlead = none
  609. }
  610. if sm.lead.Get() != wlead {
  611. t.Errorf("#%d, sm.lead = %d, want %d", i, sm.lead.Get(), none)
  612. }
  613. }
  614. }
  615. }
  616. func TestLeaderAppResp(t *testing.T) {
  617. tests := []struct {
  618. index int64
  619. wmsgNum int
  620. windex int64
  621. wcommitted int64
  622. }{
  623. {-1, 1, 1, 0}, // bad resp; leader does not commit; reply with log entries
  624. {2, 2, 2, 2}, // good resp; leader commits; broadcast with commit index
  625. }
  626. for i, tt := range tests {
  627. // sm term is 1 after it becomes the leader.
  628. // thus the last log term must be 1 to be committed.
  629. sm := newRaft(0, []int64{0, 1, 2})
  630. sm.raftLog = &raftLog{ents: []Entry{{}, {Term: 0}, {Term: 1}}}
  631. sm.becomeCandidate()
  632. sm.becomeLeader()
  633. sm.ReadMessages()
  634. sm.Step(Message{From: 1, Type: msgAppResp, Index: tt.index, Term: sm.Term})
  635. msgs := sm.ReadMessages()
  636. if len(msgs) != tt.wmsgNum {
  637. t.Errorf("#%d msgNum = %d, want %d", i, len(msgs), tt.wmsgNum)
  638. }
  639. for j, msg := range msgs {
  640. if msg.Index != tt.windex {
  641. t.Errorf("#%d.%d index = %d, want %d", i, j, msg.Index, tt.windex)
  642. }
  643. if msg.Commit != tt.wcommitted {
  644. t.Errorf("#%d.%d commit = %d, want %d", i, j, msg.Commit, tt.wcommitted)
  645. }
  646. }
  647. }
  648. }
  649. // tests the output of the statemachine when receiving msgBeat
  650. func TestRecvMsgBeat(t *testing.T) {
  651. tests := []struct {
  652. state stateType
  653. wMsg int
  654. }{
  655. {stateLeader, 2},
  656. // candidate and follower should ignore msgBeat
  657. {stateCandidate, 0},
  658. {stateFollower, 0},
  659. }
  660. for i, tt := range tests {
  661. sm := newRaft(0, []int64{0, 1, 2})
  662. sm.raftLog = &raftLog{ents: []Entry{{}, {Term: 0}, {Term: 1}}}
  663. sm.Term = 1
  664. sm.state = tt.state
  665. sm.Step(Message{From: 0, To: 0, Type: msgBeat})
  666. msgs := sm.ReadMessages()
  667. if len(msgs) != tt.wMsg {
  668. t.Errorf("%d: len(msgs) = %d, want %d", i, len(msgs), tt.wMsg)
  669. }
  670. for _, m := range msgs {
  671. if m.Type != msgApp {
  672. t.Errorf("%d: msg.type = %v, want %v", m.Type, msgApp)
  673. }
  674. }
  675. }
  676. }
  677. func TestRestore(t *testing.T) {
  678. s := Snapshot{
  679. ClusterId: 0xBEEF,
  680. Index: defaultCompactThreshold + 1,
  681. Term: defaultCompactThreshold + 1,
  682. Nodes: []int64{0, 1, 2},
  683. }
  684. sm := newRaft(0, []int64{0, 1})
  685. if ok := sm.restore(s); !ok {
  686. t.Fatal("restore fail, want succeed")
  687. }
  688. if sm.clusterId != s.ClusterId {
  689. t.Errorf("sm.cluster = %x, want %x", sm.clusterId, s.ClusterId)
  690. }
  691. if sm.raftLog.lastIndex() != s.Index {
  692. t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Index)
  693. }
  694. if sm.raftLog.term(s.Index) != s.Term {
  695. t.Errorf("log.lastTerm = %d, want %d", sm.raftLog.term(s.Index), s.Term)
  696. }
  697. sg := int64Slice(sm.nodes())
  698. sw := int64Slice(s.Nodes)
  699. sort.Sort(sg)
  700. sort.Sort(sw)
  701. if !reflect.DeepEqual(sg, sw) {
  702. t.Errorf("sm.Nodes = %+v, want %+v", sg, sw)
  703. }
  704. if !reflect.DeepEqual(sm.raftLog.snapshot, s) {
  705. t.Errorf("snapshot = %+v, want %+v", sm.raftLog.snapshot, s)
  706. }
  707. if ok := sm.restore(s); ok {
  708. t.Fatal("restore succeed, want fail")
  709. }
  710. }
  711. func TestProvideSnap(t *testing.T) {
  712. s := Snapshot{
  713. Index: defaultCompactThreshold + 1,
  714. Term: defaultCompactThreshold + 1,
  715. Nodes: []int64{0, 1},
  716. }
  717. sm := newRaft(0, []int64{0})
  718. // restore the statemachin from a snapshot
  719. // so it has a compacted log and a snapshot
  720. sm.restore(s)
  721. sm.becomeCandidate()
  722. sm.becomeLeader()
  723. sm.Step(Message{From: 0, To: 0, Type: msgBeat})
  724. msgs := sm.ReadMessages()
  725. if len(msgs) != 1 {
  726. t.Errorf("len(msgs) = %d, want 1", len(msgs))
  727. }
  728. m := msgs[0]
  729. if m.Type != msgApp {
  730. t.Errorf("m.Type = %v, want %v", m.Type, msgApp)
  731. }
  732. // force set the next of node 1, so that
  733. // node 1 needs a snapshot
  734. sm.prs[1].next = sm.raftLog.offset
  735. sm.Step(Message{From: 1, To: 0, Type: msgAppResp, Index: -1})
  736. msgs = sm.ReadMessages()
  737. if len(msgs) != 1 {
  738. t.Errorf("len(msgs) = %d, want 1", len(msgs))
  739. }
  740. m = msgs[0]
  741. if m.Type != msgSnap {
  742. t.Errorf("m.Type = %v, want %v", m.Type, msgSnap)
  743. }
  744. }
  745. func TestRestoreFromSnapMsg(t *testing.T) {
  746. s := Snapshot{
  747. Index: defaultCompactThreshold + 1,
  748. Term: defaultCompactThreshold + 1,
  749. Nodes: []int64{0, 1},
  750. }
  751. m := Message{Type: msgSnap, From: 0, Term: 1, Snapshot: s}
  752. sm := newRaft(1, []int64{0, 1})
  753. sm.Step(m)
  754. if !reflect.DeepEqual(sm.raftLog.snapshot, s) {
  755. t.Errorf("snapshot = %+v, want %+v", sm.raftLog.snapshot, s)
  756. }
  757. }
  758. func TestSlowNodeRestore(t *testing.T) {
  759. nt := newNetwork(nil, nil, nil)
  760. nt.send(Message{From: 0, To: 0, Type: msgHup})
  761. nt.isolate(2)
  762. for j := 0; j < defaultCompactThreshold+1; j++ {
  763. nt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{}}})
  764. }
  765. lead := nt.peers[0].(*raft)
  766. lead.nextEnts()
  767. lead.compact(nil)
  768. nt.recover()
  769. nt.send(Message{From: 0, To: 0, Type: msgBeat})
  770. follower := nt.peers[2].(*raft)
  771. if !reflect.DeepEqual(follower.raftLog.snapshot, lead.raftLog.snapshot) {
  772. t.Errorf("follower.snap = %+v, want %+v", follower.raftLog.snapshot, lead.raftLog.snapshot)
  773. }
  774. committed := follower.raftLog.lastIndex()
  775. nt.send(Message{From: 0, To: 0, Type: msgProp, Entries: []Entry{{}}})
  776. if follower.raftLog.committed != committed+1 {
  777. t.Errorf("follower.comitted = %d, want %d", follower.raftLog.committed, committed+1)
  778. }
  779. }
  780. func ents(terms ...int64) *raft {
  781. ents := []Entry{{}}
  782. for _, term := range terms {
  783. ents = append(ents, Entry{Term: term})
  784. }
  785. sm := &raft{raftLog: &raftLog{ents: ents}}
  786. sm.reset(0)
  787. return sm
  788. }
  789. type network struct {
  790. peers map[int64]Interface
  791. dropm map[connem]float64
  792. ignorem map[messageType]bool
  793. }
  794. // newNetwork initializes a network from peers.
  795. // A nil node will be replaced with a new *stateMachine.
  796. // A *stateMachine will get its k, id.
  797. // When using stateMachine, the address list is always [0, n).
  798. func newNetwork(peers ...Interface) *network {
  799. size := len(peers)
  800. defaultPeerAddrs := make([]int64, size)
  801. for i := 0; i < size; i++ {
  802. defaultPeerAddrs[i] = int64(i)
  803. }
  804. npeers := make(map[int64]Interface, size)
  805. for id, p := range peers {
  806. nid := int64(id)
  807. switch v := p.(type) {
  808. case nil:
  809. sm := newRaft(nid, defaultPeerAddrs)
  810. npeers[nid] = sm
  811. case *raft:
  812. v.id = nid
  813. v.prs = make(map[int64]*progress)
  814. for i := 0; i < size; i++ {
  815. v.prs[int64(i)] = &progress{}
  816. }
  817. v.reset(0)
  818. npeers[nid] = v
  819. default:
  820. npeers[nid] = v
  821. }
  822. }
  823. return &network{
  824. peers: npeers,
  825. dropm: make(map[connem]float64),
  826. ignorem: make(map[messageType]bool),
  827. }
  828. }
  829. func (nw *network) send(msgs ...Message) {
  830. for len(msgs) > 0 {
  831. m := msgs[0]
  832. p := nw.peers[m.To]
  833. p.Step(m)
  834. msgs = append(msgs[1:], nw.filter(p.ReadMessages())...)
  835. }
  836. }
  837. func (nw *network) drop(from, to int64, perc float64) {
  838. nw.dropm[connem{from, to}] = perc
  839. }
  840. func (nw *network) cut(one, other int64) {
  841. nw.drop(one, other, 1)
  842. nw.drop(other, one, 1)
  843. }
  844. func (nw *network) isolate(id int64) {
  845. for i := 0; i < len(nw.peers); i++ {
  846. nid := int64(i)
  847. if nid != id {
  848. nw.drop(id, nid, 1.0)
  849. nw.drop(nid, id, 1.0)
  850. }
  851. }
  852. }
  853. func (nw *network) ignore(t messageType) {
  854. nw.ignorem[t] = true
  855. }
  856. func (nw *network) recover() {
  857. nw.dropm = make(map[connem]float64)
  858. nw.ignorem = make(map[messageType]bool)
  859. }
  860. func (nw *network) filter(msgs []Message) []Message {
  861. mm := make([]Message, 0)
  862. for _, m := range msgs {
  863. if nw.ignorem[m.Type] {
  864. continue
  865. }
  866. switch m.Type {
  867. case msgHup:
  868. // hups never go over the network, so don't drop them but panic
  869. panic("unexpected msgHup")
  870. default:
  871. perc := nw.dropm[connem{m.From, m.To}]
  872. if n := rand.Float64(); n < perc {
  873. continue
  874. }
  875. }
  876. mm = append(mm, m)
  877. }
  878. return mm
  879. }
  880. type connem struct {
  881. from, to int64
  882. }
  883. type blackHole struct{}
  884. func (blackHole) Step(Message) error { return nil }
  885. func (blackHole) ReadMessages() []Message { return nil }
  886. var nopStepper = &blackHole{}