forked from etcd-io/etcd
-
Notifications
You must be signed in to change notification settings - Fork 0
/
raft.go
449 lines (407 loc) · 12 KB
/
raft.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
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
// Copyright 2015 CoreOS, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package etcdserver
import (
"encoding/json"
"expvar"
"os"
"sort"
"sync"
"sync/atomic"
"time"
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
"github.com/coreos/etcd/pkg/pbutil"
"github.com/coreos/etcd/pkg/types"
"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"github.com/coreos/etcd/rafthttp"
"github.com/coreos/etcd/wal"
"github.com/coreos/etcd/wal/walpb"
"github.com/coreos/etcd/Godeps/_workspace/src/github.com/coreos/pkg/capnslog"
)
const (
// Number of entries for slow follower to catch-up after compacting
// the raft storage entries.
// We expect the follower has a millisecond level latency with the leader.
// The max throughput is around 10K. Keep a 5K entries is enough for helping
// follower to catch up.
numberOfCatchUpEntries = 5000
// The max throughput of etcd will not exceed 100MB/s (100K * 1KB value).
// Assuming the RTT is around 10ms, 1MB max size is large enough.
maxSizePerMsg = 1 * 1024 * 1024
// Never overflow the rafthttp buffer, which is 4096.
// TODO: a better const?
maxInflightMsgs = 4096 / 8
)
var (
// indirection for expvar func interface
// expvar panics when publishing duplicate name
// expvar does not support remove a registered name
// so only register a func that calls raftStatus
// and change raftStatus as we need.
raftStatus func() raft.Status
)
func init() {
raft.SetLogger(capnslog.NewPackageLogger("github.com/coreos/etcd", "raft"))
expvar.Publish("raft.status", expvar.Func(func() interface{} { return raftStatus() }))
}
type RaftTimer interface {
Index() uint64
Term() uint64
}
// apply contains entries, snapshot be applied.
// After applied all the items, the application needs
// to send notification to done chan.
type apply struct {
entries []raftpb.Entry
snapshot raftpb.Snapshot
done chan struct{}
}
type raftNode struct {
// Cache of the latest raft index and raft term the server has seen.
// These three unit64 fields must be the first elements to keep 64-bit
// alignment for atomic access to the fields.
index uint64
term uint64
lead uint64
mu sync.Mutex
// last lead elected time
lt time.Time
raft.Node
// a chan to send out apply
applyc chan apply
// TODO: remove the etcdserver related logic from raftNode
// TODO: add a state machine interface to apply the commit entries
// and do snapshot/recover
s *EtcdServer
// utility
ticker <-chan time.Time
raftStorage *raft.MemoryStorage
storage Storage
// transport specifies the transport to send and receive msgs to members.
// Sending messages MUST NOT block. It is okay to drop messages, since
// clients should timeout and reissue their messages.
// If transport is nil, server will panic.
transport rafthttp.Transporter
stopped chan struct{}
done chan struct{}
}
// start prepares and starts raftNode in a new goroutine. It is no longer safe
// to modify the fields after it has been started.
// TODO: Ideally raftNode should get rid of the passed in server structure.
func (r *raftNode) start(s *EtcdServer) {
r.s = s
r.applyc = make(chan apply)
r.stopped = make(chan struct{})
r.done = make(chan struct{})
go func() {
var syncC <-chan time.Time
defer r.onStop()
for {
select {
case <-r.ticker:
r.Tick()
case rd := <-r.Ready():
if rd.SoftState != nil {
if lead := atomic.LoadUint64(&r.lead); rd.SoftState.Lead != raft.None && lead != rd.SoftState.Lead {
r.mu.Lock()
r.lt = time.Now()
r.mu.Unlock()
}
atomic.StoreUint64(&r.lead, rd.SoftState.Lead)
if rd.RaftState == raft.StateLeader {
syncC = r.s.SyncTicker
// TODO: remove the nil checking
// current test utility does not provide the stats
if r.s.stats != nil {
r.s.stats.BecomeLeader()
}
} else {
syncC = nil
}
}
apply := apply{
entries: rd.CommittedEntries,
snapshot: rd.Snapshot,
done: make(chan struct{}),
}
select {
case r.applyc <- apply:
case <-r.stopped:
return
}
if !raft.IsEmptySnap(rd.Snapshot) {
if err := r.storage.SaveSnap(rd.Snapshot); err != nil {
plog.Fatalf("raft save snapshot error: %v", err)
}
r.raftStorage.ApplySnapshot(rd.Snapshot)
plog.Infof("raft applied incoming snapshot at index %d", rd.Snapshot.Metadata.Index)
}
if err := r.storage.Save(rd.HardState, rd.Entries); err != nil {
plog.Fatalf("raft save state and entries error: %v", err)
}
r.raftStorage.Append(rd.Entries)
r.s.send(rd.Messages)
select {
case <-apply.done:
case <-r.stopped:
return
}
r.Advance()
case <-syncC:
r.s.sync(r.s.cfg.ReqTimeout())
case <-r.stopped:
return
}
}
}()
}
func (r *raftNode) apply() chan apply {
return r.applyc
}
func (r *raftNode) leadElectedTime() time.Time {
r.mu.Lock()
defer r.mu.Unlock()
return r.lt
}
func (r *raftNode) stop() {
r.stopped <- struct{}{}
<-r.done
}
func (r *raftNode) onStop() {
r.Stop()
r.transport.Stop()
if err := r.storage.Close(); err != nil {
plog.Panicf("raft close storage error: %v", err)
}
close(r.done)
}
// for testing
func (r *raftNode) pauseSending() {
p := r.transport.(rafthttp.Pausable)
p.Pause()
}
func (r *raftNode) resumeSending() {
p := r.transport.(rafthttp.Pausable)
p.Resume()
}
// advanceTicksForElection advances ticks to the node for fast election.
// This reduces the time to wait for first leader election if bootstrapping the whole
// cluster, while leaving at least 1 heartbeat for possible existing leader
// to contact it.
func advanceTicksForElection(n raft.Node, electionTicks int) {
for i := 0; i < electionTicks-1; i++ {
n.Tick()
}
}
func startNode(cfg *ServerConfig, cl *cluster, ids []types.ID) (id types.ID, n raft.Node, s *raft.MemoryStorage, w *wal.WAL) {
var err error
member := cl.MemberByName(cfg.Name)
metadata := pbutil.MustMarshal(
&pb.Metadata{
NodeID: uint64(member.ID),
ClusterID: uint64(cl.ID()),
},
)
if err := os.MkdirAll(cfg.SnapDir(), privateDirMode); err != nil {
plog.Fatalf("create snapshot directory error: %v", err)
}
if w, err = wal.Create(cfg.WALDir(), metadata); err != nil {
plog.Fatalf("create wal error: %v", err)
}
peers := make([]raft.Peer, len(ids))
for i, id := range ids {
ctx, err := json.Marshal((*cl).Member(id))
if err != nil {
plog.Panicf("marshal member should never fail: %v", err)
}
peers[i] = raft.Peer{ID: uint64(id), Context: ctx}
}
id = member.ID
plog.Infof("starting member %s in cluster %s", id, cl.ID())
s = raft.NewMemoryStorage()
c := &raft.Config{
ID: uint64(id),
ElectionTick: cfg.ElectionTicks,
HeartbeatTick: 1,
Storage: s,
MaxSizePerMsg: maxSizePerMsg,
MaxInflightMsgs: maxInflightMsgs,
}
n = raft.StartNode(c, peers)
raftStatus = n.Status
advanceTicksForElection(n, c.ElectionTick)
return
}
func restartNode(cfg *ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *cluster, raft.Node, *raft.MemoryStorage, *wal.WAL) {
var walsnap walpb.Snapshot
if snapshot != nil {
walsnap.Index, walsnap.Term = snapshot.Metadata.Index, snapshot.Metadata.Term
}
w, id, cid, st, ents := readWAL(cfg.WALDir(), walsnap)
plog.Infof("restarting member %s in cluster %s at commit index %d", id, cid, st.Commit)
cl := newCluster("")
cl.SetID(cid)
s := raft.NewMemoryStorage()
if snapshot != nil {
s.ApplySnapshot(*snapshot)
}
s.SetHardState(st)
s.Append(ents)
c := &raft.Config{
ID: uint64(id),
ElectionTick: cfg.ElectionTicks,
HeartbeatTick: 1,
Storage: s,
MaxSizePerMsg: maxSizePerMsg,
MaxInflightMsgs: maxInflightMsgs,
}
n := raft.RestartNode(c)
raftStatus = n.Status
advanceTicksForElection(n, c.ElectionTick)
return id, cl, n, s, w
}
func restartAsStandaloneNode(cfg *ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *cluster, raft.Node, *raft.MemoryStorage, *wal.WAL) {
var walsnap walpb.Snapshot
if snapshot != nil {
walsnap.Index, walsnap.Term = snapshot.Metadata.Index, snapshot.Metadata.Term
}
w, id, cid, st, ents := readWAL(cfg.WALDir(), walsnap)
// discard the previously uncommitted entries
for i, ent := range ents {
if ent.Index > st.Commit {
plog.Infof("discarding %d uncommitted WAL entries ", len(ents)-i)
ents = ents[:i]
break
}
}
// force append the configuration change entries
toAppEnts := createConfigChangeEnts(getIDs(snapshot, ents), uint64(id), st.Term, st.Commit)
ents = append(ents, toAppEnts...)
// force commit newly appended entries
err := w.Save(raftpb.HardState{}, toAppEnts)
if err != nil {
plog.Fatalf("%v", err)
}
if len(ents) != 0 {
st.Commit = ents[len(ents)-1].Index
}
plog.Printf("forcing restart of member %s in cluster %s at commit index %d", id, cid, st.Commit)
cl := newCluster("")
cl.SetID(cid)
s := raft.NewMemoryStorage()
if snapshot != nil {
s.ApplySnapshot(*snapshot)
}
s.SetHardState(st)
s.Append(ents)
c := &raft.Config{
ID: uint64(id),
ElectionTick: cfg.ElectionTicks,
HeartbeatTick: 1,
Storage: s,
MaxSizePerMsg: maxSizePerMsg,
MaxInflightMsgs: maxInflightMsgs,
}
n := raft.RestartNode(c)
raftStatus = n.Status
return id, cl, n, s, w
}
// getIDs returns an ordered set of IDs included in the given snapshot and
// the entries. The given snapshot/entries can contain two kinds of
// ID-related entry:
// - ConfChangeAddNode, in which case the contained ID will be added into the set.
// - ConfChangeAddRemove, in which case the contained ID will be removed from the set.
func getIDs(snap *raftpb.Snapshot, ents []raftpb.Entry) []uint64 {
ids := make(map[uint64]bool)
if snap != nil {
for _, id := range snap.Metadata.ConfState.Nodes {
ids[id] = true
}
}
for _, e := range ents {
if e.Type != raftpb.EntryConfChange {
continue
}
var cc raftpb.ConfChange
pbutil.MustUnmarshal(&cc, e.Data)
switch cc.Type {
case raftpb.ConfChangeAddNode:
ids[cc.NodeID] = true
case raftpb.ConfChangeRemoveNode:
delete(ids, cc.NodeID)
case raftpb.ConfChangeUpdateNode:
// do nothing
default:
plog.Panicf("ConfChange Type should be either ConfChangeAddNode or ConfChangeRemoveNode!")
}
}
sids := make(types.Uint64Slice, 0)
for id := range ids {
sids = append(sids, id)
}
sort.Sort(sids)
return []uint64(sids)
}
// createConfigChangeEnts creates a series of Raft entries (i.e.
// EntryConfChange) to remove the set of given IDs from the cluster. The ID
// `self` is _not_ removed, even if present in the set.
// If `self` is not inside the given ids, it creates a Raft entry to add a
// default member with the given `self`.
func createConfigChangeEnts(ids []uint64, self uint64, term, index uint64) []raftpb.Entry {
ents := make([]raftpb.Entry, 0)
next := index + 1
found := false
for _, id := range ids {
if id == self {
found = true
continue
}
cc := &raftpb.ConfChange{
Type: raftpb.ConfChangeRemoveNode,
NodeID: id,
}
e := raftpb.Entry{
Type: raftpb.EntryConfChange,
Data: pbutil.MustMarshal(cc),
Term: term,
Index: next,
}
ents = append(ents, e)
next++
}
if !found {
m := Member{
ID: types.ID(self),
RaftAttributes: RaftAttributes{PeerURLs: []string{"http://localhost:7001", "http://localhost:2380"}},
}
ctx, err := json.Marshal(m)
if err != nil {
plog.Panicf("marshal member should never fail: %v", err)
}
cc := &raftpb.ConfChange{
Type: raftpb.ConfChangeAddNode,
NodeID: self,
Context: ctx,
}
e := raftpb.Entry{
Type: raftpb.EntryConfChange,
Data: pbutil.MustMarshal(cc),
Term: term,
Index: next,
}
ents = append(ents, e)
}
return ents
}