peer.go 8.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302
  1. package raft
  2. import (
  3. "sync"
  4. "time"
  5. )
  6. //------------------------------------------------------------------------------
  7. //
  8. // Typedefs
  9. //
  10. //------------------------------------------------------------------------------
  11. // A peer is a reference to another server involved in the consensus protocol.
  12. type Peer struct {
  13. server *server
  14. Name string `json:"name"`
  15. ConnectionString string `json:"connectionString"`
  16. prevLogIndex uint64
  17. mutex sync.RWMutex
  18. stopChan chan bool
  19. heartbeatInterval time.Duration
  20. lastActivity time.Time
  21. }
  22. //------------------------------------------------------------------------------
  23. //
  24. // Constructor
  25. //
  26. //------------------------------------------------------------------------------
  27. // Creates a new peer.
  28. func newPeer(server *server, name string, connectionString string, heartbeatInterval time.Duration) *Peer {
  29. return &Peer{
  30. server: server,
  31. Name: name,
  32. ConnectionString: connectionString,
  33. heartbeatInterval: heartbeatInterval,
  34. }
  35. }
  36. //------------------------------------------------------------------------------
  37. //
  38. // Accessors
  39. //
  40. //------------------------------------------------------------------------------
  41. // Sets the heartbeat timeout.
  42. func (p *Peer) setHeartbeatInterval(duration time.Duration) {
  43. p.heartbeatInterval = duration
  44. }
  45. //--------------------------------------
  46. // Prev log index
  47. //--------------------------------------
  48. // Retrieves the previous log index.
  49. func (p *Peer) getPrevLogIndex() uint64 {
  50. p.mutex.RLock()
  51. defer p.mutex.RUnlock()
  52. return p.prevLogIndex
  53. }
  54. // Sets the previous log index.
  55. func (p *Peer) setPrevLogIndex(value uint64) {
  56. p.mutex.Lock()
  57. defer p.mutex.Unlock()
  58. p.prevLogIndex = value
  59. }
  60. // LastActivity returns the last time any response was received from the peer.
  61. func (p *Peer) LastActivity() time.Time {
  62. return p.lastActivity
  63. }
  64. //------------------------------------------------------------------------------
  65. //
  66. // Methods
  67. //
  68. //------------------------------------------------------------------------------
  69. //--------------------------------------
  70. // Heartbeat
  71. //--------------------------------------
  72. // Starts the peer heartbeat.
  73. func (p *Peer) startHeartbeat() {
  74. p.stopChan = make(chan bool)
  75. c := make(chan bool)
  76. go p.heartbeat(c)
  77. <-c
  78. }
  79. // Stops the peer heartbeat.
  80. func (p *Peer) stopHeartbeat(flush bool) {
  81. p.stopChan <- flush
  82. }
  83. //--------------------------------------
  84. // Copying
  85. //--------------------------------------
  86. // Clones the state of the peer. The clone is not attached to a server and
  87. // the heartbeat timer will not exist.
  88. func (p *Peer) clone() *Peer {
  89. p.mutex.Lock()
  90. defer p.mutex.Unlock()
  91. return &Peer{
  92. Name: p.Name,
  93. ConnectionString: p.ConnectionString,
  94. prevLogIndex: p.prevLogIndex,
  95. lastActivity: p.lastActivity,
  96. }
  97. }
  98. //--------------------------------------
  99. // Heartbeat
  100. //--------------------------------------
  101. // Listens to the heartbeat timeout and flushes an AppendEntries RPC.
  102. func (p *Peer) heartbeat(c chan bool) {
  103. stopChan := p.stopChan
  104. c <- true
  105. ticker := time.Tick(p.heartbeatInterval)
  106. debugln("peer.heartbeat: ", p.Name, p.heartbeatInterval)
  107. for {
  108. select {
  109. case flush := <-stopChan:
  110. if flush {
  111. // before we can safely remove a node
  112. // we must flush the remove command to the node first
  113. p.flush()
  114. debugln("peer.heartbeat.stop.with.flush: ", p.Name)
  115. return
  116. } else {
  117. debugln("peer.heartbeat.stop: ", p.Name)
  118. return
  119. }
  120. case <-ticker:
  121. start := time.Now()
  122. p.flush()
  123. duration := time.Now().Sub(start)
  124. p.server.DispatchEvent(newEvent(HeartbeatEventType, duration, nil))
  125. }
  126. }
  127. }
  128. func (p *Peer) flush() {
  129. debugln("peer.heartbeat.flush: ", p.Name)
  130. prevLogIndex := p.getPrevLogIndex()
  131. term := p.server.currentTerm
  132. entries, prevLogTerm := p.server.log.getEntriesAfter(prevLogIndex, p.server.maxLogEntriesPerRequest)
  133. if entries != nil {
  134. p.sendAppendEntriesRequest(newAppendEntriesRequest(term, prevLogIndex, prevLogTerm, p.server.log.CommitIndex(), p.server.name, entries))
  135. } else {
  136. p.sendSnapshotRequest(newSnapshotRequest(p.server.name, p.server.lastSnapshot))
  137. }
  138. }
  139. //--------------------------------------
  140. // Append Entries
  141. //--------------------------------------
  142. // Sends an AppendEntries request to the peer through the transport.
  143. func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
  144. tracef("peer.append.send: %s->%s [prevLog:%v length: %v]\n",
  145. p.server.Name(), p.Name, req.PrevLogIndex, len(req.Entries))
  146. resp := p.server.Transporter().SendAppendEntriesRequest(p.server, p, req)
  147. if resp == nil {
  148. p.server.DispatchEvent(newEvent(HeartbeatIntervalEventType, p, nil))
  149. debugln("peer.append.timeout: ", p.server.Name(), "->", p.Name)
  150. return
  151. }
  152. traceln("peer.append.resp: ", p.server.Name(), "<-", p.Name)
  153. // If successful then update the previous log index.
  154. p.mutex.Lock()
  155. p.lastActivity = time.Now()
  156. if resp.Success() {
  157. if len(req.Entries) > 0 {
  158. p.prevLogIndex = req.Entries[len(req.Entries)-1].GetIndex()
  159. // if peer append a log entry from the current term
  160. // we set append to true
  161. if req.Entries[len(req.Entries)-1].GetTerm() == p.server.currentTerm {
  162. resp.append = true
  163. }
  164. }
  165. traceln("peer.append.resp.success: ", p.Name, "; idx =", p.prevLogIndex)
  166. // If it was unsuccessful then decrement the previous log index and
  167. // we'll try again next time.
  168. } else {
  169. if resp.Term() > p.server.Term() {
  170. // this happens when there is a new leader comes up that this *leader* has not
  171. // known yet.
  172. // this server can know until the new leader send a ae with higher term
  173. // or this server finish processing this response.
  174. debugln("peer.append.resp.not.update: new.leader.found")
  175. } else if resp.Term() == req.Term && resp.CommitIndex() >= p.prevLogIndex {
  176. // we may miss a response from peer
  177. // so maybe the peer has committed the logs we just sent
  178. // but we did not receive the successful reply and did not increase
  179. // the prevLogIndex
  180. // peer failed to truncate the log and sent a fail reply at this time
  181. // we just need to update peer's prevLog index to commitIndex
  182. p.prevLogIndex = resp.CommitIndex()
  183. debugln("peer.append.resp.update: ", p.Name, "; idx =", p.prevLogIndex)
  184. } else if p.prevLogIndex > 0 {
  185. // Decrement the previous log index down until we find a match. Don't
  186. // let it go below where the peer's commit index is though. That's a
  187. // problem.
  188. p.prevLogIndex--
  189. // if it not enough, we directly decrease to the index of the
  190. if p.prevLogIndex > resp.Index() {
  191. p.prevLogIndex = resp.Index()
  192. }
  193. debugln("peer.append.resp.decrement: ", p.Name, "; idx =", p.prevLogIndex)
  194. }
  195. }
  196. p.mutex.Unlock()
  197. // Attach the peer to resp, thus server can know where it comes from
  198. resp.peer = p.Name
  199. // Send response to server for processing.
  200. p.server.sendAsync(resp)
  201. }
  202. // Sends an Snapshot request to the peer through the transport.
  203. func (p *Peer) sendSnapshotRequest(req *SnapshotRequest) {
  204. debugln("peer.snap.send: ", p.Name)
  205. resp := p.server.Transporter().SendSnapshotRequest(p.server, p, req)
  206. if resp == nil {
  207. debugln("peer.snap.timeout: ", p.Name)
  208. return
  209. }
  210. debugln("peer.snap.recv: ", p.Name)
  211. // If successful, the peer should have been to snapshot state
  212. // Send it the snapshot!
  213. p.lastActivity = time.Now()
  214. if resp.Success {
  215. p.sendSnapshotRecoveryRequest()
  216. } else {
  217. debugln("peer.snap.failed: ", p.Name)
  218. return
  219. }
  220. }
  221. // Sends an Snapshot Recovery request to the peer through the transport.
  222. func (p *Peer) sendSnapshotRecoveryRequest() {
  223. req := newSnapshotRecoveryRequest(p.server.name, p.server.lastSnapshot)
  224. debugln("peer.snap.recovery.send: ", p.Name)
  225. resp := p.server.Transporter().SendSnapshotRecoveryRequest(p.server, p, req)
  226. if resp == nil {
  227. debugln("peer.snap.recovery.timeout: ", p.Name)
  228. return
  229. }
  230. p.lastActivity = time.Now()
  231. if resp.Success {
  232. p.prevLogIndex = req.LastIndex
  233. } else {
  234. debugln("peer.snap.recovery.failed: ", p.Name)
  235. return
  236. }
  237. p.server.sendAsync(resp)
  238. }
  239. //--------------------------------------
  240. // Vote Requests
  241. //--------------------------------------
  242. // send VoteRequest Request
  243. func (p *Peer) sendVoteRequest(req *RequestVoteRequest, c chan *RequestVoteResponse) {
  244. debugln("peer.vote: ", p.server.Name(), "->", p.Name)
  245. req.peer = p
  246. if resp := p.server.Transporter().SendVoteRequest(p.server, p, req); resp != nil {
  247. debugln("peer.vote.recv: ", p.server.Name(), "<-", p.Name)
  248. p.lastActivity = time.Now()
  249. resp.peer = p
  250. c <- resp
  251. } else {
  252. debugln("peer.vote.failed: ", p.server.Name(), "<-", p.Name)
  253. }
  254. }