peer.go 8.6 KB

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