raft.go 10 KB

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