raft.go 12 KB

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