forked from hashicorp/raft
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathtransport.go
85 lines (68 loc) · 2.83 KB
/
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
package raft
import (
"io"
"time"
)
// RPCResponse captures both a response and a potential error.
type RPCResponse struct {
Response interface{}
Error error
}
// RPC has a command, and provides a response mechanism.
type RPC struct {
Command interface{}
Reader io.Reader // Set only for InstallSnapshot
RespChan chan<- RPCResponse
}
// Respond is used to respond with a response, error or both
func (r *RPC) Respond(resp interface{}, err error) {
r.RespChan <- RPCResponse{resp, err}
}
// Transport provides an interface for network transports
// to allow Raft to communicate with other nodes.
type Transport interface {
// Consumer returns a channel that can be used to
// consume and respond to RPC requests.
Consumer() <-chan RPC
// LocalAddr is used to return our local address to distinguish from our peers.
LocalAddr() string
// AppendEntriesPipeline returns an interface that can be used to pipeline
// AppendEntries requests.
AppendEntriesPipeline(target string) (AppendPipeline, error)
// AppendEntries sends the appropriate RPC to the target node.
AppendEntries(target string, args *AppendEntriesRequest, resp *AppendEntriesResponse) error
// RequestVote sends the appropriate RPC to the target node.
RequestVote(target string, args *RequestVoteRequest, resp *RequestVoteResponse) error
// InstallSnapshot is used to push a snapshot down to a follower. The data is read from
// the ReadCloser and streamed to the client.
InstallSnapshot(target string, args *InstallSnapshotRequest, resp *InstallSnapshotResponse, data io.Reader) error
// EncodePeer is used to serialize a peer name.
EncodePeer(string) []byte
// DecodePeer is used to deserialize a peer name.
DecodePeer([]byte) string
// SetHeartbeatHandler is used to setup a heartbeat handler
// as a fast-pass. This is to avoid head-of-line blocking from
// disk IO. If a Transport does not support this, it can simply
// ignore the call, and push the heartbeat onto the Consumer channel.
SetHeartbeatHandler(cb func(rpc RPC))
}
// AppendPipeline is used for pipelining AppendEntries requests. It is used
// to increase the replication throughput by masking latency and better
// utilizing bandwidth.
type AppendPipeline interface {
// AppendEntries is used to add another request to the pipeline.
// The send may block which is an effective form of back-pressure.
AppendEntries(args *AppendEntriesRequest, resp *AppendEntriesResponse) (AppendFuture, error)
// Consumer returns a channel that can be used to consume
// response futures when they are ready.
Consumer() <-chan AppendFuture
// Closes pipeline and cancels all inflight RPCs
Close() error
}
// AppendFuture is used to return information about a pipelined AppendEntries request.
type AppendFuture interface {
Future
Start() time.Time
Request() *AppendEntriesRequest
Response() *AppendEntriesResponse
}