forked from dgraph-io/dgraph
-
Notifications
You must be signed in to change notification settings - Fork 0
/
node.go
569 lines (513 loc) · 14.6 KB
/
node.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
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
/*
* Copyright (C) 2017 Dgraph Labs, Inc. and Contributors
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero General Public License as published by
* the Free Software Foundation, either version 3 of the License, or
* (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU Affero General Public License for more details.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
package conn
import (
"bytes"
"encoding/binary"
"fmt"
"log"
"sync"
"time"
"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"github.com/dgraph-io/badger/y"
"github.com/dgraph-io/dgraph/protos"
"github.com/dgraph-io/dgraph/raftwal"
"github.com/dgraph-io/dgraph/x"
"golang.org/x/net/context"
)
var (
errReadIndex = x.Errorf("cannot get linerized read (time expired or no configured leader)")
ErrDuplicateRaftId = x.Errorf("Node is already part of group")
)
type sendmsg struct {
to uint64
data []byte
}
type Node struct {
x.SafeMutex
// Changed after init but not protected by SafeMutex
RequestCh chan linReadReq
// SafeMutex is for fields which can be changed after init.
_confState *raftpb.ConfState
_raft raft.Node
// Fields which are never changed after init.
Cfg *raft.Config
MyAddr string
Id uint64
peers map[uint64]string
messages chan sendmsg
RaftContext *protos.RaftContext
Store *raft.MemoryStorage
Wal *raftwal.Wal
// applied is used to keep track of the applied RAFT proposals.
// The stages are proposed -> committed (accepted by cluster) ->
// applied (to PL) -> synced (to RocksDB).
Applied x.WaterMark
}
func NewNode(rc *protos.RaftContext) *Node {
store := raft.NewMemoryStorage()
n := &Node{
Id: rc.Id,
Store: store,
Cfg: &raft.Config{
ID: rc.Id,
ElectionTick: 100, // 200 ms if we call Tick() every 20 ms.
HeartbeatTick: 1, // 20 ms if we call Tick() every 20 ms.
Storage: store,
MaxSizePerMsg: 256 << 10,
MaxInflightMsgs: 256,
Logger: &raft.DefaultLogger{Logger: x.Logger},
// We use lease-based linearizable ReadIndex for performance, at the cost of
// correctness. With it, communication goes follower->leader->follower, instead of
// follower->leader->majority_of_followers->leader->follower. We lose correctness
// because the Raft ticker might not arrive promptly, in which case the leader would
// falsely believe that its lease is still good.
CheckQuorum: true,
ReadOnlyOption: raft.ReadOnlyLeaseBased,
},
// processConfChange etc are not throttled so some extra delta, so that we don't
// block tick when applyCh is full
peers: make(map[uint64]string),
RaftContext: rc,
messages: make(chan sendmsg, 100),
Applied: x.WaterMark{Name: fmt.Sprintf("Applied watermark")},
RequestCh: make(chan linReadReq, 100),
}
n.Applied.Init()
// TODO: n_ = n is a hack. We should properly init node, and make it part of the server struct.
// This can happen once we get rid of groups.
n_ = n
return n
}
func (n *Node) WaitLinearizableRead(ctx context.Context) error {
replyCh, err := n.ReadIndex(ctx)
if err != nil {
return err
}
select {
case index := <-replyCh:
if index == raft.None {
return errReadIndex
}
if err := n.Applied.WaitForMark(ctx, index); err != nil {
return err
}
return nil
case <-ctx.Done():
return ctx.Err()
}
}
// SetRaft would set the provided raft.Node to this node.
// It would check fail if the node is already set.
func (n *Node) SetRaft(r raft.Node) {
n.Lock()
defer n.Unlock()
x.AssertTrue(n._raft == nil)
n._raft = r
}
// Raft would return back the raft.Node stored in the node.
func (n *Node) Raft() raft.Node {
n.RLock()
defer n.RUnlock()
return n._raft
}
// SetConfState would store the latest ConfState generated by ApplyConfChange.
func (n *Node) SetConfState(cs *raftpb.ConfState) {
n.Lock()
defer n.Unlock()
n._confState = cs
}
// ConfState would return the latest ConfState stored in node.
func (n *Node) ConfState() *raftpb.ConfState {
n.RLock()
defer n.RUnlock()
return n._confState
}
func (n *Node) Peer(pid uint64) (string, bool) {
n.RLock()
defer n.RUnlock()
addr, ok := n.peers[pid]
return addr, ok
}
// addr must not be empty.
func (n *Node) SetPeer(pid uint64, addr string) {
x.AssertTruef(addr != "", "SetPeer for peer %d has empty addr.", pid)
Get().Connect(addr)
n.peers[pid] = addr
}
func (n *Node) Send(m raftpb.Message) {
x.AssertTruef(n.Id != m.To, "Seding message to itself")
data, err := m.Marshal()
x.Check(err)
if m.Type != raftpb.MsgHeartbeat && m.Type != raftpb.MsgHeartbeatResp {
x.Printf("\t\tSENDING: %v %v-->%v\n", m.Type, m.From, m.To)
}
select {
case n.messages <- sendmsg{to: m.To, data: data}:
// pass
default:
// ignore
}
}
func (n *Node) SaveToStorage(s raftpb.Snapshot, h raftpb.HardState,
es []raftpb.Entry) {
if !raft.IsEmptySnap(s) {
le, err := n.Store.LastIndex()
if err != nil {
log.Fatalf("While retrieving last index: %v\n", err)
}
if s.Metadata.Index <= le {
return
}
if err := n.Store.ApplySnapshot(s); err != nil {
log.Fatalf("Applying snapshot: %v", err)
}
}
if !raft.IsEmptyHardState(h) {
n.Store.SetHardState(h)
}
n.Store.Append(es)
}
func (n *Node) InitFromWal(wal *raftwal.Wal) (idx uint64, restart bool, rerr error) {
n.Wal = wal
var sp raftpb.Snapshot
sp, rerr = wal.Snapshot(n.RaftContext.Group)
if rerr != nil {
return
}
var term uint64
if !raft.IsEmptySnap(sp) {
x.Printf("Found Snapshot: %+v\n", sp)
restart = true
if rerr = n.Store.ApplySnapshot(sp); rerr != nil {
return
}
term = sp.Metadata.Term
idx = sp.Metadata.Index
}
var hd raftpb.HardState
hd, rerr = wal.HardState(n.RaftContext.Group)
if rerr != nil {
return
}
if !raft.IsEmptyHardState(hd) {
x.Printf("Found hardstate: %+v\n", hd)
restart = true
if rerr = n.Store.SetHardState(hd); rerr != nil {
return
}
}
var es []raftpb.Entry
es, rerr = wal.Entries(n.RaftContext.Group, term, idx)
if rerr != nil {
return
}
x.Printf("Group %d found %d entries\n", n.RaftContext.Group, len(es))
if len(es) > 0 {
restart = true
}
rerr = n.Store.Append(es)
return
}
const (
messageBatchSoftLimit = 10000000
)
func (n *Node) BatchAndSendMessages() {
batches := make(map[uint64]*bytes.Buffer)
for {
totalSize := 0
sm := <-n.messages
slurp_loop:
for {
var buf *bytes.Buffer
if b, ok := batches[sm.to]; !ok {
buf = new(bytes.Buffer)
batches[sm.to] = buf
} else {
buf = b
}
totalSize += 4 + len(sm.data)
x.Check(binary.Write(buf, binary.LittleEndian, uint32(len(sm.data))))
x.Check2(buf.Write(sm.data))
if totalSize > messageBatchSoftLimit {
// We limit the batch size, but we aren't pushing back on
// n.messages, because the loop below spawns a goroutine
// to do its dirty work. This is good because right now
// (*node).send fails(!) if the channel is full.
break
}
select {
case sm = <-n.messages:
default:
break slurp_loop
}
}
for to, buf := range batches {
if buf.Len() == 0 {
continue
}
data := make([]byte, buf.Len())
copy(data, buf.Bytes())
go n.doSendMessage(to, data)
buf.Reset()
}
}
}
func (n *Node) doSendMessage(to uint64, data []byte) {
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
defer cancel()
addr, has := n.peers[to]
pool, err := Get().Get(addr)
if !has || err != nil {
// No such peer exists or we got handed a bogus config (bad addr), so we
// can't send messages to this peer.
return
}
client := pool.Get()
c := protos.NewRaftClient(client)
p := &protos.Payload{Data: data}
ch := make(chan error, 1)
go func() {
_, err = c.RaftMessage(ctx, p)
ch <- err
}()
select {
case <-ctx.Done():
return
case <-ch:
// We don't need to do anything if we receive any error while sending message.
// RAFT would automatically retry.
return
}
}
// Connects the node and makes its peerPool refer to the constructed pool and address
// (possibly updating ourselves from the old address.) (Unless pid is ourselves, in which
// case this does nothing.)
func (n *Node) Connect(pid uint64, addr string) {
if pid == n.Id {
return
}
if paddr, ok := n.peers[pid]; ok && paddr == addr {
// Already connected.
return
}
// Here's what we do. Right now peerPool maps peer node id's to addr values. If
// a *pool can be created, good, but if not, we still create a peerPoolEntry with
// a nil *pool.
if addr == n.MyAddr {
// TODO: Note this fact in more general peer health info somehow.
x.Printf("Peer %d claims same host as me\n", pid)
n.peers[pid] = addr
return
}
Get().Connect(addr)
n.peers[pid] = addr
}
type linReadReq struct {
// A one-shot chan which we send a raft index upon
indexCh chan<- uint64
}
func (n *Node) ReadIndex(ctx context.Context) (chan uint64, error) {
ch := make(chan uint64, 1)
select {
case n.RequestCh <- linReadReq{ch}:
return ch, nil
case <-ctx.Done():
return nil, ctx.Err()
}
}
func (n *Node) RunReadIndexLoop(closer *y.Closer, readStateCh <-chan raft.ReadState) {
defer closer.Done()
counter := x.NewNonceCounter()
requests := []linReadReq{}
// We maintain one linearizable ReadIndex request at a time. Others wait queued behind
// requestCh.
for {
select {
case <-closer.HasBeenClosed():
return
case <-readStateCh:
// Do nothing, discard ReadState info we don't have an activeRctx for
case req := <-n.RequestCh:
slurpLoop:
for {
requests = append(requests, req)
select {
case req = <-n.RequestCh:
default:
break slurpLoop
}
}
activeRctx := counter.Generate()
// To see if the ReadIndex request succeeds, we need to use a timeout and wait for a
// successful response. If we don't see one, the raft leader wasn't configured, or the
// raft leader didn't respond.
// This is supposed to use context.Background(). We don't want to cancel the timer
// externally. We want equivalent functionality to time.NewTimer.
// TODO: Second is high, if a node gets partitioned we would have to throw error sooner.
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
// We ignore the err - it would be n.ctx cancellation (which we must ignore because
// it's our duty to continue until `stop` is triggered) or raft.ErrStopped (which we
// must ignore for the same reason).
_ = n.Raft().ReadIndex(ctx, activeRctx[:])
again:
select {
case <-closer.HasBeenClosed():
cancel()
return
case rs := <-readStateCh:
if 0 != bytes.Compare(activeRctx[:], rs.RequestCtx) {
goto again
}
cancel()
index := rs.Index
for _, req := range requests {
req.indexCh <- index
}
case <-ctx.Done():
for _, req := range requests {
req.indexCh <- raft.None
}
}
requests = requests[:0]
}
}
}
func (n *Node) AddToCluster(ctx context.Context, pid uint64) error {
addr, ok := n.peers[pid]
x.AssertTruef(ok, "Unable to find conn pool for peer: %d", pid)
rc := &protos.RaftContext{
Addr: addr,
Group: n.RaftContext.Group,
Id: pid,
}
rcBytes, err := rc.Marshal()
x.Check(err)
return n.Raft().ProposeConfChange(ctx, raftpb.ConfChange{
ID: pid,
Type: raftpb.ConfChangeAddNode,
NodeID: pid,
Context: rcBytes,
})
}
// TODO: Get rid of this in the upcoming changes.
var n_ *Node
func (w *RaftServer) GetNode() *Node {
w.nodeLock.RLock()
defer w.nodeLock.RUnlock()
return w.Node
}
type RaftServer struct {
nodeLock sync.RWMutex // protects Node.
Node *Node
}
func (w *RaftServer) JoinCluster(ctx context.Context,
rc *protos.RaftContext) (*protos.Payload, error) {
if ctx.Err() != nil {
return &protos.Payload{}, ctx.Err()
}
// Commenting out the following checks for now, until we get rid of groups.
// TODO: Uncomment this after groups is removed.
node := w.GetNode()
if node == nil || node.Raft() == nil {
return nil, errNoNode
}
// Check that the new node is from the same group as me.
if rc.Group != node.RaftContext.Group {
return nil, x.Errorf("Raft group mismatch")
}
// Also check that the new node is not me.
if rc.Id == node.RaftContext.Id {
return nil, ErrDuplicateRaftId
}
// Check that the new node is not already part of the group.
if addr, ok := node.peers[rc.Id]; ok {
Get().Connect(addr)
// There exists a healthy connection to server with same id.
if _, err := Get().Get(addr); err == nil {
return &protos.Payload{}, ErrDuplicateRaftId
}
}
node.Connect(rc.Id, rc.Addr)
c := make(chan error, 1)
go func() { c <- node.AddToCluster(ctx, rc.Id) }()
select {
case <-ctx.Done():
return &protos.Payload{}, ctx.Err()
case err := <-c:
return &protos.Payload{}, err
}
}
var (
errNoNode = fmt.Errorf("No node has been set up yet")
)
func (w *RaftServer) applyMessage(ctx context.Context, msg raftpb.Message) error {
var rc protos.RaftContext
x.Check(rc.Unmarshal(msg.Context))
node := w.GetNode()
if node == nil || node.Raft() == nil {
return errNoNode
}
if rc.Group != node.RaftContext.Group {
return errNoNode
}
node.Connect(msg.From, rc.Addr)
c := make(chan error, 1)
go func() { c <- node.Raft().Step(ctx, msg) }()
select {
case <-ctx.Done():
return ctx.Err()
case err := <-c:
return err
}
}
func (w *RaftServer) RaftMessage(ctx context.Context,
query *protos.Payload) (*protos.Payload, error) {
if ctx.Err() != nil {
return &protos.Payload{}, ctx.Err()
}
for idx := 0; idx < len(query.Data); {
x.AssertTruef(len(query.Data[idx:]) >= 4,
"Slice left of size: %v. Expected at least 4.", len(query.Data[idx:]))
sz := int(binary.LittleEndian.Uint32(query.Data[idx : idx+4]))
idx += 4
msg := raftpb.Message{}
if idx+sz > len(query.Data) {
return &protos.Payload{}, x.Errorf(
"Invalid query. Specified size %v overflows slice [%v,%v)\n",
sz, idx, len(query.Data))
}
if err := msg.Unmarshal(query.Data[idx : idx+sz]); err != nil {
x.Check(err)
}
if msg.Type != raftpb.MsgHeartbeat && msg.Type != raftpb.MsgHeartbeatResp {
x.Printf("RECEIVED: %v %v-->%v\n", msg.Type, msg.From, msg.To)
}
if err := w.applyMessage(ctx, msg); err != nil {
return &protos.Payload{}, err
}
idx += sz
}
// fmt.Printf("Got %d messages\n", count)
return &protos.Payload{}, nil
}
// Hello rpc call is used to check connection with other workers after worker
// tcp server for this instance starts.
func (w *RaftServer) Echo(ctx context.Context, in *protos.Payload) (*protos.Payload, error) {
return &protos.Payload{Data: in.Data}, nil
}