forked from hashicorp/raft
/
net_transport.go
447 lines (376 loc) · 10.3 KB
/
net_transport.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
package raft
import (
"bufio"
"errors"
"fmt"
"io"
"log"
"net"
"os"
"sync"
"time"
"github.com/ugorji/go/codec"
)
const (
rpcAppendEntries uint8 = iota
rpcRequestVote
rpcInstallSnapshot
// DefaultTimeoutScale is the default TimeoutScale in a NetworkTransport.
DefaultTimeoutScale = 256 * 1024 // 256KB
)
var (
// ErrTransportShutdown is returned when operations on a transport are
// invoked after it's been terminated.
ErrTransportShutdown = errors.New("transport shutdown")
)
/*
NetworkTransport provides a network based transport that can be
used to communicate with Raft on remote machines. It requires
an underlying stream layer to provide a stream abstraction, which can
be simple TCP, TLS, etc. Underlying addresses must be castable to TCPAddr
This transport is very simple and lightweight. Each RPC request is
framed by sending a byte that indicates the message type, followed
by the MsgPack encoded request.
The response is an error string followed by the response object,
both are encoded using MsgPack.
InstallSnapshot is special, in that after the RPC request we stream
the entire state. That socket is not re-used as the connection state
is not known if there is an error.
*/
type NetworkTransport struct {
connPool map[string][]*netConn
connPoolLock sync.Mutex
consumeCh chan RPC
logger *log.Logger
maxPool int
shutdown bool
shutdownCh chan struct{}
shutdownLock sync.Mutex
stream StreamLayer
timeout time.Duration
TimeoutScale int
}
// StreamLayer is used with the NetworkTransport to provide
// the low level stream abstraction
type StreamLayer interface {
net.Listener
// Dial is used to create a new outgoing connection
Dial(address string, timeout time.Duration) (net.Conn, error)
}
type netConn struct {
target net.Addr
conn net.Conn
r *bufio.Reader
w *bufio.Writer
dec *codec.Decoder
enc *codec.Encoder
}
func (n *netConn) Release() error {
return n.conn.Close()
}
// NewNetworkTransport creates a new network transport with the given dialer
// and listener. The maxPool controls how many connections we will pool. The
// timeout is used to apply I/O deadlines. For InstallSnapshot, we multiply
// the timeout by (SnapshotSize / TimeoutScale).
func NewNetworkTransport(
stream StreamLayer,
maxPool int,
timeout time.Duration,
logOutput io.Writer,
) *NetworkTransport {
if logOutput == nil {
logOutput = os.Stderr
}
trans := &NetworkTransport{
connPool: make(map[string][]*netConn),
consumeCh: make(chan RPC),
logger: log.New(logOutput, "", log.LstdFlags),
maxPool: maxPool,
shutdownCh: make(chan struct{}),
stream: stream,
timeout: timeout,
TimeoutScale: DefaultTimeoutScale,
}
go trans.listen()
return trans
}
// Close is used to stop the network transport
func (n *NetworkTransport) Close() error {
n.shutdownLock.Lock()
defer n.shutdownLock.Unlock()
if !n.shutdown {
close(n.shutdownCh)
n.stream.Close()
n.shutdown = true
}
return nil
}
// Consumer implements the Transport interface.
func (n *NetworkTransport) Consumer() <-chan RPC {
return n.consumeCh
}
// LocalAddr implements the Transport interface.
func (n *NetworkTransport) LocalAddr() net.Addr {
return n.stream.Addr()
}
// getExistingConn is used to grab a pooled connection
func (n *NetworkTransport) getPooledConn(target net.Addr) *netConn {
n.connPoolLock.Lock()
defer n.connPoolLock.Unlock()
key := target.String()
conns, ok := n.connPool[key]
if !ok || len(conns) == 0 {
return nil
}
var conn *netConn
num := len(conns)
conn, conns[num-1] = conns[num-1], nil
n.connPool[key] = conns[:num-1]
return conn
}
// getConn is used to get a connection from the pool
func (n *NetworkTransport) getConn(target net.Addr) (*netConn, error) {
// Check for a pooled conn
if conn := n.getPooledConn(target); conn != nil {
return conn, nil
}
// Dial a new connection
conn, err := n.stream.Dial(target.String(), n.timeout)
if err != nil {
return nil, err
}
// Wrap the conn
netConn := &netConn{
target: target,
conn: conn,
r: bufio.NewReader(conn),
w: bufio.NewWriter(conn),
}
// Setup encoder/decoders
netConn.dec = codec.NewDecoder(netConn.r, &codec.MsgpackHandle{})
netConn.enc = codec.NewEncoder(netConn.w, &codec.MsgpackHandle{})
// Done
return netConn, nil
}
// returnConn returns a connection back to the pool
func (n *NetworkTransport) returnConn(conn *netConn) {
n.connPoolLock.Lock()
defer n.connPoolLock.Unlock()
key := conn.target.String()
conns, _ := n.connPool[key]
if !n.shutdown && len(conns) < n.maxPool {
n.connPool[key] = append(conns, conn)
} else {
conn.Release()
}
}
// AppendEntries implements the Transport interface.
func (n *NetworkTransport) AppendEntries(target net.Addr, args *AppendEntriesRequest, resp *AppendEntriesResponse) error {
return n.genericRPC(target, rpcAppendEntries, args, resp)
}
// RequestVote implements the Transport interface.
func (n *NetworkTransport) RequestVote(target net.Addr, args *RequestVoteRequest, resp *RequestVoteResponse) error {
return n.genericRPC(target, rpcRequestVote, args, resp)
}
// genericRPC handles a simple request/response RPC
func (n *NetworkTransport) genericRPC(target net.Addr, rpcType uint8, args interface{}, resp interface{}) error {
// Get a conn
conn, err := n.getConn(target)
if err != nil {
return err
}
// Set a deadline
if n.timeout > 0 {
conn.conn.SetDeadline(time.Now().Add(n.timeout))
}
// Send the RPC
if err := sendRPC(conn, rpcType, args); err != nil {
return err
}
// Decode the response
return n.decodeResponse(conn, resp, true)
}
// InstallSnapshot implements the Transport interface.
func (n *NetworkTransport) InstallSnapshot(target net.Addr, args *InstallSnapshotRequest, resp *InstallSnapshotResponse, data io.Reader) error {
// Get a conn, always close for InstallSnapshot
conn, err := n.getConn(target)
if err != nil {
return err
}
defer conn.Release()
// Set a deadline, scaled by request size
if n.timeout > 0 {
timeout := n.timeout * time.Duration(args.Size/int64(n.TimeoutScale))
if timeout < n.timeout {
timeout = n.timeout
}
conn.conn.SetDeadline(time.Now().Add(timeout))
}
// Send the RPC
if err := sendRPC(conn, rpcInstallSnapshot, args); err != nil {
return err
}
// Stream the state
if _, err := io.Copy(conn.w, data); err != nil {
return err
}
// Flush
if err := conn.w.Flush(); err != nil {
return err
}
// Decode the response, do not return conn
return n.decodeResponse(conn, resp, false)
}
// EncodePeer implements the Transport interface.
func (n *NetworkTransport) EncodePeer(p net.Addr) []byte {
return []byte(p.String())
}
// DecodePeer implements the Transport interface.
func (n *NetworkTransport) DecodePeer(buf []byte) net.Addr {
addr, err := net.ResolveTCPAddr("tcp", string(buf))
if err != nil {
panic(fmt.Errorf("failed to parse network address: %s", buf))
}
return addr
}
// listen is used to handling incoming connections
func (n *NetworkTransport) listen() {
for {
// Accept incoming connections
conn, err := n.stream.Accept()
if err != nil {
if n.shutdown {
return
}
n.logger.Printf("[ERR] raft-net: Failed to accept connection: %v", err)
continue
}
n.logger.Printf("[DEBUG] raft-net: %v accepted connection from: %v", n.LocalAddr(), conn.RemoteAddr())
// Handle the connection in dedicated routine
go n.handleConn(conn)
}
}
// handleConn is used to handle an inbound connection for its lifespan
func (n *NetworkTransport) handleConn(conn net.Conn) {
defer conn.Close()
r := bufio.NewReader(conn)
w := bufio.NewWriter(conn)
dec := codec.NewDecoder(r, &codec.MsgpackHandle{})
enc := codec.NewEncoder(w, &codec.MsgpackHandle{})
for {
if err := n.handleCommand(r, dec, enc); err != nil {
if err != io.EOF {
n.logger.Printf("[ERR] raft-net: Failed to decode incoming command: %v", err)
}
return
}
if err := w.Flush(); err != nil {
n.logger.Printf("[ERR] raft-net: Failed to flush response: %v", err)
return
}
}
}
// handleCommand is used to decode and dispatch a single command
func (n *NetworkTransport) handleCommand(r *bufio.Reader, dec *codec.Decoder, enc *codec.Encoder) error {
// Get the rpc type
rpcType, err := r.ReadByte()
if err != nil {
return err
}
// Create the RPC object
respCh := make(chan RPCResponse, 1)
rpc := RPC{
RespChan: respCh,
}
// Decode the command
switch rpcType {
case rpcAppendEntries:
var req AppendEntriesRequest
if err := dec.Decode(&req); err != nil {
return err
}
rpc.Command = &req
case rpcRequestVote:
var req RequestVoteRequest
if err := dec.Decode(&req); err != nil {
return err
}
rpc.Command = &req
case rpcInstallSnapshot:
var req InstallSnapshotRequest
if err := dec.Decode(&req); err != nil {
return err
}
rpc.Command = &req
rpc.Reader = io.LimitReader(r, req.Size)
default:
return fmt.Errorf("unknown rpc type %d", rpcType)
}
// Dispatch the RPC
select {
case n.consumeCh <- rpc:
case <-n.shutdownCh:
return ErrTransportShutdown
}
// Wait for response
select {
case resp := <-respCh:
// Send the error first
respErr := ""
if resp.Error != nil {
respErr = resp.Error.Error()
}
if err := enc.Encode(respErr); err != nil {
return err
}
// Send the response
if err := enc.Encode(resp.Response); err != nil {
return err
}
case <-n.shutdownCh:
return ErrTransportShutdown
}
return nil
}
// decodeResponse is used to decode an RPC response and return the conn
func (n *NetworkTransport) decodeResponse(conn *netConn, resp interface{}, retConn bool) error {
// Decode the error if any
var rpcError string
if err := conn.dec.Decode(&rpcError); err != nil {
conn.Release()
return err
}
// Decode the response
if err := conn.dec.Decode(resp); err != nil {
conn.Release()
return err
}
// Return the conn
if retConn {
n.returnConn(conn)
}
// Format an error if any
if rpcError != "" {
return fmt.Errorf(rpcError)
}
return nil
}
// sendRPC is used to encode and send the RPC
func sendRPC(conn *netConn, rpcType uint8, args interface{}) error {
// Write the request type
if err := conn.w.WriteByte(rpcType); err != nil {
conn.Release()
return err
}
// Send the request
if err := conn.enc.Encode(args); err != nil {
conn.Release()
return err
}
// Flush
if err := conn.w.Flush(); err != nil {
conn.Release()
return err
}
return nil
}