raft.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605
  1. package raft
  2. import (
  3. "errors"
  4. "fmt"
  5. "log"
  6. "sort"
  7. "sync/atomic"
  8. )
  9. const none = -1
  10. type messageType int64
  11. const (
  12. msgHup messageType = iota
  13. msgBeat
  14. msgProp
  15. msgApp
  16. msgAppResp
  17. msgVote
  18. msgVoteResp
  19. msgSnap
  20. msgDenied
  21. )
  22. var mtmap = [...]string{
  23. msgHup: "msgHup",
  24. msgBeat: "msgBeat",
  25. msgProp: "msgProp",
  26. msgApp: "msgApp",
  27. msgAppResp: "msgAppResp",
  28. msgVote: "msgVote",
  29. msgVoteResp: "msgVoteResp",
  30. msgSnap: "msgSnap",
  31. msgDenied: "msgDenied",
  32. }
  33. func (mt messageType) String() string {
  34. return mtmap[int64(mt)]
  35. }
  36. var errNoLeader = errors.New("no leader")
  37. const (
  38. stateFollower stateType = iota
  39. stateCandidate
  40. stateLeader
  41. )
  42. type stateType int64
  43. var stmap = [...]string{
  44. stateFollower: "stateFollower",
  45. stateCandidate: "stateCandidate",
  46. stateLeader: "stateLeader",
  47. }
  48. var stepmap = [...]stepFunc{
  49. stateFollower: stepFollower,
  50. stateCandidate: stepCandidate,
  51. stateLeader: stepLeader,
  52. }
  53. func (st stateType) String() string {
  54. return stmap[int64(st)]
  55. }
  56. type State struct {
  57. Term int64
  58. Vote int64
  59. Commit int64
  60. }
  61. var emptyState = State{}
  62. type Message struct {
  63. Type messageType
  64. ClusterId int64
  65. To int64
  66. From int64
  67. Term int64
  68. LogTerm int64
  69. Index int64
  70. PrevTerm int64
  71. Entries []Entry
  72. Commit int64
  73. Snapshot Snapshot
  74. }
  75. func (m Message) String() string {
  76. return fmt.Sprintf("type=%v from=%x to=%x term=%d logTerm=%d i=%d ci=%d len(ents)=%d",
  77. m.Type, m.From, m.To, m.Term, m.LogTerm, m.Index, m.Commit, len(m.Entries))
  78. }
  79. type index struct {
  80. match, next int64
  81. }
  82. func (in *index) update(n int64) {
  83. in.match = n
  84. in.next = n + 1
  85. }
  86. func (in *index) decr() {
  87. if in.next--; in.next < 1 {
  88. in.next = 1
  89. }
  90. }
  91. func (in *index) String() string {
  92. return fmt.Sprintf("n=%d m=%d", in.next, in.match)
  93. }
  94. // An AtomicInt is an int64 to be accessed atomically.
  95. type atomicInt int64
  96. func (i *atomicInt) Set(n int64) {
  97. atomic.StoreInt64((*int64)(i), n)
  98. }
  99. func (i *atomicInt) Get() int64 {
  100. return atomic.LoadInt64((*int64)(i))
  101. }
  102. // int64Slice implements sort interface
  103. type int64Slice []int64
  104. func (p int64Slice) Len() int { return len(p) }
  105. func (p int64Slice) Less(i, j int) bool { return p[i] < p[j] }
  106. func (p int64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
  107. type stateMachine struct {
  108. clusterId int64
  109. id int64
  110. // the term we are participating in at any time
  111. term atomicInt
  112. index atomicInt
  113. // who we voted for in term
  114. vote int64
  115. // the log
  116. raftLog *raftLog
  117. ins map[int64]*index
  118. state stateType
  119. votes map[int64]bool
  120. msgs []Message
  121. // the leader id
  122. lead atomicInt
  123. // pending reconfiguration
  124. pendingConf bool
  125. snapshoter Snapshoter
  126. unstableState State
  127. }
  128. func newStateMachine(id int64, peers []int64) *stateMachine {
  129. if id == none {
  130. panic("cannot use none id")
  131. }
  132. sm := &stateMachine{id: id, clusterId: none, lead: none, raftLog: newLog(), ins: make(map[int64]*index)}
  133. for _, p := range peers {
  134. sm.ins[p] = &index{}
  135. }
  136. sm.reset(0)
  137. return sm
  138. }
  139. func (sm *stateMachine) String() string {
  140. s := fmt.Sprintf(`state=%v term=%d`, sm.state, sm.term)
  141. switch sm.state {
  142. case stateFollower:
  143. s += fmt.Sprintf(" vote=%v lead=%v", sm.vote, sm.lead)
  144. case stateCandidate:
  145. s += fmt.Sprintf(` votes="%v"`, sm.votes)
  146. case stateLeader:
  147. s += fmt.Sprintf(` ins="%v"`, sm.ins)
  148. }
  149. return s
  150. }
  151. func (sm *stateMachine) setSnapshoter(snapshoter Snapshoter) {
  152. sm.snapshoter = snapshoter
  153. }
  154. func (sm *stateMachine) poll(id int64, v bool) (granted int) {
  155. if _, ok := sm.votes[id]; !ok {
  156. sm.votes[id] = v
  157. }
  158. for _, vv := range sm.votes {
  159. if vv {
  160. granted++
  161. }
  162. }
  163. return granted
  164. }
  165. // send persists state to stable storage and then sends to its mailbox.
  166. func (sm *stateMachine) send(m Message) {
  167. m.ClusterId = sm.clusterId
  168. m.From = sm.id
  169. m.Term = sm.term.Get()
  170. log.Printf("raft.send msg %v\n", m)
  171. sm.msgs = append(sm.msgs, m)
  172. }
  173. // sendAppend sends RRPC, with entries to the given peer.
  174. func (sm *stateMachine) sendAppend(to int64) {
  175. in := sm.ins[to]
  176. m := Message{}
  177. m.To = to
  178. m.Index = in.next - 1
  179. if sm.needSnapshot(m.Index) {
  180. m.Type = msgSnap
  181. m.Snapshot = sm.snapshoter.GetSnap()
  182. } else {
  183. m.Type = msgApp
  184. m.LogTerm = sm.raftLog.term(in.next - 1)
  185. m.Entries = sm.raftLog.entries(in.next)
  186. m.Commit = sm.raftLog.committed
  187. }
  188. sm.send(m)
  189. }
  190. // sendHeartbeat sends RRPC, without entries to the given peer.
  191. func (sm *stateMachine) sendHeartbeat(to int64) {
  192. in := sm.ins[to]
  193. index := max(in.next-1, sm.raftLog.lastIndex())
  194. m := Message{
  195. To: to,
  196. Type: msgApp,
  197. Index: index,
  198. LogTerm: sm.raftLog.term(index),
  199. Commit: sm.raftLog.committed,
  200. }
  201. sm.send(m)
  202. }
  203. // bcastAppend sends RRPC, with entries to all peers that are not up-to-date according to sm.mis.
  204. func (sm *stateMachine) bcastAppend() {
  205. for i := range sm.ins {
  206. if i == sm.id {
  207. continue
  208. }
  209. sm.sendAppend(i)
  210. }
  211. }
  212. // bcastHeartbeat sends RRPC, without entries to all the peers.
  213. func (sm *stateMachine) bcastHeartbeat() {
  214. for i := range sm.ins {
  215. if i == sm.id {
  216. continue
  217. }
  218. sm.sendHeartbeat(i)
  219. }
  220. }
  221. func (sm *stateMachine) maybeCommit() bool {
  222. // TODO(bmizerany): optimize.. Currently naive
  223. mis := make(int64Slice, 0, len(sm.ins))
  224. for i := range sm.ins {
  225. mis = append(mis, sm.ins[i].match)
  226. }
  227. sort.Sort(sort.Reverse(mis))
  228. mci := mis[sm.q()-1]
  229. return sm.raftLog.maybeCommit(mci, sm.term.Get())
  230. }
  231. // nextEnts returns the appliable entries and updates the applied index
  232. func (sm *stateMachine) nextEnts() (ents []Entry) {
  233. return sm.raftLog.nextEnts()
  234. }
  235. func (sm *stateMachine) reset(term int64) {
  236. sm.setTerm(term)
  237. sm.lead.Set(none)
  238. sm.setVote(none)
  239. sm.votes = make(map[int64]bool)
  240. for i := range sm.ins {
  241. sm.ins[i] = &index{next: sm.raftLog.lastIndex() + 1}
  242. if i == sm.id {
  243. sm.ins[i].match = sm.raftLog.lastIndex()
  244. }
  245. }
  246. }
  247. func (sm *stateMachine) q() int {
  248. return len(sm.ins)/2 + 1
  249. }
  250. func (sm *stateMachine) appendEntry(e Entry) {
  251. e.Term = sm.term.Get()
  252. sm.index.Set(sm.raftLog.append(sm.raftLog.lastIndex(), e))
  253. sm.ins[sm.id].update(sm.raftLog.lastIndex())
  254. sm.maybeCommit()
  255. }
  256. // promotable indicates whether state machine could be promoted.
  257. // New machine has to wait for the first log entry to be committed, or it will
  258. // always start as a one-node cluster.
  259. func (sm *stateMachine) promotable() bool {
  260. return sm.raftLog.committed != 0
  261. }
  262. func (sm *stateMachine) becomeFollower(term int64, lead int64) {
  263. sm.reset(term)
  264. sm.lead.Set(lead)
  265. sm.state = stateFollower
  266. sm.pendingConf = false
  267. }
  268. func (sm *stateMachine) becomeCandidate() {
  269. // TODO(xiangli) remove the panic when the raft implementation is stable
  270. if sm.state == stateLeader {
  271. panic("invalid transition [leader -> candidate]")
  272. }
  273. sm.reset(sm.term.Get() + 1)
  274. sm.setVote(sm.id)
  275. sm.state = stateCandidate
  276. }
  277. func (sm *stateMachine) becomeLeader() {
  278. // TODO(xiangli) remove the panic when the raft implementation is stable
  279. if sm.state == stateFollower {
  280. panic("invalid transition [follower -> leader]")
  281. }
  282. sm.reset(sm.term.Get())
  283. sm.lead.Set(sm.id)
  284. sm.state = stateLeader
  285. for _, e := range sm.raftLog.entries(sm.raftLog.committed + 1) {
  286. if e.isConfig() {
  287. sm.pendingConf = true
  288. }
  289. }
  290. sm.appendEntry(Entry{Type: Normal, Data: nil})
  291. }
  292. func (sm *stateMachine) Msgs() []Message {
  293. msgs := sm.msgs
  294. sm.msgs = make([]Message, 0)
  295. return msgs
  296. }
  297. func (sm *stateMachine) Step(m Message) (ok bool) {
  298. log.Printf("raft.step beforeState %v\n", sm)
  299. log.Printf("raft.step beforeLog %v\n", sm.raftLog)
  300. defer log.Printf("raft.step afterLog %v\n", sm.raftLog)
  301. defer log.Printf("raft.step afterState %v\n", sm)
  302. log.Printf("raft.step msg %v\n", m)
  303. if m.Type == msgHup {
  304. sm.becomeCandidate()
  305. if sm.q() == sm.poll(sm.id, true) {
  306. sm.becomeLeader()
  307. return true
  308. }
  309. for i := range sm.ins {
  310. if i == sm.id {
  311. continue
  312. }
  313. lasti := sm.raftLog.lastIndex()
  314. sm.send(Message{To: i, Type: msgVote, Index: lasti, LogTerm: sm.raftLog.term(lasti)})
  315. }
  316. return true
  317. }
  318. switch {
  319. case m.Term == 0:
  320. // local message
  321. case m.Term > sm.term.Get():
  322. lead := m.From
  323. if m.Type == msgVote {
  324. lead = none
  325. }
  326. sm.becomeFollower(m.Term, lead)
  327. case m.Term < sm.term.Get():
  328. // ignore
  329. return true
  330. }
  331. return stepmap[sm.state](sm, m)
  332. }
  333. func (sm *stateMachine) handleAppendEntries(m Message) {
  334. if sm.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...) {
  335. sm.index.Set(sm.raftLog.lastIndex())
  336. sm.send(Message{To: m.From, Type: msgAppResp, Index: sm.raftLog.lastIndex()})
  337. } else {
  338. sm.send(Message{To: m.From, Type: msgAppResp, Index: -1})
  339. }
  340. }
  341. func (sm *stateMachine) handleSnapshot(m Message) {
  342. sm.restore(m.Snapshot)
  343. sm.send(Message{To: m.From, Type: msgAppResp, Index: sm.raftLog.lastIndex()})
  344. }
  345. func (sm *stateMachine) addNode(id int64) {
  346. sm.addIns(id, 0, sm.raftLog.lastIndex()+1)
  347. sm.pendingConf = false
  348. }
  349. func (sm *stateMachine) removeNode(id int64) {
  350. sm.deleteIns(id)
  351. sm.pendingConf = false
  352. }
  353. type stepFunc func(sm *stateMachine, m Message) bool
  354. func stepLeader(sm *stateMachine, m Message) bool {
  355. switch m.Type {
  356. case msgBeat:
  357. sm.bcastHeartbeat()
  358. case msgProp:
  359. if len(m.Entries) != 1 {
  360. panic("unexpected length(entries) of a msgProp")
  361. }
  362. e := m.Entries[0]
  363. if e.isConfig() {
  364. if sm.pendingConf {
  365. return false
  366. }
  367. sm.pendingConf = true
  368. }
  369. sm.appendEntry(e)
  370. sm.bcastAppend()
  371. case msgAppResp:
  372. if m.Index < 0 {
  373. sm.ins[m.From].decr()
  374. sm.sendAppend(m.From)
  375. } else {
  376. sm.ins[m.From].update(m.Index)
  377. if sm.maybeCommit() {
  378. sm.bcastAppend()
  379. }
  380. }
  381. case msgVote:
  382. sm.send(Message{To: m.From, Type: msgVoteResp, Index: -1})
  383. }
  384. return true
  385. }
  386. func stepCandidate(sm *stateMachine, m Message) bool {
  387. switch m.Type {
  388. case msgProp:
  389. return false
  390. case msgApp:
  391. sm.becomeFollower(sm.term.Get(), m.From)
  392. sm.handleAppendEntries(m)
  393. case msgSnap:
  394. sm.becomeFollower(m.Term, m.From)
  395. sm.handleSnapshot(m)
  396. case msgVote:
  397. sm.send(Message{To: m.From, Type: msgVoteResp, Index: -1})
  398. case msgVoteResp:
  399. gr := sm.poll(m.From, m.Index >= 0)
  400. switch sm.q() {
  401. case gr:
  402. sm.becomeLeader()
  403. sm.bcastAppend()
  404. case len(sm.votes) - gr:
  405. sm.becomeFollower(sm.term.Get(), none)
  406. }
  407. }
  408. return true
  409. }
  410. func stepFollower(sm *stateMachine, m Message) bool {
  411. switch m.Type {
  412. case msgProp:
  413. if sm.lead.Get() == none {
  414. return false
  415. }
  416. m.To = sm.lead.Get()
  417. sm.send(m)
  418. case msgApp:
  419. sm.lead.Set(m.From)
  420. sm.handleAppendEntries(m)
  421. case msgSnap:
  422. sm.handleSnapshot(m)
  423. case msgVote:
  424. if (sm.vote == none || sm.vote == m.From) && sm.raftLog.isUpToDate(m.Index, m.LogTerm) {
  425. sm.setVote(m.From)
  426. sm.send(Message{To: m.From, Type: msgVoteResp, Index: sm.raftLog.lastIndex()})
  427. } else {
  428. sm.send(Message{To: m.From, Type: msgVoteResp, Index: -1})
  429. }
  430. }
  431. return true
  432. }
  433. // maybeCompact tries to compact the log. It calls the snapshoter to take a snapshot and
  434. // then compact the log up-to the index at which the snapshot was taken.
  435. func (sm *stateMachine) maybeCompact() bool {
  436. if sm.snapshoter == nil || !sm.raftLog.shouldCompact() {
  437. return false
  438. }
  439. sm.snapshoter.Snap(sm.raftLog.applied, sm.raftLog.term(sm.raftLog.applied), sm.nodes())
  440. sm.raftLog.compact(sm.raftLog.applied)
  441. return true
  442. }
  443. // restore recovers the statemachine from a snapshot. It restores the log and the
  444. // configuration of statemachine. It calls the snapshoter to restore from the given
  445. // snapshot.
  446. func (sm *stateMachine) restore(s Snapshot) {
  447. if sm.snapshoter == nil {
  448. panic("try to restore from snapshot, but snapshoter is nil")
  449. }
  450. sm.raftLog.restore(s.Index, s.Term)
  451. sm.index.Set(sm.raftLog.lastIndex())
  452. sm.ins = make(map[int64]*index)
  453. for _, n := range s.Nodes {
  454. if n == sm.id {
  455. sm.addIns(n, sm.raftLog.lastIndex(), sm.raftLog.lastIndex()+1)
  456. } else {
  457. sm.addIns(n, 0, sm.raftLog.lastIndex()+1)
  458. }
  459. }
  460. sm.pendingConf = false
  461. sm.snapshoter.Restore(s)
  462. }
  463. func (sm *stateMachine) needSnapshot(i int64) bool {
  464. if i < sm.raftLog.offset {
  465. if sm.snapshoter == nil {
  466. panic("need snapshot but snapshoter is nil")
  467. }
  468. return true
  469. }
  470. return false
  471. }
  472. func (sm *stateMachine) nodes() []int64 {
  473. nodes := make([]int64, 0, len(sm.ins))
  474. for k := range sm.ins {
  475. nodes = append(nodes, k)
  476. }
  477. return nodes
  478. }
  479. func (sm *stateMachine) setTerm(term int64) {
  480. sm.term.Set(term)
  481. sm.saveState()
  482. }
  483. func (sm *stateMachine) setVote(vote int64) {
  484. sm.vote = vote
  485. sm.saveState()
  486. }
  487. func (sm *stateMachine) addIns(id, match, next int64) {
  488. sm.ins[id] = &index{next: next, match: match}
  489. sm.saveState()
  490. }
  491. func (sm *stateMachine) deleteIns(id int64) {
  492. delete(sm.ins, id)
  493. sm.saveState()
  494. }
  495. // saveState saves the state to sm.unstableState
  496. // When there is a term change, vote change or configuration change, raft
  497. // must call saveState.
  498. func (sm *stateMachine) saveState() {
  499. sm.setState(sm.vote, sm.term.Get(), sm.raftLog.committed)
  500. }
  501. func (sm *stateMachine) clearState() {
  502. sm.setState(0, 0, 0)
  503. }
  504. func (sm *stateMachine) setState(vote, term, commit int64) {
  505. sm.unstableState.Vote = vote
  506. sm.unstableState.Term = term
  507. sm.unstableState.Commit = commit
  508. }
  509. func (sm *stateMachine) loadEnts(ents []Entry) {
  510. if !sm.raftLog.isEmpty() {
  511. panic("cannot load entries when log is not empty")
  512. }
  513. sm.raftLog.append(0, ents...)
  514. sm.raftLog.unstable = sm.raftLog.lastIndex() + 1
  515. }
  516. func (sm *stateMachine) loadState(state State) {
  517. sm.raftLog.committed = state.Commit
  518. sm.setTerm(state.Term)
  519. sm.setVote(state.Vote)
  520. }