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