server.go 35 KB

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