server.go 36 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354
  1. package raft
  2. import (
  3. "encoding/json"
  4. "errors"
  5. "fmt"
  6. "hash/crc32"
  7. "io/ioutil"
  8. "os"
  9. "path"
  10. "sort"
  11. "sync"
  12. "time"
  13. )
  14. //------------------------------------------------------------------------------
  15. //
  16. // Constants
  17. //
  18. //------------------------------------------------------------------------------
  19. const (
  20. Stopped = "stopped"
  21. Follower = "follower"
  22. Candidate = "candidate"
  23. Leader = "leader"
  24. Snapshotting = "snapshotting"
  25. )
  26. const (
  27. MaxLogEntriesPerRequest = 2000
  28. NumberOfLogEntriesAfterSnapshot = 200
  29. )
  30. const (
  31. // DefaultHeartbeatInterval is the interval that the leader will send
  32. // AppendEntriesRequests to followers to maintain leadership.
  33. DefaultHeartbeatInterval = 50 * time.Millisecond
  34. DefaultElectionTimeout = 150 * time.Millisecond
  35. )
  36. // ElectionTimeoutThresholdPercent specifies the threshold at which the server
  37. // will dispatch warning events that the heartbeat RTT is too close to the
  38. // election timeout.
  39. const ElectionTimeoutThresholdPercent = 0.8
  40. var stopValue interface{}
  41. //------------------------------------------------------------------------------
  42. //
  43. // Errors
  44. //
  45. //------------------------------------------------------------------------------
  46. var NotLeaderError = errors.New("raft.Server: Not current leader")
  47. var DuplicatePeerError = errors.New("raft.Server: Duplicate peer")
  48. var CommandTimeoutError = errors.New("raft: Command timeout")
  49. //------------------------------------------------------------------------------
  50. //
  51. // Typedefs
  52. //
  53. //------------------------------------------------------------------------------
  54. // A server is involved in the consensus protocol and can act as a follower,
  55. // candidate or a leader.
  56. type Server interface {
  57. Name() string
  58. Context() interface{}
  59. StateMachine() StateMachine
  60. Leader() string
  61. State() string
  62. Path() string
  63. LogPath() string
  64. SnapshotPath(lastIndex uint64, lastTerm uint64) string
  65. Term() uint64
  66. CommitIndex() uint64
  67. VotedFor() string
  68. MemberCount() int
  69. QuorumSize() int
  70. IsLogEmpty() bool
  71. LogEntries() []*LogEntry
  72. LastCommandName() string
  73. GetState() string
  74. ElectionTimeout() time.Duration
  75. SetElectionTimeout(duration time.Duration)
  76. HeartbeatInterval() time.Duration
  77. SetHeartbeatInterval(duration time.Duration)
  78. Transporter() Transporter
  79. SetTransporter(t Transporter)
  80. AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse
  81. RequestVote(req *RequestVoteRequest) *RequestVoteResponse
  82. RequestSnapshot(req *SnapshotRequest) *SnapshotResponse
  83. SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse
  84. AddPeer(name string, connectiongString string) error
  85. RemovePeer(name string) error
  86. Peers() map[string]*Peer
  87. Start() error
  88. Stop()
  89. Running() bool
  90. Do(command Command) (interface{}, error)
  91. TakeSnapshot() error
  92. LoadSnapshot() error
  93. AddEventListener(string, EventListener)
  94. }
  95. type server struct {
  96. *eventDispatcher
  97. name string
  98. path string
  99. state string
  100. transporter Transporter
  101. context interface{}
  102. currentTerm uint64
  103. votedFor string
  104. log *Log
  105. leader string
  106. peers map[string]*Peer
  107. mutex sync.RWMutex
  108. syncedPeer map[string]bool
  109. stopped chan bool
  110. c chan *ev
  111. electionTimeout time.Duration
  112. heartbeatInterval time.Duration
  113. currentSnapshot *Snapshot
  114. lastSnapshot *Snapshot
  115. stateMachine StateMachine
  116. maxLogEntriesPerRequest uint64
  117. connectionString string
  118. }
  119. // An internal event to be processed by the server's event loop.
  120. type ev struct {
  121. target interface{}
  122. returnValue interface{}
  123. c chan error
  124. }
  125. //------------------------------------------------------------------------------
  126. //
  127. // Constructor
  128. //
  129. //------------------------------------------------------------------------------
  130. // Creates a new server with a log at the given path. transporter must
  131. // not be nil. stateMachine can be nil if snapshotting and log
  132. // compaction is to be disabled. context can be anything (including nil)
  133. // and is not used by the raft package except returned by
  134. // Server.Context(). connectionString can be anything.
  135. func NewServer(name string, path string, transporter Transporter, stateMachine StateMachine, ctx interface{}, connectionString string) (Server, error) {
  136. if name == "" {
  137. return nil, errors.New("raft.Server: Name cannot be blank")
  138. }
  139. if transporter == nil {
  140. panic("raft: Transporter required")
  141. }
  142. s := &server{
  143. name: name,
  144. path: path,
  145. transporter: transporter,
  146. stateMachine: stateMachine,
  147. context: ctx,
  148. state: Stopped,
  149. peers: make(map[string]*Peer),
  150. log: newLog(),
  151. stopped: make(chan bool),
  152. c: make(chan *ev, 256),
  153. electionTimeout: DefaultElectionTimeout,
  154. heartbeatInterval: DefaultHeartbeatInterval,
  155. maxLogEntriesPerRequest: MaxLogEntriesPerRequest,
  156. connectionString: connectionString,
  157. }
  158. s.eventDispatcher = newEventDispatcher(s)
  159. // Setup apply function.
  160. s.log.ApplyFunc = func(c Command) (interface{}, error) {
  161. switch c := c.(type) {
  162. case CommandApply:
  163. return c.Apply(&context{
  164. server: s,
  165. currentTerm: s.currentTerm,
  166. currentIndex: s.log.internalCurrentIndex(),
  167. commitIndex: s.log.commitIndex,
  168. })
  169. case deprecatedCommandApply:
  170. return c.Apply(s)
  171. default:
  172. return nil, fmt.Errorf("Command does not implement Apply()")
  173. }
  174. }
  175. return s, nil
  176. }
  177. //------------------------------------------------------------------------------
  178. //
  179. // Accessors
  180. //
  181. //------------------------------------------------------------------------------
  182. //--------------------------------------
  183. // General
  184. //--------------------------------------
  185. // Retrieves the name of the server.
  186. func (s *server) Name() string {
  187. return s.name
  188. }
  189. // Retrieves the storage path for the server.
  190. func (s *server) Path() string {
  191. return s.path
  192. }
  193. // The name of the current leader.
  194. func (s *server) Leader() string {
  195. return s.leader
  196. }
  197. // Retrieves a copy of the peer data.
  198. func (s *server) Peers() map[string]*Peer {
  199. s.mutex.Lock()
  200. defer s.mutex.Unlock()
  201. peers := make(map[string]*Peer)
  202. for name, peer := range s.peers {
  203. peers[name] = peer.clone()
  204. }
  205. return peers
  206. }
  207. // Retrieves the object that transports requests.
  208. func (s *server) Transporter() Transporter {
  209. s.mutex.RLock()
  210. defer s.mutex.RUnlock()
  211. return s.transporter
  212. }
  213. func (s *server) SetTransporter(t Transporter) {
  214. s.mutex.Lock()
  215. defer s.mutex.Unlock()
  216. s.transporter = t
  217. }
  218. // Retrieves the context passed into the constructor.
  219. func (s *server) Context() interface{} {
  220. return s.context
  221. }
  222. // Retrieves the state machine passed into the constructor.
  223. func (s *server) StateMachine() StateMachine {
  224. return s.stateMachine
  225. }
  226. // Retrieves the log path for the server.
  227. func (s *server) LogPath() string {
  228. return path.Join(s.path, "log")
  229. }
  230. // Retrieves the current state of the server.
  231. func (s *server) State() string {
  232. s.mutex.RLock()
  233. defer s.mutex.RUnlock()
  234. return s.state
  235. }
  236. // Sets the state of the server.
  237. func (s *server) setState(state string) {
  238. s.mutex.Lock()
  239. defer s.mutex.Unlock()
  240. // Temporarily store previous values.
  241. prevState := s.state
  242. prevLeader := s.leader
  243. // Update state and leader.
  244. s.state = state
  245. if state == Leader {
  246. s.leader = s.Name()
  247. s.syncedPeer = make(map[string]bool)
  248. }
  249. // Dispatch state and leader change events.
  250. if prevState != state {
  251. s.DispatchEvent(newEvent(StateChangeEventType, s.state, prevState))
  252. }
  253. if prevLeader != s.leader {
  254. s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader))
  255. }
  256. }
  257. // Retrieves the current term of the server.
  258. func (s *server) Term() uint64 {
  259. s.mutex.RLock()
  260. defer s.mutex.RUnlock()
  261. return s.currentTerm
  262. }
  263. // Retrieves the current commit index of the server.
  264. func (s *server) CommitIndex() uint64 {
  265. s.log.mutex.RLock()
  266. defer s.log.mutex.RUnlock()
  267. return s.log.commitIndex
  268. }
  269. // Retrieves the name of the candidate this server voted for in this term.
  270. func (s *server) VotedFor() string {
  271. return s.votedFor
  272. }
  273. // Retrieves whether the server's log has no entries.
  274. func (s *server) IsLogEmpty() bool {
  275. return s.log.isEmpty()
  276. }
  277. // A list of all the log entries. This should only be used for debugging purposes.
  278. func (s *server) LogEntries() []*LogEntry {
  279. return s.log.entries
  280. }
  281. // A reference to the command name of the last entry.
  282. func (s *server) LastCommandName() string {
  283. return s.log.lastCommandName()
  284. }
  285. // Get the state of the server for debugging
  286. func (s *server) GetState() string {
  287. s.mutex.RLock()
  288. defer s.mutex.RUnlock()
  289. return fmt.Sprintf("Name: %s, State: %s, Term: %v, CommitedIndex: %v ", s.name, s.state, s.currentTerm, s.log.commitIndex)
  290. }
  291. // Check if the server is promotable
  292. func (s *server) promotable() bool {
  293. return s.log.currentIndex() > 0
  294. }
  295. //--------------------------------------
  296. // Membership
  297. //--------------------------------------
  298. // Retrieves the number of member servers in the consensus.
  299. func (s *server) MemberCount() int {
  300. s.mutex.Lock()
  301. defer s.mutex.Unlock()
  302. return len(s.peers) + 1
  303. }
  304. // Retrieves the number of servers required to make a quorum.
  305. func (s *server) QuorumSize() int {
  306. return (s.MemberCount() / 2) + 1
  307. }
  308. //--------------------------------------
  309. // Election timeout
  310. //--------------------------------------
  311. // Retrieves the election timeout.
  312. func (s *server) ElectionTimeout() time.Duration {
  313. s.mutex.RLock()
  314. defer s.mutex.RUnlock()
  315. return s.electionTimeout
  316. }
  317. // Sets the election timeout.
  318. func (s *server) SetElectionTimeout(duration time.Duration) {
  319. s.mutex.Lock()
  320. defer s.mutex.Unlock()
  321. s.electionTimeout = duration
  322. }
  323. //--------------------------------------
  324. // Heartbeat timeout
  325. //--------------------------------------
  326. // Retrieves the heartbeat timeout.
  327. func (s *server) HeartbeatInterval() time.Duration {
  328. s.mutex.RLock()
  329. defer s.mutex.RUnlock()
  330. return s.heartbeatInterval
  331. }
  332. // Sets the heartbeat timeout.
  333. func (s *server) SetHeartbeatInterval(duration time.Duration) {
  334. s.mutex.Lock()
  335. defer s.mutex.Unlock()
  336. s.heartbeatInterval = duration
  337. for _, peer := range s.peers {
  338. peer.setHeartbeatInterval(duration)
  339. }
  340. }
  341. //------------------------------------------------------------------------------
  342. //
  343. // Methods
  344. //
  345. //------------------------------------------------------------------------------
  346. //--------------------------------------
  347. // Initialization
  348. //--------------------------------------
  349. // Reg the NOPCommand
  350. func init() {
  351. RegisterCommand(&NOPCommand{})
  352. RegisterCommand(&DefaultJoinCommand{})
  353. RegisterCommand(&DefaultLeaveCommand{})
  354. }
  355. // Start as follow
  356. // If log entries exist then allow promotion to candidate if no AEs received.
  357. // If no log entries exist then wait for AEs from another node.
  358. // If no log entries exist and a self-join command is issued then
  359. // immediately become leader and commit entry.
  360. func (s *server) Start() error {
  361. // Exit if the server is already running.
  362. if s.State() != Stopped {
  363. return errors.New("raft.Server: Server already running")
  364. }
  365. // Create snapshot directory if not exist
  366. os.Mkdir(path.Join(s.path, "snapshot"), 0700)
  367. if err := s.readConf(); err != nil {
  368. s.debugln("raft: Conf file error: ", err)
  369. return fmt.Errorf("raft: Initialization error: %s", err)
  370. }
  371. // Initialize the log and load it up.
  372. if err := s.log.open(s.LogPath()); err != nil {
  373. s.debugln("raft: Log error: ", err)
  374. return fmt.Errorf("raft: Initialization error: %s", err)
  375. }
  376. // Update the term to the last term in the log.
  377. _, s.currentTerm = s.log.lastInfo()
  378. s.setState(Follower)
  379. // If no log entries exist then
  380. // 1. wait for AEs from another node
  381. // 2. wait for self-join command
  382. // to set itself promotable
  383. if !s.promotable() {
  384. s.debugln("start as a new raft server")
  385. // If log entries exist then allow promotion to candidate
  386. // if no AEs received.
  387. } else {
  388. s.debugln("start from previous saved state")
  389. }
  390. debugln(s.GetState())
  391. go s.loop()
  392. return nil
  393. }
  394. // Shuts down the server.
  395. func (s *server) Stop() {
  396. s.send(&stopValue)
  397. // make sure the server has stopped before we close the log
  398. <-s.stopped
  399. s.log.close()
  400. }
  401. // Checks if the server is currently running.
  402. func (s *server) Running() bool {
  403. s.mutex.RLock()
  404. defer s.mutex.RUnlock()
  405. return s.state != Stopped
  406. }
  407. //--------------------------------------
  408. // Term
  409. //--------------------------------------
  410. // Sets the current term for the server. This is only used when an external
  411. // current term is found.
  412. func (s *server) setCurrentTerm(term uint64, leaderName string, append bool) {
  413. s.mutex.Lock()
  414. defer s.mutex.Unlock()
  415. // Store previous values temporarily.
  416. prevState := s.state
  417. prevTerm := s.currentTerm
  418. prevLeader := s.leader
  419. if term > s.currentTerm {
  420. // stop heartbeats before step-down
  421. if s.state == Leader {
  422. s.mutex.Unlock()
  423. for _, peer := range s.peers {
  424. peer.stopHeartbeat(false)
  425. }
  426. s.mutex.Lock()
  427. }
  428. // update the term and clear vote for
  429. s.state = Follower
  430. s.currentTerm = term
  431. s.leader = leaderName
  432. s.votedFor = ""
  433. } else if term == s.currentTerm && s.state != Leader && append {
  434. // discover new leader when candidate
  435. // save leader name when follower
  436. s.state = Follower
  437. s.leader = leaderName
  438. }
  439. // Dispatch change events.
  440. if prevState != s.state {
  441. s.DispatchEvent(newEvent(StateChangeEventType, s.state, prevState))
  442. }
  443. if prevLeader != s.leader {
  444. s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader))
  445. }
  446. if prevTerm != s.currentTerm {
  447. s.DispatchEvent(newEvent(TermChangeEventType, s.currentTerm, prevTerm))
  448. }
  449. }
  450. //--------------------------------------
  451. // Event Loop
  452. //--------------------------------------
  453. // ________
  454. // --|Snapshot| timeout
  455. // | -------- ______
  456. // recover | ^ | |
  457. // snapshot / | |snapshot | |
  458. // higher | | v | recv majority votes
  459. // term | -------- timeout ----------- -----------
  460. // |-> |Follower| ----------> | Candidate |--------------------> | Leader |
  461. // -------- ----------- -----------
  462. // ^ higher term/ | higher term |
  463. // | new leader | |
  464. // |_______________________|____________________________________ |
  465. // The main event loop for the server
  466. func (s *server) loop() {
  467. defer s.debugln("server.loop.end")
  468. for {
  469. state := s.State()
  470. s.debugln("server.loop.run ", state)
  471. switch state {
  472. case Follower:
  473. s.followerLoop()
  474. case Candidate:
  475. s.candidateLoop()
  476. case Leader:
  477. s.leaderLoop()
  478. case Snapshotting:
  479. s.snapshotLoop()
  480. case Stopped:
  481. s.stopped <- true
  482. return
  483. }
  484. }
  485. }
  486. // Sends an event to the event loop to be processed. The function will wait
  487. // until the event is actually processed before returning.
  488. func (s *server) send(value interface{}) (interface{}, error) {
  489. event := &ev{target: value, c: make(chan error, 1)}
  490. s.c <- event
  491. err := <-event.c
  492. return event.returnValue, err
  493. }
  494. func (s *server) sendAsync(value interface{}) {
  495. event := &ev{target: value, c: make(chan error, 1)}
  496. // try a non-blocking send first
  497. // in most cases, this should not be blocking
  498. // avoid create unnecessary go routines
  499. select {
  500. case s.c <- event:
  501. return
  502. default:
  503. }
  504. go func() {
  505. s.c <- event
  506. }()
  507. }
  508. // The event loop that is run when the server is in a Follower state.
  509. // Responds to RPCs from candidates and leaders.
  510. // Converts to candidate if election timeout elapses without either:
  511. // 1.Receiving valid AppendEntries RPC, or
  512. // 2.Granting vote to candidate
  513. func (s *server) followerLoop() {
  514. s.setState(Follower)
  515. since := time.Now()
  516. electionTimeout := s.ElectionTimeout()
  517. timeoutChan := afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2)
  518. for s.State() == Follower {
  519. var err error
  520. update := false
  521. select {
  522. case e := <-s.c:
  523. if e.target == &stopValue {
  524. s.setState(Stopped)
  525. } else {
  526. switch req := e.target.(type) {
  527. case JoinCommand:
  528. //If no log entries exist and a self-join command is issued
  529. //then immediately become leader and commit entry.
  530. if s.log.currentIndex() == 0 && req.NodeName() == s.Name() {
  531. s.debugln("selfjoin and promote to leader")
  532. s.setState(Leader)
  533. s.processCommand(req, e)
  534. } else {
  535. err = NotLeaderError
  536. }
  537. case *AppendEntriesRequest:
  538. // If heartbeats get too close to the election timeout then send an event.
  539. elapsedTime := time.Now().Sub(since)
  540. if elapsedTime > time.Duration(float64(electionTimeout)*ElectionTimeoutThresholdPercent) {
  541. s.DispatchEvent(newEvent(ElectionTimeoutThresholdEventType, elapsedTime, nil))
  542. }
  543. e.returnValue, update = s.processAppendEntriesRequest(req)
  544. case *RequestVoteRequest:
  545. e.returnValue, update = s.processRequestVoteRequest(req)
  546. case *SnapshotRequest:
  547. e.returnValue = s.processSnapshotRequest(req)
  548. default:
  549. err = NotLeaderError
  550. }
  551. }
  552. // Callback to event.
  553. e.c <- err
  554. case <-timeoutChan:
  555. // only allow synced follower to promote to candidate
  556. if s.promotable() {
  557. s.setState(Candidate)
  558. } else {
  559. update = true
  560. }
  561. }
  562. // Converts to candidate if election timeout elapses without either:
  563. // 1.Receiving valid AppendEntries RPC, or
  564. // 2.Granting vote to candidate
  565. if update {
  566. since = time.Now()
  567. timeoutChan = afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2)
  568. }
  569. }
  570. }
  571. // The event loop that is run when the server is in a Candidate state.
  572. func (s *server) candidateLoop() {
  573. lastLogIndex, lastLogTerm := s.log.lastInfo()
  574. // Clear leader value.
  575. prevLeader := s.leader
  576. s.leader = ""
  577. if prevLeader != s.leader {
  578. s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader))
  579. }
  580. for s.State() == Candidate {
  581. // Increment current term, vote for self.
  582. s.currentTerm++
  583. s.votedFor = s.name
  584. // Send RequestVote RPCs to all other servers.
  585. respChan := make(chan *RequestVoteResponse, len(s.peers))
  586. for _, peer := range s.peers {
  587. go peer.sendVoteRequest(newRequestVoteRequest(s.currentTerm, s.name, lastLogIndex, lastLogTerm), respChan)
  588. }
  589. // Wait for either:
  590. // * Votes received from majority of servers: become leader
  591. // * AppendEntries RPC received from new leader: step down.
  592. // * Election timeout elapses without election resolution: increment term, start new election
  593. // * Discover higher term: step down (§5.1)
  594. votesGranted := 1
  595. timeoutChan := afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2)
  596. timeout := false
  597. for {
  598. // If we received enough votes then stop waiting for more votes.
  599. s.debugln("server.candidate.votes: ", votesGranted, " quorum:", s.QuorumSize())
  600. if votesGranted >= s.QuorumSize() {
  601. s.setState(Leader)
  602. break
  603. }
  604. // Collect votes from peers.
  605. select {
  606. case resp := <-respChan:
  607. if resp.VoteGranted {
  608. s.debugln("server.candidate.vote.granted: ", votesGranted)
  609. votesGranted++
  610. } else if resp.Term > s.currentTerm {
  611. s.debugln("server.candidate.vote.failed")
  612. s.setCurrentTerm(resp.Term, "", false)
  613. } else {
  614. s.debugln("server.candidate.vote: denied")
  615. }
  616. case e := <-s.c:
  617. var err error
  618. if e.target == &stopValue {
  619. s.setState(Stopped)
  620. } else {
  621. switch req := e.target.(type) {
  622. case Command:
  623. err = NotLeaderError
  624. case *AppendEntriesRequest:
  625. e.returnValue, _ = s.processAppendEntriesRequest(req)
  626. case *RequestVoteRequest:
  627. e.returnValue, _ = s.processRequestVoteRequest(req)
  628. }
  629. }
  630. // Callback to event.
  631. e.c <- err
  632. case <-timeoutChan:
  633. timeout = true
  634. }
  635. // both process AER and RVR can make the server to follower
  636. // also break when timeout happens
  637. if s.State() != Candidate || timeout {
  638. break
  639. }
  640. }
  641. // continue when timeout happened
  642. }
  643. }
  644. // The event loop that is run when the server is in a Leader state.
  645. func (s *server) leaderLoop() {
  646. s.setState(Leader)
  647. logIndex, _ := s.log.lastInfo()
  648. // Update the peers prevLogIndex to leader's lastLogIndex and start heartbeat.
  649. s.debugln("leaderLoop.set.PrevIndex to ", logIndex)
  650. for _, peer := range s.peers {
  651. peer.setPrevLogIndex(logIndex)
  652. peer.startHeartbeat()
  653. }
  654. // Commit a NOP after the server becomes leader. From the Raft paper:
  655. // "Upon election: send initial empty AppendEntries RPCs (heartbeat) to
  656. // each server; repeat during idle periods to prevent election timeouts
  657. // (§5.2)". The heartbeats started above do the "idle" period work.
  658. go s.Do(NOPCommand{})
  659. // Begin to collect response from followers
  660. for s.State() == Leader {
  661. var err error
  662. select {
  663. case e := <-s.c:
  664. if e.target == &stopValue {
  665. // Stop all peers before stop
  666. for _, peer := range s.peers {
  667. peer.stopHeartbeat(false)
  668. }
  669. s.setState(Stopped)
  670. } else {
  671. switch req := e.target.(type) {
  672. case Command:
  673. s.processCommand(req, e)
  674. continue
  675. case *AppendEntriesRequest:
  676. e.returnValue, _ = s.processAppendEntriesRequest(req)
  677. case *AppendEntriesResponse:
  678. s.processAppendEntriesResponse(req)
  679. case *RequestVoteRequest:
  680. e.returnValue, _ = s.processRequestVoteRequest(req)
  681. }
  682. }
  683. // Callback to event.
  684. e.c <- err
  685. }
  686. }
  687. s.syncedPeer = nil
  688. }
  689. func (s *server) snapshotLoop() {
  690. s.setState(Snapshotting)
  691. for s.State() == Snapshotting {
  692. var err error
  693. e := <-s.c
  694. if e.target == &stopValue {
  695. s.setState(Stopped)
  696. } else {
  697. switch req := e.target.(type) {
  698. case Command:
  699. err = NotLeaderError
  700. case *AppendEntriesRequest:
  701. e.returnValue, _ = s.processAppendEntriesRequest(req)
  702. case *RequestVoteRequest:
  703. e.returnValue, _ = s.processRequestVoteRequest(req)
  704. case *SnapshotRecoveryRequest:
  705. e.returnValue = s.processSnapshotRecoveryRequest(req)
  706. }
  707. }
  708. // Callback to event.
  709. e.c <- err
  710. }
  711. }
  712. //--------------------------------------
  713. // Commands
  714. //--------------------------------------
  715. // Attempts to execute a command and replicate it. The function will return
  716. // when the command has been successfully committed or an error has occurred.
  717. func (s *server) Do(command Command) (interface{}, error) {
  718. return s.send(command)
  719. }
  720. // Processes a command.
  721. func (s *server) processCommand(command Command, e *ev) {
  722. s.debugln("server.command.process")
  723. // Create an entry for the command in the log.
  724. entry, err := s.log.createEntry(s.currentTerm, command, e)
  725. if err != nil {
  726. s.debugln("server.command.log.entry.error:", err)
  727. e.c <- err
  728. return
  729. }
  730. if err := s.log.appendEntry(entry); err != nil {
  731. s.debugln("server.command.log.error:", err)
  732. e.c <- err
  733. return
  734. }
  735. s.syncedPeer[s.Name()] = true
  736. if len(s.peers) == 0 {
  737. commitIndex := s.log.currentIndex()
  738. s.log.setCommitIndex(commitIndex)
  739. s.debugln("commit index ", commitIndex)
  740. }
  741. }
  742. //--------------------------------------
  743. // Append Entries
  744. //--------------------------------------
  745. // Appends zero or more log entry from the leader to this server.
  746. func (s *server) AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse {
  747. ret, _ := s.send(req)
  748. resp, _ := ret.(*AppendEntriesResponse)
  749. return resp
  750. }
  751. // Processes the "append entries" request.
  752. func (s *server) processAppendEntriesRequest(req *AppendEntriesRequest) (*AppendEntriesResponse, bool) {
  753. s.traceln("server.ae.process")
  754. if req.Term < s.currentTerm {
  755. s.debugln("server.ae.error: stale term")
  756. return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), false
  757. }
  758. // Update term and leader.
  759. s.setCurrentTerm(req.Term, req.LeaderName, true)
  760. // Reject if log doesn't contain a matching previous entry.
  761. if err := s.log.truncate(req.PrevLogIndex, req.PrevLogTerm); err != nil {
  762. s.debugln("server.ae.truncate.error: ", err)
  763. return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), true
  764. }
  765. // Append entries to the log.
  766. if err := s.log.appendEntries(req.Entries); err != nil {
  767. s.debugln("server.ae.append.error: ", err)
  768. return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), true
  769. }
  770. // Commit up to the commit index.
  771. if err := s.log.setCommitIndex(req.CommitIndex); err != nil {
  772. s.debugln("server.ae.commit.error: ", err)
  773. return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), true
  774. }
  775. // once the server appended and committed all the log entries from the leader
  776. return newAppendEntriesResponse(s.currentTerm, true, s.log.currentIndex(), s.log.CommitIndex()), true
  777. }
  778. // Processes the "append entries" response from the peer. This is only
  779. // processed when the server is a leader. Responses received during other
  780. // states are dropped.
  781. func (s *server) processAppendEntriesResponse(resp *AppendEntriesResponse) {
  782. // If we find a higher term then change to a follower and exit.
  783. if resp.Term() > s.Term() {
  784. s.setCurrentTerm(resp.Term(), "", false)
  785. return
  786. }
  787. // panic response if it's not successful.
  788. if !resp.Success() {
  789. return
  790. }
  791. // if one peer successfully append a log from the leader term,
  792. // we add it to the synced list
  793. if resp.append == true {
  794. s.syncedPeer[resp.peer] = true
  795. }
  796. // Increment the commit count to make sure we have a quorum before committing.
  797. if len(s.syncedPeer) < s.QuorumSize() {
  798. return
  799. }
  800. // Determine the committed index that a majority has.
  801. var indices []uint64
  802. indices = append(indices, s.log.currentIndex())
  803. for _, peer := range s.peers {
  804. indices = append(indices, peer.getPrevLogIndex())
  805. }
  806. sort.Sort(sort.Reverse(uint64Slice(indices)))
  807. // We can commit up to the index which the majority of the members have appended.
  808. commitIndex := indices[s.QuorumSize()-1]
  809. committedIndex := s.log.commitIndex
  810. if commitIndex > committedIndex {
  811. // leader needs to do a fsync before committing log entries
  812. s.log.sync()
  813. s.log.setCommitIndex(commitIndex)
  814. s.debugln("commit index ", commitIndex)
  815. }
  816. }
  817. //--------------------------------------
  818. // Request Vote
  819. //--------------------------------------
  820. // Requests a vote from a server. A vote can be obtained if the vote's term is
  821. // at the server's current term and the server has not made a vote yet. A vote
  822. // can also be obtained if the term is greater than the server's current term.
  823. func (s *server) RequestVote(req *RequestVoteRequest) *RequestVoteResponse {
  824. ret, _ := s.send(req)
  825. resp, _ := ret.(*RequestVoteResponse)
  826. return resp
  827. }
  828. // Processes a "request vote" request.
  829. func (s *server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVoteResponse, bool) {
  830. // If the request is coming from an old term then reject it.
  831. if req.Term < s.Term() {
  832. s.debugln("server.rv.deny.vote: cause stale term")
  833. return newRequestVoteResponse(s.currentTerm, false), false
  834. }
  835. s.setCurrentTerm(req.Term, "", false)
  836. // If we've already voted for a different candidate then don't vote for this candidate.
  837. if s.votedFor != "" && s.votedFor != req.CandidateName {
  838. s.debugln("server.deny.vote: cause duplicate vote: ", req.CandidateName,
  839. " already vote for ", s.votedFor)
  840. return newRequestVoteResponse(s.currentTerm, false), false
  841. }
  842. // If the candidate's log is not at least as up-to-date as our last log then don't vote.
  843. lastIndex, lastTerm := s.log.lastInfo()
  844. if lastIndex > req.LastLogIndex || lastTerm > req.LastLogTerm {
  845. s.debugln("server.deny.vote: cause out of date log: ", req.CandidateName,
  846. "Index :[", lastIndex, "]", " [", req.LastLogIndex, "]",
  847. "Term :[", lastTerm, "]", " [", req.LastLogTerm, "]")
  848. return newRequestVoteResponse(s.currentTerm, false), false
  849. }
  850. // If we made it this far then cast a vote and reset our election time out.
  851. s.debugln("server.rv.vote: ", s.name, " votes for", req.CandidateName, "at term", req.Term)
  852. s.votedFor = req.CandidateName
  853. return newRequestVoteResponse(s.currentTerm, true), true
  854. }
  855. //--------------------------------------
  856. // Membership
  857. //--------------------------------------
  858. // Adds a peer to the server.
  859. func (s *server) AddPeer(name string, connectiongString string) error {
  860. s.debugln("server.peer.add: ", name, len(s.peers))
  861. // Do not allow peers to be added twice.
  862. if s.peers[name] != nil {
  863. return nil
  864. }
  865. // Skip the Peer if it has the same name as the Server
  866. if s.name != name {
  867. peer := newPeer(s, name, connectiongString, s.heartbeatInterval)
  868. if s.State() == Leader {
  869. peer.startHeartbeat()
  870. }
  871. s.peers[peer.Name] = peer
  872. s.DispatchEvent(newEvent(AddPeerEventType, name, nil))
  873. }
  874. // Write the configuration to file.
  875. s.writeConf()
  876. return nil
  877. }
  878. // Removes a peer from the server.
  879. func (s *server) RemovePeer(name string) error {
  880. s.debugln("server.peer.remove: ", name, len(s.peers))
  881. // Skip the Peer if it has the same name as the Server
  882. if name != s.Name() {
  883. // Return error if peer doesn't exist.
  884. peer := s.peers[name]
  885. if peer == nil {
  886. return fmt.Errorf("raft: Peer not found: %s", name)
  887. }
  888. // Stop peer and remove it.
  889. if s.State() == Leader {
  890. peer.stopHeartbeat(true)
  891. }
  892. delete(s.peers, name)
  893. s.DispatchEvent(newEvent(RemovePeerEventType, name, nil))
  894. }
  895. // Write the configuration to file.
  896. s.writeConf()
  897. return nil
  898. }
  899. //--------------------------------------
  900. // Log compaction
  901. //--------------------------------------
  902. func (s *server) TakeSnapshot() error {
  903. // TODO: put a snapshot mutex
  904. s.debugln("take Snapshot")
  905. // Exit if the server is currently creating a snapshot.
  906. if s.currentSnapshot != nil {
  907. return errors.New("handling snapshot")
  908. }
  909. // Exit if there are no logs yet in the system.
  910. lastIndex, lastTerm := s.log.commitInfo()
  911. path := s.SnapshotPath(lastIndex, lastTerm)
  912. if lastIndex == 0 {
  913. return errors.New("No logs")
  914. }
  915. var state []byte
  916. var err error
  917. if s.stateMachine != nil {
  918. state, err = s.stateMachine.Save()
  919. if err != nil {
  920. return err
  921. }
  922. } else {
  923. state = []byte{0}
  924. }
  925. // Clone the list of peers.
  926. peers := make([]*Peer, 0, len(s.peers)+1)
  927. for _, peer := range s.peers {
  928. peers = append(peers, peer.clone())
  929. }
  930. peers = append(peers, &Peer{Name: s.Name(), ConnectionString: s.connectionString})
  931. // Attach current snapshot and save it to disk.
  932. s.currentSnapshot = &Snapshot{lastIndex, lastTerm, peers, state, path}
  933. s.saveSnapshot()
  934. // We keep some log entries after the snapshot.
  935. // We do not want to send the whole snapshot to the slightly slow machines
  936. if lastIndex-s.log.startIndex > NumberOfLogEntriesAfterSnapshot {
  937. compactIndex := lastIndex - NumberOfLogEntriesAfterSnapshot
  938. compactTerm := s.log.getEntry(compactIndex).Term()
  939. s.log.compact(compactIndex, compactTerm)
  940. }
  941. return nil
  942. }
  943. // Retrieves the log path for the server.
  944. func (s *server) saveSnapshot() error {
  945. if s.currentSnapshot == nil {
  946. return errors.New("no snapshot to save")
  947. }
  948. // Write snapshot to disk.
  949. if err := s.currentSnapshot.save(); err != nil {
  950. return err
  951. }
  952. // Swap the current and last snapshots.
  953. tmp := s.lastSnapshot
  954. s.lastSnapshot = s.currentSnapshot
  955. // Delete the previous snapshot if there is any change
  956. if tmp != nil && !(tmp.LastIndex == s.lastSnapshot.LastIndex && tmp.LastTerm == s.lastSnapshot.LastTerm) {
  957. tmp.remove()
  958. }
  959. s.currentSnapshot = nil
  960. return nil
  961. }
  962. // Retrieves the log path for the server.
  963. func (s *server) SnapshotPath(lastIndex uint64, lastTerm uint64) string {
  964. return path.Join(s.path, "snapshot", fmt.Sprintf("%v_%v.ss", lastTerm, lastIndex))
  965. }
  966. func (s *server) RequestSnapshot(req *SnapshotRequest) *SnapshotResponse {
  967. ret, _ := s.send(req)
  968. resp, _ := ret.(*SnapshotResponse)
  969. return resp
  970. }
  971. func (s *server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse {
  972. // If the follower’s log contains an entry at the snapshot’s last index with a term
  973. // that matches the snapshot’s last term, then the follower already has all the
  974. // information found in the snapshot and can reply false.
  975. entry := s.log.getEntry(req.LastIndex)
  976. if entry != nil && entry.Term() == req.LastTerm {
  977. return newSnapshotResponse(false)
  978. }
  979. // Update state.
  980. s.setState(Snapshotting)
  981. return newSnapshotResponse(true)
  982. }
  983. func (s *server) SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
  984. ret, _ := s.send(req)
  985. resp, _ := ret.(*SnapshotRecoveryResponse)
  986. return resp
  987. }
  988. func (s *server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
  989. // Recover state sent from request.
  990. if err := s.stateMachine.Recovery(req.State); err != nil {
  991. return newSnapshotRecoveryResponse(req.LastTerm, false, req.LastIndex)
  992. }
  993. // Recover the cluster configuration.
  994. s.peers = make(map[string]*Peer)
  995. for _, peer := range req.Peers {
  996. s.AddPeer(peer.Name, peer.ConnectionString)
  997. }
  998. // Update log state.
  999. s.currentTerm = req.LastTerm
  1000. s.log.updateCommitIndex(req.LastIndex)
  1001. // Create local snapshot.
  1002. s.currentSnapshot = &Snapshot{req.LastIndex, req.LastTerm, req.Peers, req.State, s.SnapshotPath(req.LastIndex, req.LastTerm)}
  1003. s.saveSnapshot()
  1004. // Clear the previous log entries.
  1005. s.log.compact(req.LastIndex, req.LastTerm)
  1006. return newSnapshotRecoveryResponse(req.LastTerm, true, req.LastIndex)
  1007. }
  1008. // Load a snapshot at restart
  1009. func (s *server) LoadSnapshot() error {
  1010. // Open snapshot/ directory.
  1011. dir, err := os.OpenFile(path.Join(s.path, "snapshot"), os.O_RDONLY, 0)
  1012. if err != nil {
  1013. return err
  1014. }
  1015. // Retrieve a list of all snapshots.
  1016. filenames, err := dir.Readdirnames(-1)
  1017. if err != nil {
  1018. dir.Close()
  1019. panic(err)
  1020. }
  1021. dir.Close()
  1022. if len(filenames) == 0 {
  1023. return errors.New("no snapshot")
  1024. }
  1025. // Grab the latest snapshot.
  1026. sort.Strings(filenames)
  1027. snapshotPath := path.Join(s.path, "snapshot", filenames[len(filenames)-1])
  1028. // Read snapshot data.
  1029. file, err := os.OpenFile(snapshotPath, os.O_RDONLY, 0)
  1030. if err != nil {
  1031. return err
  1032. }
  1033. defer file.Close()
  1034. // Check checksum.
  1035. var checksum uint32
  1036. n, err := fmt.Fscanf(file, "%08x\n", &checksum)
  1037. if err != nil {
  1038. return err
  1039. } else if n != 1 {
  1040. return errors.New("Bad snapshot file")
  1041. }
  1042. // Load remaining snapshot contents.
  1043. b, err := ioutil.ReadAll(file)
  1044. if err != nil {
  1045. return err
  1046. }
  1047. // Generate checksum.
  1048. byteChecksum := crc32.ChecksumIEEE(b)
  1049. if uint32(checksum) != byteChecksum {
  1050. s.debugln(checksum, " ", byteChecksum)
  1051. return errors.New("bad snapshot file")
  1052. }
  1053. // Decode snapshot.
  1054. if err = json.Unmarshal(b, &s.lastSnapshot); err != nil {
  1055. s.debugln("unmarshal error: ", err)
  1056. return err
  1057. }
  1058. // Recover snapshot into state machine.
  1059. if err = s.stateMachine.Recovery(s.lastSnapshot.State); err != nil {
  1060. s.debugln("recovery error: ", err)
  1061. return err
  1062. }
  1063. // Recover cluster configuration.
  1064. for _, peer := range s.lastSnapshot.Peers {
  1065. s.AddPeer(peer.Name, peer.ConnectionString)
  1066. }
  1067. // Update log state.
  1068. s.log.startTerm = s.lastSnapshot.LastTerm
  1069. s.log.startIndex = s.lastSnapshot.LastIndex
  1070. s.log.updateCommitIndex(s.lastSnapshot.LastIndex)
  1071. return err
  1072. }
  1073. //--------------------------------------
  1074. // Config File
  1075. //--------------------------------------
  1076. func (s *server) writeConf() {
  1077. peers := make([]*Peer, len(s.peers))
  1078. i := 0
  1079. for _, peer := range s.peers {
  1080. peers[i] = peer.clone()
  1081. i++
  1082. }
  1083. r := &Config{
  1084. CommitIndex: s.log.commitIndex,
  1085. Peers: peers,
  1086. }
  1087. b, _ := json.Marshal(r)
  1088. confPath := path.Join(s.path, "conf")
  1089. tmpConfPath := path.Join(s.path, "conf.tmp")
  1090. err := writeFileSynced(tmpConfPath, b, 0600)
  1091. if err != nil {
  1092. panic(err)
  1093. }
  1094. os.Rename(tmpConfPath, confPath)
  1095. }
  1096. // Read the configuration for the server.
  1097. func (s *server) readConf() error {
  1098. confPath := path.Join(s.path, "conf")
  1099. s.debugln("readConf.open ", confPath)
  1100. // open conf file
  1101. b, err := ioutil.ReadFile(confPath)
  1102. if err != nil {
  1103. return nil
  1104. }
  1105. conf := &Config{}
  1106. if err = json.Unmarshal(b, conf); err != nil {
  1107. return err
  1108. }
  1109. s.log.updateCommitIndex(conf.CommitIndex)
  1110. return nil
  1111. }
  1112. //--------------------------------------
  1113. // Debugging
  1114. //--------------------------------------
  1115. func (s *server) debugln(v ...interface{}) {
  1116. if logLevel > Debug {
  1117. debugf("[%s Term:%d] %s", s.name, s.Term(), fmt.Sprintln(v...))
  1118. }
  1119. }
  1120. func (s *server) traceln(v ...interface{}) {
  1121. if logLevel > Trace {
  1122. tracef("[%s] %s", s.name, fmt.Sprintln(v...))
  1123. }
  1124. }