├── proto ├── Makefile ├── transport.proto ├── transport_grpc.pb.go └── transport.pb.go ├── options.go ├── .github └── workflows │ └── go.yml ├── go.mod ├── README.md ├── LICENSE ├── transport.go ├── completeness_test.go ├── grpcapi.go ├── fromproto.go ├── toproto.go ├── pair_test.go ├── raftapi.go └── go.sum /proto/Makefile: -------------------------------------------------------------------------------- 1 | transport.pb.go: transport.proto 2 | protoc --go_out=. --go_opt=paths=source_relative --go-grpc_out=. --go-grpc_opt=paths=source_relative transport.proto 3 | 4 | force: 5 | rm -f transport.pb.go 6 | make transport.pb.go 7 | -------------------------------------------------------------------------------- /options.go: -------------------------------------------------------------------------------- 1 | package transport 2 | 3 | import "time" 4 | 5 | type Option func(m *Manager) 6 | 7 | // WithHeartbeatTimeout configures the transport to not wait for more than d 8 | // for a heartbeat to be executed by a remote peer. 9 | func WithHeartbeatTimeout(d time.Duration) Option { 10 | return func(m *Manager) { 11 | m.heartbeatTimeout = d 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /.github/workflows/go.yml: -------------------------------------------------------------------------------- 1 | name: Build and test 2 | 3 | on: 4 | push: 5 | branches: [ "master" ] 6 | pull_request: 7 | branches: [ "master" ] 8 | 9 | jobs: 10 | 11 | build_and_test: 12 | runs-on: ubuntu-latest 13 | steps: 14 | - uses: actions/checkout@v3 15 | 16 | - name: Set up Go 17 | uses: actions/setup-go@v3 18 | 19 | - name: Build 20 | run: go build -v ./... 21 | 22 | - name: Test 23 | run: go test -v ./... 24 | -------------------------------------------------------------------------------- /go.mod: -------------------------------------------------------------------------------- 1 | module github.com/Jille/raft-grpc-transport 2 | 3 | go 1.13 4 | 5 | require ( 6 | github.com/google/go-cmp v0.5.9 7 | github.com/google/gofuzz v1.2.0 8 | github.com/hashicorp/go-immutable-radix v1.3.1 // indirect 9 | github.com/hashicorp/go-multierror v1.1.1 10 | github.com/hashicorp/golang-lru v0.5.4 // indirect 11 | github.com/hashicorp/raft v1.7.0 12 | github.com/pkg/errors v0.9.1 13 | go.uber.org/goleak v1.1.12 14 | google.golang.org/genproto v0.0.0-20210903162649-d08c68adba83 // indirect 15 | google.golang.org/grpc v1.40.0 16 | google.golang.org/protobuf v1.28.1 17 | ) 18 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # raft-grpc-transport 2 | 3 | [![Godoc](https://godoc.org/github.com/Jille/raft-grpc-transport?status.svg)](https://godoc.org/github.com/Jille/raft-grpc-transport) 4 | 5 | This library provides a [Transport](https://godoc.org/github.com/hashicorp/raft#Transport) for https://github.com/hashicorp/raft over gRPC. 6 | 7 | One benefit of this is that gRPC is easy to multiplex over a single port. 8 | 9 | ## Usage 10 | 11 | ```go 12 | // ... 13 | tm := transport.New(raft.ServerAddress(myAddress), []grpc.DialOption{grpc.WithInsecure()}) 14 | s := grpc.NewServer() 15 | tm.Register(s) 16 | r, err := raft.NewRaft(..., tm.Transport()) 17 | // ... 18 | ``` 19 | 20 | Want more example code? Check out main.go at https://github.com/Jille/raft-grpc-example 21 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | BSD 2-Clause License 2 | 3 | Copyright (c) 2020, Jille Timmermans 4 | All rights reserved. 5 | 6 | Redistribution and use in source and binary forms, with or without 7 | modification, are permitted provided that the following conditions are met: 8 | 9 | 1. Redistributions of source code must retain the above copyright notice, this 10 | list of conditions and the following disclaimer. 11 | 12 | 2. Redistributions in binary form must reproduce the above copyright notice, 13 | this list of conditions and the following disclaimer in the documentation 14 | and/or other materials provided with the distribution. 15 | 16 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE 19 | DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE 20 | FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL 21 | DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR 22 | SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 23 | CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, 24 | OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 25 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 26 | -------------------------------------------------------------------------------- /transport.go: -------------------------------------------------------------------------------- 1 | // Package transport provides a Transport for github.com/hashicorp/raft over gRPC. 2 | package transport 3 | 4 | import ( 5 | "sync" 6 | "time" 7 | 8 | pb "github.com/Jille/raft-grpc-transport/proto" 9 | "github.com/hashicorp/go-multierror" 10 | "github.com/hashicorp/raft" 11 | "github.com/pkg/errors" 12 | "google.golang.org/grpc" 13 | ) 14 | 15 | var ( 16 | errCloseErr = errors.New("error closing connections") 17 | ) 18 | 19 | type Manager struct { 20 | localAddress raft.ServerAddress 21 | dialOptions []grpc.DialOption 22 | 23 | rpcChan chan raft.RPC 24 | heartbeatFunc func(raft.RPC) 25 | heartbeatFuncMtx sync.Mutex 26 | heartbeatTimeout time.Duration 27 | 28 | connectionsMtx sync.Mutex 29 | connections map[raft.ServerAddress]*conn 30 | 31 | shutdown bool 32 | shutdownCh chan struct{} 33 | shutdownLock sync.Mutex 34 | } 35 | 36 | // New creates both components of raft-grpc-transport: a gRPC service and a Raft Transport. 37 | func New(localAddress raft.ServerAddress, dialOptions []grpc.DialOption, options ...Option) *Manager { 38 | m := &Manager{ 39 | localAddress: localAddress, 40 | dialOptions: dialOptions, 41 | 42 | rpcChan: make(chan raft.RPC), 43 | connections: map[raft.ServerAddress]*conn{}, 44 | 45 | shutdownCh: make(chan struct{}), 46 | } 47 | for _, opt := range options { 48 | opt(m) 49 | } 50 | return m 51 | } 52 | 53 | // Register the RaftTransport gRPC service on a gRPC server. 54 | func (m *Manager) Register(s grpc.ServiceRegistrar) { 55 | pb.RegisterRaftTransportServer(s, gRPCAPI{manager: m}) 56 | } 57 | 58 | // Transport returns a raft.Transport that communicates over gRPC. 59 | func (m *Manager) Transport() raft.Transport { 60 | return raftAPI{m} 61 | } 62 | 63 | func (m *Manager) Close() error { 64 | m.shutdownLock.Lock() 65 | defer m.shutdownLock.Unlock() 66 | 67 | if m.shutdown { 68 | return nil 69 | } 70 | 71 | close(m.shutdownCh) 72 | m.shutdown = true 73 | return m.disconnectAll() 74 | } 75 | 76 | func (m *Manager) disconnectAll() error { 77 | m.connectionsMtx.Lock() 78 | defer m.connectionsMtx.Unlock() 79 | 80 | err := errCloseErr 81 | for k, conn := range m.connections { 82 | // Lock conn.mtx to ensure Dial() is complete 83 | conn.mtx.Lock() 84 | conn.mtx.Unlock() 85 | closeErr := conn.clientConn.Close() 86 | if closeErr != nil { 87 | err = multierror.Append(err, closeErr) 88 | } 89 | delete(m.connections, k) 90 | } 91 | 92 | if err != errCloseErr { 93 | return err 94 | } 95 | 96 | return nil 97 | } 98 | -------------------------------------------------------------------------------- /proto/transport.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | option go_package = "github.com/Jille/raft-grpc-transport/proto"; 4 | 5 | import "google/protobuf/timestamp.proto"; 6 | 7 | service RaftTransport { 8 | // AppendEntriesPipeline opens an AppendEntries message stream. 9 | rpc AppendEntriesPipeline(stream AppendEntriesRequest) returns (stream AppendEntriesResponse) {} 10 | 11 | // AppendEntries performs a single append entries request / response. 12 | rpc AppendEntries(AppendEntriesRequest) returns (AppendEntriesResponse) {} 13 | // RequestVote is the command used by a candidate to ask a Raft peer for a vote in an election. 14 | rpc RequestVote(RequestVoteRequest) returns (RequestVoteResponse) {} 15 | // TimeoutNow is used to start a leadership transfer to the target node. 16 | rpc TimeoutNow(TimeoutNowRequest) returns (TimeoutNowResponse) {} 17 | // InstallSnapshot is the command sent to a Raft peer to bootstrap its log (and state machine) from a snapshot on another peer. 18 | rpc InstallSnapshot(stream InstallSnapshotRequest) returns (InstallSnapshotResponse) {} 19 | // RequestPreVote is the command used by a candidate to ask a Raft peer for a vote in an election. 20 | rpc RequestPreVote(RequestPreVoteRequest) returns (RequestPreVoteResponse) {} 21 | } 22 | 23 | message RPCHeader { 24 | int64 protocol_version = 1; 25 | bytes id = 2; 26 | bytes addr = 3; 27 | } 28 | 29 | message Log { 30 | enum LogType { 31 | LOG_COMMAND = 0; 32 | LOG_NOOP = 1; 33 | LOG_ADD_PEER_DEPRECATED = 2; 34 | LOG_REMOVE_PEER_DEPRECATED = 3; 35 | LOG_BARRIER = 4; 36 | LOG_CONFIGURATION = 5; 37 | } 38 | uint64 index = 1; 39 | uint64 term = 2; 40 | LogType type = 3; 41 | bytes data = 4; 42 | bytes extensions = 5; 43 | google.protobuf.Timestamp appended_at = 6; 44 | } 45 | 46 | message AppendEntriesRequest { 47 | RPCHeader rpc_header = 1; 48 | uint64 term = 2; 49 | bytes leader = 3; 50 | uint64 prev_log_entry = 4; 51 | uint64 prev_log_term = 5; 52 | repeated Log entries = 6; 53 | uint64 leader_commit_index = 7; 54 | } 55 | 56 | message AppendEntriesResponse { 57 | RPCHeader rpc_header = 1; 58 | uint64 term = 2; 59 | uint64 last_log = 3; 60 | bool success = 4; 61 | bool no_retry_backoff = 5; 62 | } 63 | 64 | message RequestVoteRequest { 65 | RPCHeader rpc_header = 1; 66 | uint64 term = 2; 67 | bytes candidate = 3; 68 | uint64 last_log_index = 4; 69 | uint64 last_log_term = 5; 70 | bool leadership_transfer = 6; 71 | } 72 | 73 | message RequestVoteResponse { 74 | RPCHeader rpc_header = 1; 75 | uint64 term = 2; 76 | bytes peers = 3; 77 | bool granted = 4; 78 | } 79 | 80 | message TimeoutNowRequest { 81 | RPCHeader rpc_header = 1; 82 | } 83 | 84 | message TimeoutNowResponse { 85 | RPCHeader rpc_header = 1; 86 | } 87 | 88 | // The first InstallSnapshotRequest on the stream contains all the metadata. 89 | // All further messages contain only data. 90 | message InstallSnapshotRequest { 91 | RPCHeader rpc_header = 1; 92 | int64 snapshot_version = 11; 93 | uint64 term = 2; 94 | bytes leader = 3; 95 | uint64 last_log_index = 4; 96 | uint64 last_log_term = 5; 97 | bytes peers = 6; 98 | bytes configuration = 7; 99 | uint64 configuration_index = 8; 100 | int64 size = 9; 101 | 102 | bytes data = 10; 103 | } 104 | 105 | message InstallSnapshotResponse { 106 | RPCHeader rpc_header = 1; 107 | uint64 term = 2; 108 | bool success = 3; 109 | } 110 | 111 | message RequestPreVoteRequest { 112 | RPCHeader rpc_header = 1; 113 | uint64 term = 2; 114 | uint64 last_log_index = 3; 115 | uint64 last_log_term = 4; 116 | } 117 | 118 | message RequestPreVoteResponse { 119 | RPCHeader rpc_header = 1; 120 | uint64 term = 2; 121 | bool granted = 3; 122 | } 123 | -------------------------------------------------------------------------------- /completeness_test.go: -------------------------------------------------------------------------------- 1 | package transport 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/google/go-cmp/cmp" 7 | fuzz "github.com/google/gofuzz" 8 | "github.com/hashicorp/raft" 9 | "go.uber.org/goleak" 10 | ) 11 | 12 | func fuzzLogType(lt *raft.LogType, c fuzz.Continue) { 13 | *lt = raft.LogType(c.Intn(int(raft.LogConfiguration + 1))) 14 | } 15 | 16 | func doFuzz(rm interface{}, i int) { 17 | f := fuzz.NewWithSeed(int64(i)).NilChance(0).Funcs(fuzzLogType) 18 | f.Fuzz(rm) 19 | } 20 | 21 | func verify(t *testing.T, rm1, rm2 interface{}) { 22 | t.Helper() 23 | if diff := cmp.Diff(rm1, rm2); diff != "" { 24 | t.Errorf("encode+decode return another value: %s", diff) 25 | } 26 | } 27 | 28 | func TestAppendEntriesRequest(t *testing.T) { 29 | defer goleak.VerifyNone(t) 30 | 31 | for i := 0; 1000 > i; i++ { 32 | rm := raft.AppendEntriesRequest{} 33 | doFuzz(&rm, i) 34 | pm := encodeAppendEntriesRequest(&rm) 35 | rm2 := decodeAppendEntriesRequest(pm) 36 | verify(t, &rm, rm2) 37 | } 38 | } 39 | 40 | func TestAppendEntriesResponse(t *testing.T) { 41 | defer goleak.VerifyNone(t) 42 | 43 | for i := 0; 1000 > i; i++ { 44 | rm := raft.AppendEntriesResponse{} 45 | doFuzz(&rm, i) 46 | pm := encodeAppendEntriesResponse(&rm) 47 | rm2 := decodeAppendEntriesResponse(pm) 48 | verify(t, &rm, rm2) 49 | } 50 | } 51 | 52 | func TestRequestVoteRequest(t *testing.T) { 53 | for i := 0; 1000 > i; i++ { 54 | rm := raft.RequestVoteRequest{} 55 | doFuzz(&rm, i) 56 | pm := encodeRequestVoteRequest(&rm) 57 | rm2 := decodeRequestVoteRequest(pm) 58 | verify(t, &rm, rm2) 59 | } 60 | } 61 | 62 | func TestRequestVoteResponse(t *testing.T) { 63 | defer goleak.VerifyNone(t) 64 | 65 | for i := 0; 1000 > i; i++ { 66 | rm := raft.RequestVoteResponse{} 67 | doFuzz(&rm, i) 68 | pm := encodeRequestVoteResponse(&rm) 69 | rm2 := decodeRequestVoteResponse(pm) 70 | verify(t, &rm, rm2) 71 | } 72 | } 73 | 74 | func TestInstallSnapshotRequest(t *testing.T) { 75 | defer goleak.VerifyNone(t) 76 | 77 | for i := 0; 1000 > i; i++ { 78 | rm := raft.InstallSnapshotRequest{} 79 | doFuzz(&rm, i) 80 | pm := encodeInstallSnapshotRequest(&rm) 81 | rm2 := decodeInstallSnapshotRequest(pm) 82 | verify(t, &rm, rm2) 83 | } 84 | } 85 | 86 | func TestInstallSnapshotResponse(t *testing.T) { 87 | defer goleak.VerifyNone(t) 88 | 89 | for i := 0; 1000 > i; i++ { 90 | rm := raft.InstallSnapshotResponse{} 91 | doFuzz(&rm, i) 92 | pm := encodeInstallSnapshotResponse(&rm) 93 | rm2 := decodeInstallSnapshotResponse(pm) 94 | verify(t, &rm, rm2) 95 | } 96 | } 97 | 98 | func TestTimeoutNowRequest(t *testing.T) { 99 | defer goleak.VerifyNone(t) 100 | 101 | for i := 0; 1000 > i; i++ { 102 | rm := raft.TimeoutNowRequest{} 103 | doFuzz(&rm, i) 104 | pm := encodeTimeoutNowRequest(&rm) 105 | rm2 := decodeTimeoutNowRequest(pm) 106 | verify(t, &rm, rm2) 107 | } 108 | } 109 | 110 | func TestTimeoutNowResponse(t *testing.T) { 111 | defer goleak.VerifyNone(t) 112 | 113 | for i := 0; 1000 > i; i++ { 114 | rm := raft.TimeoutNowResponse{} 115 | doFuzz(&rm, i) 116 | pm := encodeTimeoutNowResponse(&rm) 117 | rm2 := decodeTimeoutNowResponse(pm) 118 | verify(t, &rm, rm2) 119 | } 120 | } 121 | 122 | func TestRequestPreVoteRequest(t *testing.T) { 123 | defer goleak.VerifyNone(t) 124 | 125 | for i := 0; 1000 > i; i++ { 126 | rm := raft.RequestPreVoteRequest{} 127 | doFuzz(&rm, i) 128 | pm := encodeRequestPreVoteRequest(&rm) 129 | rm2 := decodeRequestPreVoteRequest(pm) 130 | verify(t, &rm, rm2) 131 | } 132 | } 133 | 134 | func TestRequestPreVoteResponse(t *testing.T) { 135 | defer goleak.VerifyNone(t) 136 | 137 | for i := 0; 1000 > i; i++ { 138 | rm := raft.RequestPreVoteResponse{} 139 | doFuzz(&rm, i) 140 | pm := encodeRequestPreVoteResponse(&rm) 141 | rm2 := decodeRequestPreVoteResponse(pm) 142 | verify(t, &rm, rm2) 143 | } 144 | } 145 | -------------------------------------------------------------------------------- /grpcapi.go: -------------------------------------------------------------------------------- 1 | package transport 2 | 3 | import ( 4 | "context" 5 | "io" 6 | 7 | pb "github.com/Jille/raft-grpc-transport/proto" 8 | "github.com/hashicorp/raft" 9 | ) 10 | 11 | // These are requests incoming over gRPC that we need to relay to the Raft engine. 12 | 13 | type gRPCAPI struct { 14 | manager *Manager 15 | 16 | // "Unsafe" to ensure compilation fails if new methods are added but not implemented 17 | pb.UnsafeRaftTransportServer 18 | } 19 | 20 | func (g gRPCAPI) handleRPC(command interface{}, data io.Reader) (interface{}, error) { 21 | ch := make(chan raft.RPCResponse, 1) 22 | rpc := raft.RPC{ 23 | Command: command, 24 | RespChan: ch, 25 | Reader: data, 26 | } 27 | if isHeartbeat(command) { 28 | // We can take the fast path and use the heartbeat callback and skip the queue in g.manager.rpcChan. 29 | g.manager.heartbeatFuncMtx.Lock() 30 | fn := g.manager.heartbeatFunc 31 | g.manager.heartbeatFuncMtx.Unlock() 32 | if fn != nil { 33 | fn(rpc) 34 | goto wait 35 | } 36 | } 37 | select { 38 | case g.manager.rpcChan <- rpc: 39 | case <-g.manager.shutdownCh: 40 | return nil, raft.ErrTransportShutdown 41 | } 42 | 43 | wait: 44 | select { 45 | case resp := <-ch: 46 | if resp.Error != nil { 47 | return nil, resp.Error 48 | } 49 | return resp.Response, nil 50 | case <-g.manager.shutdownCh: 51 | return nil, raft.ErrTransportShutdown 52 | } 53 | } 54 | 55 | func (g gRPCAPI) AppendEntries(ctx context.Context, req *pb.AppendEntriesRequest) (*pb.AppendEntriesResponse, error) { 56 | resp, err := g.handleRPC(decodeAppendEntriesRequest(req), nil) 57 | if err != nil { 58 | return nil, err 59 | } 60 | return encodeAppendEntriesResponse(resp.(*raft.AppendEntriesResponse)), nil 61 | } 62 | 63 | func (g gRPCAPI) RequestVote(ctx context.Context, req *pb.RequestVoteRequest) (*pb.RequestVoteResponse, error) { 64 | resp, err := g.handleRPC(decodeRequestVoteRequest(req), nil) 65 | if err != nil { 66 | return nil, err 67 | } 68 | return encodeRequestVoteResponse(resp.(*raft.RequestVoteResponse)), nil 69 | } 70 | 71 | func (g gRPCAPI) TimeoutNow(ctx context.Context, req *pb.TimeoutNowRequest) (*pb.TimeoutNowResponse, error) { 72 | resp, err := g.handleRPC(decodeTimeoutNowRequest(req), nil) 73 | if err != nil { 74 | return nil, err 75 | } 76 | return encodeTimeoutNowResponse(resp.(*raft.TimeoutNowResponse)), nil 77 | } 78 | 79 | func (g gRPCAPI) RequestPreVote(ctx context.Context, req *pb.RequestPreVoteRequest) (*pb.RequestPreVoteResponse, error) { 80 | resp, err := g.handleRPC(decodeRequestPreVoteRequest(req), nil) 81 | if err != nil { 82 | return nil, err 83 | } 84 | return encodeRequestPreVoteResponse(resp.(*raft.RequestPreVoteResponse)), nil 85 | } 86 | 87 | func (g gRPCAPI) InstallSnapshot(s pb.RaftTransport_InstallSnapshotServer) error { 88 | isr, err := s.Recv() 89 | if err != nil { 90 | return err 91 | } 92 | resp, err := g.handleRPC(decodeInstallSnapshotRequest(isr), &snapshotStream{s, isr.GetData()}) 93 | if err != nil { 94 | return err 95 | } 96 | return s.SendAndClose(encodeInstallSnapshotResponse(resp.(*raft.InstallSnapshotResponse))) 97 | } 98 | 99 | type snapshotStream struct { 100 | s pb.RaftTransport_InstallSnapshotServer 101 | 102 | buf []byte 103 | } 104 | 105 | func (s *snapshotStream) Read(b []byte) (int, error) { 106 | if len(s.buf) > 0 { 107 | n := copy(b, s.buf) 108 | s.buf = s.buf[n:] 109 | return n, nil 110 | } 111 | m, err := s.s.Recv() 112 | if err != nil { 113 | return 0, err 114 | } 115 | n := copy(b, m.GetData()) 116 | if n < len(m.GetData()) { 117 | s.buf = m.GetData()[n:] 118 | } 119 | return n, nil 120 | } 121 | 122 | func (g gRPCAPI) AppendEntriesPipeline(s pb.RaftTransport_AppendEntriesPipelineServer) error { 123 | for { 124 | msg, err := s.Recv() 125 | if err != nil { 126 | return err 127 | } 128 | resp, err := g.handleRPC(decodeAppendEntriesRequest(msg), nil) 129 | if err != nil { 130 | // TODO(quis): One failure doesn't have to break the entire stream? 131 | // Or does it all go wrong when it's out of order anyway? 132 | return err 133 | } 134 | if err := s.Send(encodeAppendEntriesResponse(resp.(*raft.AppendEntriesResponse))); err != nil { 135 | return err 136 | } 137 | } 138 | } 139 | 140 | func isHeartbeat(command interface{}) bool { 141 | req, ok := command.(*raft.AppendEntriesRequest) 142 | if !ok { 143 | return false 144 | } 145 | return req.Term != 0 && len(req.Leader) != 0 && req.PrevLogEntry == 0 && req.PrevLogTerm == 0 && len(req.Entries) == 0 && req.LeaderCommitIndex == 0 146 | } 147 | -------------------------------------------------------------------------------- /fromproto.go: -------------------------------------------------------------------------------- 1 | package transport 2 | 3 | import ( 4 | pb "github.com/Jille/raft-grpc-transport/proto" 5 | "github.com/hashicorp/raft" 6 | ) 7 | 8 | func decodeAppendEntriesRequest(m *pb.AppendEntriesRequest) *raft.AppendEntriesRequest { 9 | return &raft.AppendEntriesRequest{ 10 | RPCHeader: decodeRPCHeader(m.RpcHeader), 11 | Term: m.Term, 12 | Leader: m.Leader, 13 | PrevLogEntry: m.PrevLogEntry, 14 | PrevLogTerm: m.PrevLogTerm, 15 | Entries: decodeLogs(m.Entries), 16 | LeaderCommitIndex: m.LeaderCommitIndex, 17 | } 18 | } 19 | 20 | func decodeRPCHeader(m *pb.RPCHeader) raft.RPCHeader { 21 | return raft.RPCHeader{ 22 | ProtocolVersion: raft.ProtocolVersion(m.ProtocolVersion), 23 | ID: m.Id, 24 | Addr: m.Addr, 25 | } 26 | } 27 | 28 | func decodeLogs(m []*pb.Log) []*raft.Log { 29 | ret := make([]*raft.Log, len(m)) 30 | for i, l := range m { 31 | ret[i] = decodeLog(l) 32 | } 33 | return ret 34 | } 35 | 36 | func decodeLog(m *pb.Log) *raft.Log { 37 | return &raft.Log{ 38 | Index: m.Index, 39 | Term: m.Term, 40 | Type: decodeLogType(m.Type), 41 | Data: m.Data, 42 | Extensions: m.Extensions, 43 | AppendedAt: m.AppendedAt.AsTime(), 44 | } 45 | } 46 | 47 | func decodeLogType(m pb.Log_LogType) raft.LogType { 48 | switch m { 49 | case pb.Log_LOG_COMMAND: 50 | return raft.LogCommand 51 | case pb.Log_LOG_NOOP: 52 | return raft.LogNoop 53 | case pb.Log_LOG_ADD_PEER_DEPRECATED: 54 | return raft.LogAddPeerDeprecated 55 | case pb.Log_LOG_REMOVE_PEER_DEPRECATED: 56 | return raft.LogRemovePeerDeprecated 57 | case pb.Log_LOG_BARRIER: 58 | return raft.LogBarrier 59 | case pb.Log_LOG_CONFIGURATION: 60 | return raft.LogConfiguration 61 | default: 62 | panic("invalid LogType") 63 | } 64 | } 65 | 66 | func decodeAppendEntriesResponse(m *pb.AppendEntriesResponse) *raft.AppendEntriesResponse { 67 | return &raft.AppendEntriesResponse{ 68 | RPCHeader: decodeRPCHeader(m.RpcHeader), 69 | Term: m.Term, 70 | LastLog: m.LastLog, 71 | Success: m.Success, 72 | NoRetryBackoff: m.NoRetryBackoff, 73 | } 74 | } 75 | 76 | func decodeRequestVoteRequest(m *pb.RequestVoteRequest) *raft.RequestVoteRequest { 77 | return &raft.RequestVoteRequest{ 78 | RPCHeader: decodeRPCHeader(m.RpcHeader), 79 | Term: m.Term, 80 | Candidate: m.Candidate, 81 | LastLogIndex: m.LastLogIndex, 82 | LastLogTerm: m.LastLogTerm, 83 | LeadershipTransfer: m.LeadershipTransfer, 84 | } 85 | } 86 | 87 | func decodeRequestVoteResponse(m *pb.RequestVoteResponse) *raft.RequestVoteResponse { 88 | return &raft.RequestVoteResponse{ 89 | RPCHeader: decodeRPCHeader(m.RpcHeader), 90 | Term: m.Term, 91 | Peers: m.Peers, 92 | Granted: m.Granted, 93 | } 94 | } 95 | 96 | func decodeInstallSnapshotRequest(m *pb.InstallSnapshotRequest) *raft.InstallSnapshotRequest { 97 | return &raft.InstallSnapshotRequest{ 98 | RPCHeader: decodeRPCHeader(m.RpcHeader), 99 | SnapshotVersion: raft.SnapshotVersion(m.SnapshotVersion), 100 | Term: m.Term, 101 | Leader: m.Leader, 102 | LastLogIndex: m.LastLogIndex, 103 | LastLogTerm: m.LastLogTerm, 104 | Peers: m.Peers, 105 | Configuration: m.Configuration, 106 | ConfigurationIndex: m.ConfigurationIndex, 107 | Size: m.Size, 108 | } 109 | } 110 | 111 | func decodeInstallSnapshotResponse(m *pb.InstallSnapshotResponse) *raft.InstallSnapshotResponse { 112 | return &raft.InstallSnapshotResponse{ 113 | RPCHeader: decodeRPCHeader(m.RpcHeader), 114 | Term: m.Term, 115 | Success: m.Success, 116 | } 117 | } 118 | 119 | func decodeTimeoutNowRequest(m *pb.TimeoutNowRequest) *raft.TimeoutNowRequest { 120 | return &raft.TimeoutNowRequest{ 121 | RPCHeader: decodeRPCHeader(m.RpcHeader), 122 | } 123 | } 124 | 125 | func decodeTimeoutNowResponse(m *pb.TimeoutNowResponse) *raft.TimeoutNowResponse { 126 | return &raft.TimeoutNowResponse{ 127 | RPCHeader: decodeRPCHeader(m.RpcHeader), 128 | } 129 | } 130 | 131 | func decodeRequestPreVoteRequest(m *pb.RequestPreVoteRequest) *raft.RequestPreVoteRequest { 132 | return &raft.RequestPreVoteRequest{ 133 | RPCHeader: decodeRPCHeader(m.RpcHeader), 134 | Term: m.Term, 135 | LastLogIndex: m.LastLogIndex, 136 | LastLogTerm: m.LastLogTerm, 137 | } 138 | } 139 | 140 | func decodeRequestPreVoteResponse(m *pb.RequestPreVoteResponse) *raft.RequestPreVoteResponse { 141 | return &raft.RequestPreVoteResponse{ 142 | RPCHeader: decodeRPCHeader(m.RpcHeader), 143 | Term: m.Term, 144 | Granted: m.Granted, 145 | } 146 | } 147 | -------------------------------------------------------------------------------- /toproto.go: -------------------------------------------------------------------------------- 1 | package transport 2 | 3 | import ( 4 | pb "github.com/Jille/raft-grpc-transport/proto" 5 | "github.com/hashicorp/raft" 6 | "google.golang.org/protobuf/types/known/timestamppb" 7 | ) 8 | 9 | func encodeAppendEntriesRequest(s *raft.AppendEntriesRequest) *pb.AppendEntriesRequest { 10 | return &pb.AppendEntriesRequest{ 11 | RpcHeader: encodeRPCHeader(s.RPCHeader), 12 | Term: s.Term, 13 | Leader: s.Leader, 14 | PrevLogEntry: s.PrevLogEntry, 15 | PrevLogTerm: s.PrevLogTerm, 16 | Entries: encodeLogs(s.Entries), 17 | LeaderCommitIndex: s.LeaderCommitIndex, 18 | } 19 | } 20 | 21 | func encodeRPCHeader(s raft.RPCHeader) *pb.RPCHeader { 22 | return &pb.RPCHeader{ 23 | ProtocolVersion: int64(s.ProtocolVersion), 24 | Id: s.ID, 25 | Addr: s.Addr, 26 | } 27 | } 28 | 29 | func encodeLogs(s []*raft.Log) []*pb.Log { 30 | ret := make([]*pb.Log, len(s)) 31 | for i, l := range s { 32 | ret[i] = encodeLog(l) 33 | } 34 | return ret 35 | } 36 | 37 | func encodeLog(s *raft.Log) *pb.Log { 38 | return &pb.Log{ 39 | Index: s.Index, 40 | Term: s.Term, 41 | Type: encodeLogType(s.Type), 42 | Data: s.Data, 43 | Extensions: s.Extensions, 44 | AppendedAt: timestamppb.New(s.AppendedAt), 45 | } 46 | } 47 | 48 | func encodeLogType(s raft.LogType) pb.Log_LogType { 49 | switch s { 50 | case raft.LogCommand: 51 | return pb.Log_LOG_COMMAND 52 | case raft.LogNoop: 53 | return pb.Log_LOG_NOOP 54 | case raft.LogAddPeerDeprecated: 55 | return pb.Log_LOG_ADD_PEER_DEPRECATED 56 | case raft.LogRemovePeerDeprecated: 57 | return pb.Log_LOG_REMOVE_PEER_DEPRECATED 58 | case raft.LogBarrier: 59 | return pb.Log_LOG_BARRIER 60 | case raft.LogConfiguration: 61 | return pb.Log_LOG_CONFIGURATION 62 | default: 63 | panic("invalid LogType") 64 | } 65 | } 66 | 67 | func encodeAppendEntriesResponse(s *raft.AppendEntriesResponse) *pb.AppendEntriesResponse { 68 | return &pb.AppendEntriesResponse{ 69 | RpcHeader: encodeRPCHeader(s.RPCHeader), 70 | Term: s.Term, 71 | LastLog: s.LastLog, 72 | Success: s.Success, 73 | NoRetryBackoff: s.NoRetryBackoff, 74 | } 75 | } 76 | 77 | func encodeRequestVoteRequest(s *raft.RequestVoteRequest) *pb.RequestVoteRequest { 78 | return &pb.RequestVoteRequest{ 79 | RpcHeader: encodeRPCHeader(s.RPCHeader), 80 | Term: s.Term, 81 | Candidate: s.Candidate, 82 | LastLogIndex: s.LastLogIndex, 83 | LastLogTerm: s.LastLogTerm, 84 | LeadershipTransfer: s.LeadershipTransfer, 85 | } 86 | } 87 | 88 | func encodeRequestVoteResponse(s *raft.RequestVoteResponse) *pb.RequestVoteResponse { 89 | return &pb.RequestVoteResponse{ 90 | RpcHeader: encodeRPCHeader(s.RPCHeader), 91 | Term: s.Term, 92 | Peers: s.Peers, 93 | Granted: s.Granted, 94 | } 95 | } 96 | 97 | func encodeInstallSnapshotRequest(s *raft.InstallSnapshotRequest) *pb.InstallSnapshotRequest { 98 | return &pb.InstallSnapshotRequest{ 99 | RpcHeader: encodeRPCHeader(s.RPCHeader), 100 | SnapshotVersion: int64(s.SnapshotVersion), 101 | Term: s.Term, 102 | Leader: s.Leader, 103 | LastLogIndex: s.LastLogIndex, 104 | LastLogTerm: s.LastLogTerm, 105 | Peers: s.Peers, 106 | Configuration: s.Configuration, 107 | ConfigurationIndex: s.ConfigurationIndex, 108 | Size: s.Size, 109 | } 110 | } 111 | 112 | func encodeInstallSnapshotResponse(s *raft.InstallSnapshotResponse) *pb.InstallSnapshotResponse { 113 | return &pb.InstallSnapshotResponse{ 114 | RpcHeader: encodeRPCHeader(s.RPCHeader), 115 | Term: s.Term, 116 | Success: s.Success, 117 | } 118 | } 119 | 120 | func encodeTimeoutNowRequest(s *raft.TimeoutNowRequest) *pb.TimeoutNowRequest { 121 | return &pb.TimeoutNowRequest{ 122 | RpcHeader: encodeRPCHeader(s.RPCHeader), 123 | } 124 | } 125 | 126 | func encodeTimeoutNowResponse(s *raft.TimeoutNowResponse) *pb.TimeoutNowResponse { 127 | return &pb.TimeoutNowResponse{ 128 | RpcHeader: encodeRPCHeader(s.RPCHeader), 129 | } 130 | } 131 | 132 | func encodeRequestPreVoteRequest(s *raft.RequestPreVoteRequest) *pb.RequestPreVoteRequest { 133 | return &pb.RequestPreVoteRequest{ 134 | RpcHeader: encodeRPCHeader(s.RPCHeader), 135 | Term: s.Term, 136 | LastLogIndex: s.LastLogIndex, 137 | LastLogTerm: s.LastLogTerm, 138 | } 139 | } 140 | 141 | func encodeRequestPreVoteResponse(s *raft.RequestPreVoteResponse) *pb.RequestPreVoteResponse { 142 | return &pb.RequestPreVoteResponse{ 143 | RpcHeader: encodeRPCHeader(s.RPCHeader), 144 | Term: s.Term, 145 | Granted: s.Granted, 146 | } 147 | } 148 | -------------------------------------------------------------------------------- /pair_test.go: -------------------------------------------------------------------------------- 1 | package transport_test 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "io" 7 | "log" 8 | "net" 9 | "reflect" 10 | "testing" 11 | 12 | transport "github.com/Jille/raft-grpc-transport" 13 | "github.com/hashicorp/raft" 14 | "go.uber.org/goleak" 15 | "google.golang.org/grpc" 16 | "google.golang.org/grpc/test/bufconn" 17 | ) 18 | 19 | func makeTestPair(ctx context.Context, t *testing.T) (raft.Transport, raft.Transport, chan struct{}) { 20 | t.Helper() 21 | t1Listen := bufconn.Listen(1024) 22 | t2Listen := bufconn.Listen(1024) 23 | shutdownSig := make(chan struct{}) 24 | 25 | t1 := transport.New(raft.ServerAddress("t1"), []grpc.DialOption{grpc.WithInsecure(), grpc.WithContextDialer(func(context.Context, string) (net.Conn, error) { 26 | return t2Listen.Dial() 27 | })}) 28 | t2 := transport.New(raft.ServerAddress("t2"), []grpc.DialOption{grpc.WithInsecure(), grpc.WithContextDialer(func(context.Context, string) (net.Conn, error) { 29 | return t1Listen.Dial() 30 | })}) 31 | 32 | s1 := grpc.NewServer() 33 | t1.Register(s1) 34 | go func() { 35 | if err := s1.Serve(t1Listen); err != nil { 36 | log.Fatalf("t1 exited with error: %v", err) 37 | } 38 | }() 39 | 40 | s2 := grpc.NewServer() 41 | t2.Register(s2) 42 | go func() { 43 | if err := s2.Serve(t2Listen); err != nil { 44 | log.Fatalf("t2 exited with error: %v", err) 45 | } 46 | }() 47 | 48 | go func() { 49 | <-ctx.Done() 50 | if t1Err := t1.Close(); t1Err != nil { 51 | t.Fatalf("received error on t1 close: %s", t1Err) 52 | } 53 | if t2Err := t2.Close(); t2Err != nil { 54 | t.Fatalf("received error on t1 close: %s", t2Err) 55 | } 56 | 57 | s1.GracefulStop() 58 | s2.GracefulStop() 59 | 60 | close(shutdownSig) 61 | }() 62 | 63 | return t1.Transport(), t2.Transport(), shutdownSig 64 | } 65 | 66 | func TestAppendEntries(t *testing.T) { 67 | defer goleak.VerifyNone(t) 68 | 69 | ctx, cancel := context.WithCancel(context.Background()) 70 | t1, t2, shutdownSig := makeTestPair(ctx, t) 71 | defer func() { 72 | cancel() 73 | <-shutdownSig 74 | }() 75 | 76 | stop := make(chan struct{}) 77 | go func() { 78 | for { 79 | select { 80 | case <-stop: 81 | return 82 | case rpc := <-t2.Consumer(): 83 | if got, want := rpc.Command.(*raft.AppendEntriesRequest).Leader, []byte{3, 2, 1}; !bytes.Equal(got, want) { 84 | t.Errorf("request.Leader = %v, want %v", got, want) 85 | } 86 | if got, want := rpc.Command.(*raft.AppendEntriesRequest).Entries, []*raft.Log{ 87 | {Type: raft.LogNoop, Extensions: []byte{1}, Data: []byte{55}}, 88 | }; !reflect.DeepEqual(got, want) { 89 | t.Errorf("request.Entries = %v, want %v", got, want) 90 | } 91 | rpc.Respond(&raft.AppendEntriesResponse{ 92 | Success: true, 93 | LastLog: 12396, 94 | }, nil) 95 | } 96 | } 97 | }() 98 | 99 | var resp raft.AppendEntriesResponse 100 | if err := t1.AppendEntries("t2", "t2", &raft.AppendEntriesRequest{ 101 | Leader: []byte{3, 2, 1}, 102 | Entries: []*raft.Log{ 103 | {Type: raft.LogNoop, Extensions: []byte{1}, Data: []byte{55}}, 104 | }, 105 | }, &resp); err != nil { 106 | t.Errorf("AppendEntries() failed: %v", err) 107 | } 108 | if got, want := resp.LastLog, uint64(12396); got != want { 109 | t.Errorf("resp.LastLog = %v, want %v", got, want) 110 | } 111 | 112 | close(stop) 113 | } 114 | 115 | func TestSnapshot(t *testing.T) { 116 | defer goleak.VerifyNone(t) 117 | 118 | ctx, cancel := context.WithCancel(context.Background()) 119 | t1, t2, shutdownSig := makeTestPair(ctx, t) 120 | defer func() { 121 | cancel() 122 | <-shutdownSig 123 | }() 124 | 125 | stop := make(chan struct{}) 126 | go func() { 127 | for { 128 | select { 129 | case <-stop: 130 | return 131 | case rpc := <-t2.Consumer(): 132 | if got, want := rpc.Command.(*raft.InstallSnapshotRequest), (&raft.InstallSnapshotRequest{ 133 | Term: 123, 134 | Leader: []byte{2}, 135 | Configuration: []byte{4, 2, 3}, 136 | ConfigurationIndex: 3, 137 | Size: 654321, 138 | Peers: []byte{8}, 139 | }); !reflect.DeepEqual(got, want) { 140 | t.Errorf("request = %+v, want %+v", got, want) 141 | } 142 | 143 | var i int 144 | for { 145 | var buf [431]byte 146 | n, err := rpc.Reader.Read(buf[:]) 147 | if err != nil { 148 | if err == io.EOF { 149 | break 150 | } 151 | t.Errorf("Read() returned: %v", err) 152 | } 153 | i += n 154 | if !bytes.Equal(buf[:n], bytes.Repeat([]byte{89}, n)) { 155 | t.Errorf("Bad data: got %v, want %v", buf[:n], bytes.Repeat([]byte{89}, n)) 156 | } 157 | } 158 | if got, want := int64(i), rpc.Command.(*raft.InstallSnapshotRequest).Size; got != want { 159 | t.Errorf("read %d bytes, want %d", got, want) 160 | } 161 | 162 | rpc.Respond(&raft.InstallSnapshotResponse{ 163 | Success: true, 164 | }, nil) 165 | } 166 | } 167 | }() 168 | 169 | var resp raft.InstallSnapshotResponse 170 | b := bytes.Repeat([]byte{89}, 654321) 171 | if err := t1.InstallSnapshot("t2", "t2", &raft.InstallSnapshotRequest{ 172 | Term: 123, 173 | Leader: []byte{2}, 174 | Configuration: []byte{4, 2, 3}, 175 | ConfigurationIndex: 3, 176 | Size: int64(len(b)), 177 | Peers: []byte{8}, 178 | }, &resp, bytes.NewReader(b)); err != nil { 179 | t.Errorf("InstallSnapshot() failed: %v", err) 180 | } 181 | if got, want := resp.Success, true; got != want { 182 | t.Errorf("resp.Success = %v, want %v", got, want) 183 | } 184 | 185 | close(stop) 186 | } 187 | -------------------------------------------------------------------------------- /raftapi.go: -------------------------------------------------------------------------------- 1 | package transport 2 | 3 | import ( 4 | "context" 5 | "io" 6 | "sync" 7 | "time" 8 | 9 | pb "github.com/Jille/raft-grpc-transport/proto" 10 | "github.com/hashicorp/raft" 11 | "google.golang.org/grpc" 12 | ) 13 | 14 | // These are calls from the Raft engine that we need to send out over gRPC. 15 | 16 | type raftAPI struct { 17 | manager *Manager 18 | } 19 | 20 | var _ raft.Transport = raftAPI{} 21 | var _ raft.WithClose = raftAPI{} 22 | var _ raft.WithPeers = raftAPI{} 23 | var _ raft.WithPreVote = raftAPI{} 24 | 25 | type conn struct { 26 | clientConn *grpc.ClientConn 27 | client pb.RaftTransportClient 28 | mtx sync.Mutex 29 | } 30 | 31 | // Consumer returns a channel that can be used to consume and respond to RPC requests. 32 | func (r raftAPI) Consumer() <-chan raft.RPC { 33 | return r.manager.rpcChan 34 | } 35 | 36 | // LocalAddr is used to return our local address to distinguish from our peers. 37 | func (r raftAPI) LocalAddr() raft.ServerAddress { 38 | return r.manager.localAddress 39 | } 40 | 41 | func (r raftAPI) getPeer(target raft.ServerAddress) (pb.RaftTransportClient, error) { 42 | r.manager.connectionsMtx.Lock() 43 | c, ok := r.manager.connections[target] 44 | if !ok { 45 | c = &conn{} 46 | c.mtx.Lock() 47 | r.manager.connections[target] = c 48 | } 49 | r.manager.connectionsMtx.Unlock() 50 | if ok { 51 | c.mtx.Lock() 52 | } 53 | defer c.mtx.Unlock() 54 | if c.clientConn == nil { 55 | conn, err := grpc.Dial(string(target), r.manager.dialOptions...) 56 | if err != nil { 57 | return nil, err 58 | } 59 | c.clientConn = conn 60 | c.client = pb.NewRaftTransportClient(conn) 61 | } 62 | return c.client, nil 63 | } 64 | 65 | // AppendEntries sends the appropriate RPC to the target node. 66 | func (r raftAPI) AppendEntries(id raft.ServerID, target raft.ServerAddress, args *raft.AppendEntriesRequest, resp *raft.AppendEntriesResponse) error { 67 | c, err := r.getPeer(target) 68 | if err != nil { 69 | return err 70 | } 71 | ctx := context.TODO() 72 | if r.manager.heartbeatTimeout > 0 && isHeartbeat(args) { 73 | var cancel context.CancelFunc 74 | ctx, cancel = context.WithTimeout(ctx, r.manager.heartbeatTimeout) 75 | defer cancel() 76 | } 77 | ret, err := c.AppendEntries(ctx, encodeAppendEntriesRequest(args)) 78 | if err != nil { 79 | return err 80 | } 81 | *resp = *decodeAppendEntriesResponse(ret) 82 | return nil 83 | } 84 | 85 | // RequestVote sends the appropriate RPC to the target node. 86 | func (r raftAPI) RequestVote(id raft.ServerID, target raft.ServerAddress, args *raft.RequestVoteRequest, resp *raft.RequestVoteResponse) error { 87 | c, err := r.getPeer(target) 88 | if err != nil { 89 | return err 90 | } 91 | ret, err := c.RequestVote(context.TODO(), encodeRequestVoteRequest(args)) 92 | if err != nil { 93 | return err 94 | } 95 | *resp = *decodeRequestVoteResponse(ret) 96 | return nil 97 | } 98 | 99 | // TimeoutNow is used to start a leadership transfer to the target node. 100 | func (r raftAPI) TimeoutNow(id raft.ServerID, target raft.ServerAddress, args *raft.TimeoutNowRequest, resp *raft.TimeoutNowResponse) error { 101 | c, err := r.getPeer(target) 102 | if err != nil { 103 | return err 104 | } 105 | ret, err := c.TimeoutNow(context.TODO(), encodeTimeoutNowRequest(args)) 106 | if err != nil { 107 | return err 108 | } 109 | *resp = *decodeTimeoutNowResponse(ret) 110 | return nil 111 | } 112 | 113 | // RequestPreVote is the command used by a candidate to ask a Raft peer for a vote in an election. 114 | func (r raftAPI) RequestPreVote(id raft.ServerID, target raft.ServerAddress, args *raft.RequestPreVoteRequest, resp *raft.RequestPreVoteResponse) error { 115 | c, err := r.getPeer(target) 116 | if err != nil { 117 | return err 118 | } 119 | ret, err := c.RequestPreVote(context.TODO(), encodeRequestPreVoteRequest(args)) 120 | if err != nil { 121 | return err 122 | } 123 | *resp = *decodeRequestPreVoteResponse(ret) 124 | return nil 125 | } 126 | 127 | // InstallSnapshot is used to push a snapshot down to a follower. The data is read from 128 | // the ReadCloser and streamed to the client. 129 | func (r raftAPI) InstallSnapshot(id raft.ServerID, target raft.ServerAddress, req *raft.InstallSnapshotRequest, resp *raft.InstallSnapshotResponse, data io.Reader) error { 130 | c, err := r.getPeer(target) 131 | if err != nil { 132 | return err 133 | } 134 | stream, err := c.InstallSnapshot(context.TODO()) 135 | if err != nil { 136 | return err 137 | } 138 | if err := stream.Send(encodeInstallSnapshotRequest(req)); err != nil { 139 | return err 140 | } 141 | var buf [16384]byte 142 | for { 143 | n, err := data.Read(buf[:]) 144 | if err == io.EOF || (err == nil && n == 0) { 145 | break 146 | } 147 | if err != nil { 148 | return err 149 | } 150 | if err := stream.Send(&pb.InstallSnapshotRequest{ 151 | Data: buf[:n], 152 | }); err != nil { 153 | return err 154 | } 155 | } 156 | ret, err := stream.CloseAndRecv() 157 | if err != nil { 158 | return err 159 | } 160 | *resp = *decodeInstallSnapshotResponse(ret) 161 | return nil 162 | } 163 | 164 | // AppendEntriesPipeline returns an interface that can be used to pipeline 165 | // AppendEntries requests. 166 | func (r raftAPI) AppendEntriesPipeline(id raft.ServerID, target raft.ServerAddress) (raft.AppendPipeline, error) { 167 | c, err := r.getPeer(target) 168 | if err != nil { 169 | return nil, err 170 | } 171 | ctx := context.TODO() 172 | ctx, cancel := context.WithCancel(ctx) 173 | stream, err := c.AppendEntriesPipeline(ctx) 174 | if err != nil { 175 | cancel() 176 | return nil, err 177 | } 178 | rpa := raftPipelineAPI{ 179 | stream: stream, 180 | cancel: cancel, 181 | inflightCh: make(chan *appendFuture, 20), 182 | doneCh: make(chan raft.AppendFuture, 20), 183 | } 184 | go rpa.receiver() 185 | return rpa, nil 186 | } 187 | 188 | type raftPipelineAPI struct { 189 | stream pb.RaftTransport_AppendEntriesPipelineClient 190 | cancel func() 191 | inflightChMtx sync.Mutex 192 | inflightCh chan *appendFuture 193 | doneCh chan raft.AppendFuture 194 | } 195 | 196 | // AppendEntries is used to add another request to the pipeline. 197 | // The send may block which is an effective form of back-pressure. 198 | func (r raftPipelineAPI) AppendEntries(req *raft.AppendEntriesRequest, resp *raft.AppendEntriesResponse) (raft.AppendFuture, error) { 199 | af := &appendFuture{ 200 | start: time.Now(), 201 | request: req, 202 | done: make(chan struct{}), 203 | } 204 | if err := r.stream.Send(encodeAppendEntriesRequest(req)); err != nil { 205 | return nil, err 206 | } 207 | r.inflightChMtx.Lock() 208 | select { 209 | case <-r.stream.Context().Done(): 210 | default: 211 | r.inflightCh <- af 212 | } 213 | r.inflightChMtx.Unlock() 214 | return af, nil 215 | } 216 | 217 | // Consumer returns a channel that can be used to consume 218 | // response futures when they are ready. 219 | func (r raftPipelineAPI) Consumer() <-chan raft.AppendFuture { 220 | return r.doneCh 221 | } 222 | 223 | // Close closes the pipeline and cancels all inflight RPCs 224 | func (r raftPipelineAPI) Close() error { 225 | r.cancel() 226 | r.inflightChMtx.Lock() 227 | close(r.inflightCh) 228 | r.inflightChMtx.Unlock() 229 | return nil 230 | } 231 | 232 | func (r raftPipelineAPI) receiver() { 233 | for af := range r.inflightCh { 234 | msg, err := r.stream.Recv() 235 | if err != nil { 236 | af.err = err 237 | } else { 238 | af.response = *decodeAppendEntriesResponse(msg) 239 | } 240 | close(af.done) 241 | r.doneCh <- af 242 | } 243 | } 244 | 245 | type appendFuture struct { 246 | raft.AppendFuture 247 | 248 | start time.Time 249 | request *raft.AppendEntriesRequest 250 | response raft.AppendEntriesResponse 251 | err error 252 | done chan struct{} 253 | } 254 | 255 | // Error blocks until the future arrives and then 256 | // returns the error status of the future. 257 | // This may be called any number of times - all 258 | // calls will return the same value. 259 | // Note that it is not OK to call this method 260 | // twice concurrently on the same Future instance. 261 | func (f *appendFuture) Error() error { 262 | <-f.done 263 | return f.err 264 | } 265 | 266 | // Start returns the time that the append request was started. 267 | // It is always OK to call this method. 268 | func (f *appendFuture) Start() time.Time { 269 | return f.start 270 | } 271 | 272 | // Request holds the parameters of the AppendEntries call. 273 | // It is always OK to call this method. 274 | func (f *appendFuture) Request() *raft.AppendEntriesRequest { 275 | return f.request 276 | } 277 | 278 | // Response holds the results of the AppendEntries call. 279 | // This method must only be called after the Error 280 | // method returns, and will only be valid on success. 281 | func (f *appendFuture) Response() *raft.AppendEntriesResponse { 282 | return &f.response 283 | } 284 | 285 | // EncodePeer is used to serialize a peer's address. 286 | func (r raftAPI) EncodePeer(id raft.ServerID, addr raft.ServerAddress) []byte { 287 | return []byte(addr) 288 | } 289 | 290 | // DecodePeer is used to deserialize a peer's address. 291 | func (r raftAPI) DecodePeer(p []byte) raft.ServerAddress { 292 | return raft.ServerAddress(p) 293 | } 294 | 295 | // SetHeartbeatHandler is used to setup a heartbeat handler 296 | // as a fast-pass. This is to avoid head-of-line blocking from 297 | // disk IO. If a Transport does not support this, it can simply 298 | // ignore the call, and push the heartbeat onto the Consumer channel. 299 | func (r raftAPI) SetHeartbeatHandler(cb func(rpc raft.RPC)) { 300 | r.manager.heartbeatFuncMtx.Lock() 301 | r.manager.heartbeatFunc = cb 302 | r.manager.heartbeatFuncMtx.Unlock() 303 | } 304 | 305 | func (r raftAPI) Close() error { 306 | return r.manager.Close() 307 | } 308 | 309 | func (r raftAPI) Connect(target raft.ServerAddress, t raft.Transport) { 310 | _, _ = r.getPeer(target) 311 | } 312 | 313 | func (r raftAPI) Disconnect(target raft.ServerAddress) { 314 | r.manager.connectionsMtx.Lock() 315 | c, ok := r.manager.connections[target] 316 | if !ok { 317 | delete(r.manager.connections, target) 318 | } 319 | r.manager.connectionsMtx.Unlock() 320 | if ok { 321 | c.mtx.Lock() 322 | c.mtx.Unlock() 323 | _ = c.clientConn.Close() 324 | } 325 | } 326 | 327 | func (r raftAPI) DisconnectAll() { 328 | _ = r.manager.disconnectAll() 329 | } 330 | -------------------------------------------------------------------------------- /proto/transport_grpc.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go-grpc. DO NOT EDIT. 2 | // versions: 3 | // - protoc-gen-go-grpc v1.2.0 4 | // - protoc v3.21.3 5 | // source: transport.proto 6 | 7 | package proto 8 | 9 | import ( 10 | context "context" 11 | grpc "google.golang.org/grpc" 12 | codes "google.golang.org/grpc/codes" 13 | status "google.golang.org/grpc/status" 14 | ) 15 | 16 | // This is a compile-time assertion to ensure that this generated file 17 | // is compatible with the grpc package it is being compiled against. 18 | // Requires gRPC-Go v1.32.0 or later. 19 | const _ = grpc.SupportPackageIsVersion7 20 | 21 | // RaftTransportClient is the client API for RaftTransport service. 22 | // 23 | // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. 24 | type RaftTransportClient interface { 25 | // AppendEntriesPipeline opens an AppendEntries message stream. 26 | AppendEntriesPipeline(ctx context.Context, opts ...grpc.CallOption) (RaftTransport_AppendEntriesPipelineClient, error) 27 | // AppendEntries performs a single append entries request / response. 28 | AppendEntries(ctx context.Context, in *AppendEntriesRequest, opts ...grpc.CallOption) (*AppendEntriesResponse, error) 29 | // RequestVote is the command used by a candidate to ask a Raft peer for a vote in an election. 30 | RequestVote(ctx context.Context, in *RequestVoteRequest, opts ...grpc.CallOption) (*RequestVoteResponse, error) 31 | // TimeoutNow is used to start a leadership transfer to the target node. 32 | TimeoutNow(ctx context.Context, in *TimeoutNowRequest, opts ...grpc.CallOption) (*TimeoutNowResponse, error) 33 | // InstallSnapshot is the command sent to a Raft peer to bootstrap its log (and state machine) from a snapshot on another peer. 34 | InstallSnapshot(ctx context.Context, opts ...grpc.CallOption) (RaftTransport_InstallSnapshotClient, error) 35 | // RequestPreVote is the command used by a candidate to ask a Raft peer for a vote in an election. 36 | RequestPreVote(ctx context.Context, in *RequestPreVoteRequest, opts ...grpc.CallOption) (*RequestPreVoteResponse, error) 37 | } 38 | 39 | type raftTransportClient struct { 40 | cc grpc.ClientConnInterface 41 | } 42 | 43 | func NewRaftTransportClient(cc grpc.ClientConnInterface) RaftTransportClient { 44 | return &raftTransportClient{cc} 45 | } 46 | 47 | func (c *raftTransportClient) AppendEntriesPipeline(ctx context.Context, opts ...grpc.CallOption) (RaftTransport_AppendEntriesPipelineClient, error) { 48 | stream, err := c.cc.NewStream(ctx, &RaftTransport_ServiceDesc.Streams[0], "/RaftTransport/AppendEntriesPipeline", opts...) 49 | if err != nil { 50 | return nil, err 51 | } 52 | x := &raftTransportAppendEntriesPipelineClient{stream} 53 | return x, nil 54 | } 55 | 56 | type RaftTransport_AppendEntriesPipelineClient interface { 57 | Send(*AppendEntriesRequest) error 58 | Recv() (*AppendEntriesResponse, error) 59 | grpc.ClientStream 60 | } 61 | 62 | type raftTransportAppendEntriesPipelineClient struct { 63 | grpc.ClientStream 64 | } 65 | 66 | func (x *raftTransportAppendEntriesPipelineClient) Send(m *AppendEntriesRequest) error { 67 | return x.ClientStream.SendMsg(m) 68 | } 69 | 70 | func (x *raftTransportAppendEntriesPipelineClient) Recv() (*AppendEntriesResponse, error) { 71 | m := new(AppendEntriesResponse) 72 | if err := x.ClientStream.RecvMsg(m); err != nil { 73 | return nil, err 74 | } 75 | return m, nil 76 | } 77 | 78 | func (c *raftTransportClient) AppendEntries(ctx context.Context, in *AppendEntriesRequest, opts ...grpc.CallOption) (*AppendEntriesResponse, error) { 79 | out := new(AppendEntriesResponse) 80 | err := c.cc.Invoke(ctx, "/RaftTransport/AppendEntries", in, out, opts...) 81 | if err != nil { 82 | return nil, err 83 | } 84 | return out, nil 85 | } 86 | 87 | func (c *raftTransportClient) RequestVote(ctx context.Context, in *RequestVoteRequest, opts ...grpc.CallOption) (*RequestVoteResponse, error) { 88 | out := new(RequestVoteResponse) 89 | err := c.cc.Invoke(ctx, "/RaftTransport/RequestVote", in, out, opts...) 90 | if err != nil { 91 | return nil, err 92 | } 93 | return out, nil 94 | } 95 | 96 | func (c *raftTransportClient) TimeoutNow(ctx context.Context, in *TimeoutNowRequest, opts ...grpc.CallOption) (*TimeoutNowResponse, error) { 97 | out := new(TimeoutNowResponse) 98 | err := c.cc.Invoke(ctx, "/RaftTransport/TimeoutNow", in, out, opts...) 99 | if err != nil { 100 | return nil, err 101 | } 102 | return out, nil 103 | } 104 | 105 | func (c *raftTransportClient) InstallSnapshot(ctx context.Context, opts ...grpc.CallOption) (RaftTransport_InstallSnapshotClient, error) { 106 | stream, err := c.cc.NewStream(ctx, &RaftTransport_ServiceDesc.Streams[1], "/RaftTransport/InstallSnapshot", opts...) 107 | if err != nil { 108 | return nil, err 109 | } 110 | x := &raftTransportInstallSnapshotClient{stream} 111 | return x, nil 112 | } 113 | 114 | type RaftTransport_InstallSnapshotClient interface { 115 | Send(*InstallSnapshotRequest) error 116 | CloseAndRecv() (*InstallSnapshotResponse, error) 117 | grpc.ClientStream 118 | } 119 | 120 | type raftTransportInstallSnapshotClient struct { 121 | grpc.ClientStream 122 | } 123 | 124 | func (x *raftTransportInstallSnapshotClient) Send(m *InstallSnapshotRequest) error { 125 | return x.ClientStream.SendMsg(m) 126 | } 127 | 128 | func (x *raftTransportInstallSnapshotClient) CloseAndRecv() (*InstallSnapshotResponse, error) { 129 | if err := x.ClientStream.CloseSend(); err != nil { 130 | return nil, err 131 | } 132 | m := new(InstallSnapshotResponse) 133 | if err := x.ClientStream.RecvMsg(m); err != nil { 134 | return nil, err 135 | } 136 | return m, nil 137 | } 138 | 139 | func (c *raftTransportClient) RequestPreVote(ctx context.Context, in *RequestPreVoteRequest, opts ...grpc.CallOption) (*RequestPreVoteResponse, error) { 140 | out := new(RequestPreVoteResponse) 141 | err := c.cc.Invoke(ctx, "/RaftTransport/RequestPreVote", in, out, opts...) 142 | if err != nil { 143 | return nil, err 144 | } 145 | return out, nil 146 | } 147 | 148 | // RaftTransportServer is the server API for RaftTransport service. 149 | // All implementations must embed UnimplementedRaftTransportServer 150 | // for forward compatibility 151 | type RaftTransportServer interface { 152 | // AppendEntriesPipeline opens an AppendEntries message stream. 153 | AppendEntriesPipeline(RaftTransport_AppendEntriesPipelineServer) error 154 | // AppendEntries performs a single append entries request / response. 155 | AppendEntries(context.Context, *AppendEntriesRequest) (*AppendEntriesResponse, error) 156 | // RequestVote is the command used by a candidate to ask a Raft peer for a vote in an election. 157 | RequestVote(context.Context, *RequestVoteRequest) (*RequestVoteResponse, error) 158 | // TimeoutNow is used to start a leadership transfer to the target node. 159 | TimeoutNow(context.Context, *TimeoutNowRequest) (*TimeoutNowResponse, error) 160 | // InstallSnapshot is the command sent to a Raft peer to bootstrap its log (and state machine) from a snapshot on another peer. 161 | InstallSnapshot(RaftTransport_InstallSnapshotServer) error 162 | // RequestPreVote is the command used by a candidate to ask a Raft peer for a vote in an election. 163 | RequestPreVote(context.Context, *RequestPreVoteRequest) (*RequestPreVoteResponse, error) 164 | mustEmbedUnimplementedRaftTransportServer() 165 | } 166 | 167 | // UnimplementedRaftTransportServer must be embedded to have forward compatible implementations. 168 | type UnimplementedRaftTransportServer struct { 169 | } 170 | 171 | func (UnimplementedRaftTransportServer) AppendEntriesPipeline(RaftTransport_AppendEntriesPipelineServer) error { 172 | return status.Errorf(codes.Unimplemented, "method AppendEntriesPipeline not implemented") 173 | } 174 | func (UnimplementedRaftTransportServer) AppendEntries(context.Context, *AppendEntriesRequest) (*AppendEntriesResponse, error) { 175 | return nil, status.Errorf(codes.Unimplemented, "method AppendEntries not implemented") 176 | } 177 | func (UnimplementedRaftTransportServer) RequestVote(context.Context, *RequestVoteRequest) (*RequestVoteResponse, error) { 178 | return nil, status.Errorf(codes.Unimplemented, "method RequestVote not implemented") 179 | } 180 | func (UnimplementedRaftTransportServer) TimeoutNow(context.Context, *TimeoutNowRequest) (*TimeoutNowResponse, error) { 181 | return nil, status.Errorf(codes.Unimplemented, "method TimeoutNow not implemented") 182 | } 183 | func (UnimplementedRaftTransportServer) InstallSnapshot(RaftTransport_InstallSnapshotServer) error { 184 | return status.Errorf(codes.Unimplemented, "method InstallSnapshot not implemented") 185 | } 186 | func (UnimplementedRaftTransportServer) RequestPreVote(context.Context, *RequestPreVoteRequest) (*RequestPreVoteResponse, error) { 187 | return nil, status.Errorf(codes.Unimplemented, "method RequestPreVote not implemented") 188 | } 189 | func (UnimplementedRaftTransportServer) mustEmbedUnimplementedRaftTransportServer() {} 190 | 191 | // UnsafeRaftTransportServer may be embedded to opt out of forward compatibility for this service. 192 | // Use of this interface is not recommended, as added methods to RaftTransportServer will 193 | // result in compilation errors. 194 | type UnsafeRaftTransportServer interface { 195 | mustEmbedUnimplementedRaftTransportServer() 196 | } 197 | 198 | func RegisterRaftTransportServer(s grpc.ServiceRegistrar, srv RaftTransportServer) { 199 | s.RegisterService(&RaftTransport_ServiceDesc, srv) 200 | } 201 | 202 | func _RaftTransport_AppendEntriesPipeline_Handler(srv interface{}, stream grpc.ServerStream) error { 203 | return srv.(RaftTransportServer).AppendEntriesPipeline(&raftTransportAppendEntriesPipelineServer{stream}) 204 | } 205 | 206 | type RaftTransport_AppendEntriesPipelineServer interface { 207 | Send(*AppendEntriesResponse) error 208 | Recv() (*AppendEntriesRequest, error) 209 | grpc.ServerStream 210 | } 211 | 212 | type raftTransportAppendEntriesPipelineServer struct { 213 | grpc.ServerStream 214 | } 215 | 216 | func (x *raftTransportAppendEntriesPipelineServer) Send(m *AppendEntriesResponse) error { 217 | return x.ServerStream.SendMsg(m) 218 | } 219 | 220 | func (x *raftTransportAppendEntriesPipelineServer) Recv() (*AppendEntriesRequest, error) { 221 | m := new(AppendEntriesRequest) 222 | if err := x.ServerStream.RecvMsg(m); err != nil { 223 | return nil, err 224 | } 225 | return m, nil 226 | } 227 | 228 | func _RaftTransport_AppendEntries_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { 229 | in := new(AppendEntriesRequest) 230 | if err := dec(in); err != nil { 231 | return nil, err 232 | } 233 | if interceptor == nil { 234 | return srv.(RaftTransportServer).AppendEntries(ctx, in) 235 | } 236 | info := &grpc.UnaryServerInfo{ 237 | Server: srv, 238 | FullMethod: "/RaftTransport/AppendEntries", 239 | } 240 | handler := func(ctx context.Context, req interface{}) (interface{}, error) { 241 | return srv.(RaftTransportServer).AppendEntries(ctx, req.(*AppendEntriesRequest)) 242 | } 243 | return interceptor(ctx, in, info, handler) 244 | } 245 | 246 | func _RaftTransport_RequestVote_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { 247 | in := new(RequestVoteRequest) 248 | if err := dec(in); err != nil { 249 | return nil, err 250 | } 251 | if interceptor == nil { 252 | return srv.(RaftTransportServer).RequestVote(ctx, in) 253 | } 254 | info := &grpc.UnaryServerInfo{ 255 | Server: srv, 256 | FullMethod: "/RaftTransport/RequestVote", 257 | } 258 | handler := func(ctx context.Context, req interface{}) (interface{}, error) { 259 | return srv.(RaftTransportServer).RequestVote(ctx, req.(*RequestVoteRequest)) 260 | } 261 | return interceptor(ctx, in, info, handler) 262 | } 263 | 264 | func _RaftTransport_TimeoutNow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { 265 | in := new(TimeoutNowRequest) 266 | if err := dec(in); err != nil { 267 | return nil, err 268 | } 269 | if interceptor == nil { 270 | return srv.(RaftTransportServer).TimeoutNow(ctx, in) 271 | } 272 | info := &grpc.UnaryServerInfo{ 273 | Server: srv, 274 | FullMethod: "/RaftTransport/TimeoutNow", 275 | } 276 | handler := func(ctx context.Context, req interface{}) (interface{}, error) { 277 | return srv.(RaftTransportServer).TimeoutNow(ctx, req.(*TimeoutNowRequest)) 278 | } 279 | return interceptor(ctx, in, info, handler) 280 | } 281 | 282 | func _RaftTransport_InstallSnapshot_Handler(srv interface{}, stream grpc.ServerStream) error { 283 | return srv.(RaftTransportServer).InstallSnapshot(&raftTransportInstallSnapshotServer{stream}) 284 | } 285 | 286 | type RaftTransport_InstallSnapshotServer interface { 287 | SendAndClose(*InstallSnapshotResponse) error 288 | Recv() (*InstallSnapshotRequest, error) 289 | grpc.ServerStream 290 | } 291 | 292 | type raftTransportInstallSnapshotServer struct { 293 | grpc.ServerStream 294 | } 295 | 296 | func (x *raftTransportInstallSnapshotServer) SendAndClose(m *InstallSnapshotResponse) error { 297 | return x.ServerStream.SendMsg(m) 298 | } 299 | 300 | func (x *raftTransportInstallSnapshotServer) Recv() (*InstallSnapshotRequest, error) { 301 | m := new(InstallSnapshotRequest) 302 | if err := x.ServerStream.RecvMsg(m); err != nil { 303 | return nil, err 304 | } 305 | return m, nil 306 | } 307 | 308 | func _RaftTransport_RequestPreVote_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { 309 | in := new(RequestPreVoteRequest) 310 | if err := dec(in); err != nil { 311 | return nil, err 312 | } 313 | if interceptor == nil { 314 | return srv.(RaftTransportServer).RequestPreVote(ctx, in) 315 | } 316 | info := &grpc.UnaryServerInfo{ 317 | Server: srv, 318 | FullMethod: "/RaftTransport/RequestPreVote", 319 | } 320 | handler := func(ctx context.Context, req interface{}) (interface{}, error) { 321 | return srv.(RaftTransportServer).RequestPreVote(ctx, req.(*RequestPreVoteRequest)) 322 | } 323 | return interceptor(ctx, in, info, handler) 324 | } 325 | 326 | // RaftTransport_ServiceDesc is the grpc.ServiceDesc for RaftTransport service. 327 | // It's only intended for direct use with grpc.RegisterService, 328 | // and not to be introspected or modified (even as a copy) 329 | var RaftTransport_ServiceDesc = grpc.ServiceDesc{ 330 | ServiceName: "RaftTransport", 331 | HandlerType: (*RaftTransportServer)(nil), 332 | Methods: []grpc.MethodDesc{ 333 | { 334 | MethodName: "AppendEntries", 335 | Handler: _RaftTransport_AppendEntries_Handler, 336 | }, 337 | { 338 | MethodName: "RequestVote", 339 | Handler: _RaftTransport_RequestVote_Handler, 340 | }, 341 | { 342 | MethodName: "TimeoutNow", 343 | Handler: _RaftTransport_TimeoutNow_Handler, 344 | }, 345 | { 346 | MethodName: "RequestPreVote", 347 | Handler: _RaftTransport_RequestPreVote_Handler, 348 | }, 349 | }, 350 | Streams: []grpc.StreamDesc{ 351 | { 352 | StreamName: "AppendEntriesPipeline", 353 | Handler: _RaftTransport_AppendEntriesPipeline_Handler, 354 | ServerStreams: true, 355 | ClientStreams: true, 356 | }, 357 | { 358 | StreamName: "InstallSnapshot", 359 | Handler: _RaftTransport_InstallSnapshot_Handler, 360 | ClientStreams: true, 361 | }, 362 | }, 363 | Metadata: "transport.proto", 364 | } 365 | -------------------------------------------------------------------------------- /go.sum: -------------------------------------------------------------------------------- 1 | cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= 2 | cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= 3 | github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= 4 | github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= 5 | github.com/DataDog/zstd v1.5.2/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= 6 | github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= 7 | github.com/Sereal/Sereal/Go/sereal v0.0.0-20231009093132-b9187f1a92c6/go.mod h1:JwrycNnC8+sZPDyzM3MQ86LvaGzSpfxg885KOOwFRW4= 8 | github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= 9 | github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= 10 | github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= 11 | github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= 12 | github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= 13 | github.com/armon/go-metrics v0.4.1 h1:hR91U9KYmb6bLBYLQjyM+3j+rcd/UhE+G78SFnF8gJA= 14 | github.com/armon/go-metrics v0.4.1/go.mod h1:E6amYzXo6aW1tqzoZGT755KkbgrJsSdpwZ+3JqfkOG4= 15 | github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= 16 | github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= 17 | github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= 18 | github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= 19 | github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= 20 | github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 21 | github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag= 22 | github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I= 23 | github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= 24 | github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= 25 | github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= 26 | github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= 27 | github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= 28 | github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= 29 | github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= 30 | github.com/davecgh/go-xdr v0.0.0-20161123171359-e6a2ba005892/go.mod h1:CTDl0pzVzE5DEzZhPfvhY/9sPFMQIxaJ9VAMs9AagrE= 31 | github.com/dchest/siphash v1.2.3/go.mod h1:0NvQU092bT0ipiFN++/rXm69QG9tVxLAlQHIXMPAkHc= 32 | github.com/dgryski/go-ddmin v0.0.0-20210904190556-96a6d69f1034/go.mod h1:zz4KxBkcXUWKjIcrc+uphJ1gPh/t18ymGm3PmQ+VGTk= 33 | github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= 34 | github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= 35 | github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= 36 | github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= 37 | github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= 38 | github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= 39 | github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= 40 | github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= 41 | github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= 42 | github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= 43 | github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= 44 | github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= 45 | github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= 46 | github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= 47 | github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= 48 | github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= 49 | github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= 50 | github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= 51 | github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= 52 | github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= 53 | github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= 54 | github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= 55 | github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= 56 | github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= 57 | github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= 58 | github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= 59 | github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= 60 | github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= 61 | github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= 62 | github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= 63 | github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= 64 | github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= 65 | github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= 66 | github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= 67 | github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= 68 | github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= 69 | github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= 70 | github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= 71 | github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= 72 | github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= 73 | github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= 74 | github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= 75 | github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0= 76 | github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= 77 | github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= 78 | github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= 79 | github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= 80 | github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= 81 | github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= 82 | github.com/hashicorp/go-hclog v1.6.2 h1:NOtoftovWkDheyUM/8JW3QMiXyxJK3uHRK7wV04nD2I= 83 | github.com/hashicorp/go-hclog v1.6.2/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= 84 | github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= 85 | github.com/hashicorp/go-immutable-radix v1.3.1 h1:DKHmCUm2hRBK510BaiZlwvpD40f8bJFeZnpfm2KLowc= 86 | github.com/hashicorp/go-immutable-radix v1.3.1/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= 87 | github.com/hashicorp/go-msgpack/v2 v2.1.1 h1:xQEY9yB2wnHitoSzk/B9UjXWRQ67QKu5AOm8aFp8N3I= 88 | github.com/hashicorp/go-msgpack/v2 v2.1.1/go.mod h1:upybraOAblm4S7rx0+jeNy+CWWhzywQsSRV5033mMu4= 89 | github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= 90 | github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= 91 | github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs= 92 | github.com/hashicorp/go-uuid v1.0.0 h1:RS8zrF7PhGwyNPOtxSClXXj9HA8feRnJzgnI1RJCSnM= 93 | github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= 94 | github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= 95 | github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= 96 | github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= 97 | github.com/hashicorp/raft v1.7.0 h1:4u24Qn6lQ6uwziM++UgsyiT64Q8GyRn43CV41qPiz1o= 98 | github.com/hashicorp/raft v1.7.0/go.mod h1:N1sKh6Vn47mrWvEArQgILTyng8GoDRNYlgKyK7PMjs0= 99 | github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= 100 | github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= 101 | github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= 102 | github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= 103 | github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= 104 | github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= 105 | github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= 106 | github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= 107 | github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= 108 | github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= 109 | github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= 110 | github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= 111 | github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= 112 | github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= 113 | github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= 114 | github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40= 115 | github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= 116 | github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= 117 | github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= 118 | github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= 119 | github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= 120 | github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= 121 | github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= 122 | github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= 123 | github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= 124 | github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= 125 | github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= 126 | github.com/pascaldekloe/goe v0.1.0 h1:cBOtyMzM9HTpWjXfbbunk26uA6nG3a8n06Wieeh0MwY= 127 | github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= 128 | github.com/philhofer/fwd v1.1.2/go.mod h1:qkPdfjR2SIEbspLqpe1tO4n5yICnr2DY7mqEx2tUTP0= 129 | github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 130 | github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 131 | github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= 132 | github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 133 | github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= 134 | github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= 135 | github.com/pquerna/ffjson v0.0.0-20190930134022-aa0246cd15f7/go.mod h1:YARuvh7BUWHNhzDq2OM5tzR2RiCcN2D7sapiKyCel/M= 136 | github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= 137 | github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= 138 | github.com/prometheus/client_golang v1.4.0/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= 139 | github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= 140 | github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= 141 | github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= 142 | github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= 143 | github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= 144 | github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= 145 | github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= 146 | github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= 147 | github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= 148 | github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= 149 | github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= 150 | github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= 151 | github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= 152 | github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= 153 | github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= 154 | github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= 155 | github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= 156 | github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= 157 | github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= 158 | github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= 159 | github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= 160 | github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= 161 | github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= 162 | github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals= 163 | github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= 164 | github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= 165 | github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= 166 | github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= 167 | github.com/tinylib/msgp v1.1.8/go.mod h1:qkpG+2ldGg4xRFmx+jfTvZPxfGFhi64BcnL9vkCm/Tw= 168 | github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= 169 | github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= 170 | github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= 171 | go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= 172 | go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= 173 | go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= 174 | golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= 175 | golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= 176 | golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= 177 | golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= 178 | golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= 179 | golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= 180 | golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= 181 | golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= 182 | golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= 183 | golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= 184 | golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= 185 | golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= 186 | golang.org/x/lint v0.0.0-20210508222113-6edffad5e616 h1:VLliZ0d+/avPrXXH+OakdXhpJuEoBZuwh1m2j7U6Iug= 187 | golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= 188 | golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= 189 | golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= 190 | golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= 191 | golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= 192 | golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= 193 | golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= 194 | golang.org/x/mod v0.13.0 h1:I/DsJXRlw/8l/0c24sM9yb0T4z9liZTduXvdAWYiysY= 195 | golang.org/x/mod v0.13.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= 196 | golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= 197 | golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= 198 | golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= 199 | golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= 200 | golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= 201 | golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= 202 | golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= 203 | golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= 204 | golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= 205 | golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= 206 | golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= 207 | golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= 208 | golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= 209 | golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= 210 | golang.org/x/net v0.3.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE= 211 | golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= 212 | golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= 213 | golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= 214 | golang.org/x/net v0.16.0 h1:7eBu7KsSvFDtSXUIDbh3aqlK4DPsZ1rByC8PFfBThos= 215 | golang.org/x/net v0.16.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= 216 | golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= 217 | golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= 218 | golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 219 | golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 220 | golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 221 | golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 222 | golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 223 | golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 224 | golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 225 | golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 226 | golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= 227 | golang.org/x/sync v0.4.0 h1:zxkM55ReGkDlKSM+Fu41A+zmbZuaPVbGMzvvdUPznYQ= 228 | golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= 229 | golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= 230 | golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= 231 | golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= 232 | golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= 233 | golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 234 | golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 235 | golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 236 | golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 237 | golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 238 | golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 239 | golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 240 | golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 241 | golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 242 | golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 243 | golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 244 | golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 245 | golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 246 | golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 247 | golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 248 | golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 249 | golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 250 | golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 251 | golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 252 | golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= 253 | golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 254 | golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= 255 | golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= 256 | golang.org/x/term v0.3.0/go.mod h1:q750SLmJuPmVoN1blW3UFBPREJfb1KmY3vwxfr+nFDA= 257 | golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= 258 | golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= 259 | golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= 260 | golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= 261 | golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= 262 | golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= 263 | golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= 264 | golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= 265 | golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= 266 | golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= 267 | golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= 268 | golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= 269 | golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= 270 | golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= 271 | golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= 272 | golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= 273 | golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= 274 | golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= 275 | golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= 276 | golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= 277 | golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= 278 | golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= 279 | golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= 280 | golang.org/x/tools v0.4.0/go.mod h1:UE5sM2OK9E/d67R0ANs2xJizIymRP5gJU295PvKXxjQ= 281 | golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= 282 | golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= 283 | golang.org/x/tools v0.14.0 h1:jvNa2pY0M4r62jkRQ6RwEZZyPcymeL9XZMLBbV7U2nc= 284 | golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg= 285 | golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 286 | golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 287 | golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 288 | golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 289 | google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= 290 | google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= 291 | google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= 292 | google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= 293 | google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= 294 | google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= 295 | google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= 296 | google.golang.org/genproto v0.0.0-20210903162649-d08c68adba83 h1:3V2dxSZpz4zozWWUq36vUxXEKnSYitEH2LdsAx+RUmg= 297 | google.golang.org/genproto v0.0.0-20210903162649-d08c68adba83/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= 298 | google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= 299 | google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= 300 | google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= 301 | google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= 302 | google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= 303 | google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= 304 | google.golang.org/grpc v1.40.0 h1:AGJ0Ih4mHjSeibYkFGh1dD9KJ/eOtZ93I6hoHhukQ5Q= 305 | google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= 306 | google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= 307 | google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= 308 | google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= 309 | google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= 310 | google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= 311 | google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= 312 | google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= 313 | google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= 314 | google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= 315 | google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= 316 | google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= 317 | google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= 318 | google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w= 319 | google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= 320 | gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= 321 | gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= 322 | gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= 323 | gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= 324 | gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= 325 | gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= 326 | gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= 327 | gopkg.in/vmihailenco/msgpack.v2 v2.9.2/go.mod h1:/3Dn1Npt9+MYyLpYYXjInO/5jvMLamn+AEGwNEOatn8= 328 | gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= 329 | gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= 330 | gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= 331 | gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= 332 | gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= 333 | gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= 334 | gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= 335 | gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= 336 | gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= 337 | honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= 338 | honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= 339 | -------------------------------------------------------------------------------- /proto/transport.pb.go: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. DO NOT EDIT. 2 | // versions: 3 | // protoc-gen-go v1.28.0 4 | // protoc v3.21.3 5 | // source: transport.proto 6 | 7 | package proto 8 | 9 | import ( 10 | protoreflect "google.golang.org/protobuf/reflect/protoreflect" 11 | protoimpl "google.golang.org/protobuf/runtime/protoimpl" 12 | timestamppb "google.golang.org/protobuf/types/known/timestamppb" 13 | reflect "reflect" 14 | sync "sync" 15 | ) 16 | 17 | const ( 18 | // Verify that this generated code is sufficiently up-to-date. 19 | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) 20 | // Verify that runtime/protoimpl is sufficiently up-to-date. 21 | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) 22 | ) 23 | 24 | type Log_LogType int32 25 | 26 | const ( 27 | Log_LOG_COMMAND Log_LogType = 0 28 | Log_LOG_NOOP Log_LogType = 1 29 | Log_LOG_ADD_PEER_DEPRECATED Log_LogType = 2 30 | Log_LOG_REMOVE_PEER_DEPRECATED Log_LogType = 3 31 | Log_LOG_BARRIER Log_LogType = 4 32 | Log_LOG_CONFIGURATION Log_LogType = 5 33 | ) 34 | 35 | // Enum value maps for Log_LogType. 36 | var ( 37 | Log_LogType_name = map[int32]string{ 38 | 0: "LOG_COMMAND", 39 | 1: "LOG_NOOP", 40 | 2: "LOG_ADD_PEER_DEPRECATED", 41 | 3: "LOG_REMOVE_PEER_DEPRECATED", 42 | 4: "LOG_BARRIER", 43 | 5: "LOG_CONFIGURATION", 44 | } 45 | Log_LogType_value = map[string]int32{ 46 | "LOG_COMMAND": 0, 47 | "LOG_NOOP": 1, 48 | "LOG_ADD_PEER_DEPRECATED": 2, 49 | "LOG_REMOVE_PEER_DEPRECATED": 3, 50 | "LOG_BARRIER": 4, 51 | "LOG_CONFIGURATION": 5, 52 | } 53 | ) 54 | 55 | func (x Log_LogType) Enum() *Log_LogType { 56 | p := new(Log_LogType) 57 | *p = x 58 | return p 59 | } 60 | 61 | func (x Log_LogType) String() string { 62 | return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) 63 | } 64 | 65 | func (Log_LogType) Descriptor() protoreflect.EnumDescriptor { 66 | return file_transport_proto_enumTypes[0].Descriptor() 67 | } 68 | 69 | func (Log_LogType) Type() protoreflect.EnumType { 70 | return &file_transport_proto_enumTypes[0] 71 | } 72 | 73 | func (x Log_LogType) Number() protoreflect.EnumNumber { 74 | return protoreflect.EnumNumber(x) 75 | } 76 | 77 | // Deprecated: Use Log_LogType.Descriptor instead. 78 | func (Log_LogType) EnumDescriptor() ([]byte, []int) { 79 | return file_transport_proto_rawDescGZIP(), []int{1, 0} 80 | } 81 | 82 | type RPCHeader struct { 83 | state protoimpl.MessageState 84 | sizeCache protoimpl.SizeCache 85 | unknownFields protoimpl.UnknownFields 86 | 87 | ProtocolVersion int64 `protobuf:"varint,1,opt,name=protocol_version,json=protocolVersion,proto3" json:"protocol_version,omitempty"` 88 | Id []byte `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"` 89 | Addr []byte `protobuf:"bytes,3,opt,name=addr,proto3" json:"addr,omitempty"` 90 | } 91 | 92 | func (x *RPCHeader) Reset() { 93 | *x = RPCHeader{} 94 | if protoimpl.UnsafeEnabled { 95 | mi := &file_transport_proto_msgTypes[0] 96 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 97 | ms.StoreMessageInfo(mi) 98 | } 99 | } 100 | 101 | func (x *RPCHeader) String() string { 102 | return protoimpl.X.MessageStringOf(x) 103 | } 104 | 105 | func (*RPCHeader) ProtoMessage() {} 106 | 107 | func (x *RPCHeader) ProtoReflect() protoreflect.Message { 108 | mi := &file_transport_proto_msgTypes[0] 109 | if protoimpl.UnsafeEnabled && x != nil { 110 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 111 | if ms.LoadMessageInfo() == nil { 112 | ms.StoreMessageInfo(mi) 113 | } 114 | return ms 115 | } 116 | return mi.MessageOf(x) 117 | } 118 | 119 | // Deprecated: Use RPCHeader.ProtoReflect.Descriptor instead. 120 | func (*RPCHeader) Descriptor() ([]byte, []int) { 121 | return file_transport_proto_rawDescGZIP(), []int{0} 122 | } 123 | 124 | func (x *RPCHeader) GetProtocolVersion() int64 { 125 | if x != nil { 126 | return x.ProtocolVersion 127 | } 128 | return 0 129 | } 130 | 131 | func (x *RPCHeader) GetId() []byte { 132 | if x != nil { 133 | return x.Id 134 | } 135 | return nil 136 | } 137 | 138 | func (x *RPCHeader) GetAddr() []byte { 139 | if x != nil { 140 | return x.Addr 141 | } 142 | return nil 143 | } 144 | 145 | type Log struct { 146 | state protoimpl.MessageState 147 | sizeCache protoimpl.SizeCache 148 | unknownFields protoimpl.UnknownFields 149 | 150 | Index uint64 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"` 151 | Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` 152 | Type Log_LogType `protobuf:"varint,3,opt,name=type,proto3,enum=Log_LogType" json:"type,omitempty"` 153 | Data []byte `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"` 154 | Extensions []byte `protobuf:"bytes,5,opt,name=extensions,proto3" json:"extensions,omitempty"` 155 | AppendedAt *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=appended_at,json=appendedAt,proto3" json:"appended_at,omitempty"` 156 | } 157 | 158 | func (x *Log) Reset() { 159 | *x = Log{} 160 | if protoimpl.UnsafeEnabled { 161 | mi := &file_transport_proto_msgTypes[1] 162 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 163 | ms.StoreMessageInfo(mi) 164 | } 165 | } 166 | 167 | func (x *Log) String() string { 168 | return protoimpl.X.MessageStringOf(x) 169 | } 170 | 171 | func (*Log) ProtoMessage() {} 172 | 173 | func (x *Log) ProtoReflect() protoreflect.Message { 174 | mi := &file_transport_proto_msgTypes[1] 175 | if protoimpl.UnsafeEnabled && x != nil { 176 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 177 | if ms.LoadMessageInfo() == nil { 178 | ms.StoreMessageInfo(mi) 179 | } 180 | return ms 181 | } 182 | return mi.MessageOf(x) 183 | } 184 | 185 | // Deprecated: Use Log.ProtoReflect.Descriptor instead. 186 | func (*Log) Descriptor() ([]byte, []int) { 187 | return file_transport_proto_rawDescGZIP(), []int{1} 188 | } 189 | 190 | func (x *Log) GetIndex() uint64 { 191 | if x != nil { 192 | return x.Index 193 | } 194 | return 0 195 | } 196 | 197 | func (x *Log) GetTerm() uint64 { 198 | if x != nil { 199 | return x.Term 200 | } 201 | return 0 202 | } 203 | 204 | func (x *Log) GetType() Log_LogType { 205 | if x != nil { 206 | return x.Type 207 | } 208 | return Log_LOG_COMMAND 209 | } 210 | 211 | func (x *Log) GetData() []byte { 212 | if x != nil { 213 | return x.Data 214 | } 215 | return nil 216 | } 217 | 218 | func (x *Log) GetExtensions() []byte { 219 | if x != nil { 220 | return x.Extensions 221 | } 222 | return nil 223 | } 224 | 225 | func (x *Log) GetAppendedAt() *timestamppb.Timestamp { 226 | if x != nil { 227 | return x.AppendedAt 228 | } 229 | return nil 230 | } 231 | 232 | type AppendEntriesRequest struct { 233 | state protoimpl.MessageState 234 | sizeCache protoimpl.SizeCache 235 | unknownFields protoimpl.UnknownFields 236 | 237 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 238 | Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` 239 | Leader []byte `protobuf:"bytes,3,opt,name=leader,proto3" json:"leader,omitempty"` 240 | PrevLogEntry uint64 `protobuf:"varint,4,opt,name=prev_log_entry,json=prevLogEntry,proto3" json:"prev_log_entry,omitempty"` 241 | PrevLogTerm uint64 `protobuf:"varint,5,opt,name=prev_log_term,json=prevLogTerm,proto3" json:"prev_log_term,omitempty"` 242 | Entries []*Log `protobuf:"bytes,6,rep,name=entries,proto3" json:"entries,omitempty"` 243 | LeaderCommitIndex uint64 `protobuf:"varint,7,opt,name=leader_commit_index,json=leaderCommitIndex,proto3" json:"leader_commit_index,omitempty"` 244 | } 245 | 246 | func (x *AppendEntriesRequest) Reset() { 247 | *x = AppendEntriesRequest{} 248 | if protoimpl.UnsafeEnabled { 249 | mi := &file_transport_proto_msgTypes[2] 250 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 251 | ms.StoreMessageInfo(mi) 252 | } 253 | } 254 | 255 | func (x *AppendEntriesRequest) String() string { 256 | return protoimpl.X.MessageStringOf(x) 257 | } 258 | 259 | func (*AppendEntriesRequest) ProtoMessage() {} 260 | 261 | func (x *AppendEntriesRequest) ProtoReflect() protoreflect.Message { 262 | mi := &file_transport_proto_msgTypes[2] 263 | if protoimpl.UnsafeEnabled && x != nil { 264 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 265 | if ms.LoadMessageInfo() == nil { 266 | ms.StoreMessageInfo(mi) 267 | } 268 | return ms 269 | } 270 | return mi.MessageOf(x) 271 | } 272 | 273 | // Deprecated: Use AppendEntriesRequest.ProtoReflect.Descriptor instead. 274 | func (*AppendEntriesRequest) Descriptor() ([]byte, []int) { 275 | return file_transport_proto_rawDescGZIP(), []int{2} 276 | } 277 | 278 | func (x *AppendEntriesRequest) GetRpcHeader() *RPCHeader { 279 | if x != nil { 280 | return x.RpcHeader 281 | } 282 | return nil 283 | } 284 | 285 | func (x *AppendEntriesRequest) GetTerm() uint64 { 286 | if x != nil { 287 | return x.Term 288 | } 289 | return 0 290 | } 291 | 292 | func (x *AppendEntriesRequest) GetLeader() []byte { 293 | if x != nil { 294 | return x.Leader 295 | } 296 | return nil 297 | } 298 | 299 | func (x *AppendEntriesRequest) GetPrevLogEntry() uint64 { 300 | if x != nil { 301 | return x.PrevLogEntry 302 | } 303 | return 0 304 | } 305 | 306 | func (x *AppendEntriesRequest) GetPrevLogTerm() uint64 { 307 | if x != nil { 308 | return x.PrevLogTerm 309 | } 310 | return 0 311 | } 312 | 313 | func (x *AppendEntriesRequest) GetEntries() []*Log { 314 | if x != nil { 315 | return x.Entries 316 | } 317 | return nil 318 | } 319 | 320 | func (x *AppendEntriesRequest) GetLeaderCommitIndex() uint64 { 321 | if x != nil { 322 | return x.LeaderCommitIndex 323 | } 324 | return 0 325 | } 326 | 327 | type AppendEntriesResponse struct { 328 | state protoimpl.MessageState 329 | sizeCache protoimpl.SizeCache 330 | unknownFields protoimpl.UnknownFields 331 | 332 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 333 | Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` 334 | LastLog uint64 `protobuf:"varint,3,opt,name=last_log,json=lastLog,proto3" json:"last_log,omitempty"` 335 | Success bool `protobuf:"varint,4,opt,name=success,proto3" json:"success,omitempty"` 336 | NoRetryBackoff bool `protobuf:"varint,5,opt,name=no_retry_backoff,json=noRetryBackoff,proto3" json:"no_retry_backoff,omitempty"` 337 | } 338 | 339 | func (x *AppendEntriesResponse) Reset() { 340 | *x = AppendEntriesResponse{} 341 | if protoimpl.UnsafeEnabled { 342 | mi := &file_transport_proto_msgTypes[3] 343 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 344 | ms.StoreMessageInfo(mi) 345 | } 346 | } 347 | 348 | func (x *AppendEntriesResponse) String() string { 349 | return protoimpl.X.MessageStringOf(x) 350 | } 351 | 352 | func (*AppendEntriesResponse) ProtoMessage() {} 353 | 354 | func (x *AppendEntriesResponse) ProtoReflect() protoreflect.Message { 355 | mi := &file_transport_proto_msgTypes[3] 356 | if protoimpl.UnsafeEnabled && x != nil { 357 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 358 | if ms.LoadMessageInfo() == nil { 359 | ms.StoreMessageInfo(mi) 360 | } 361 | return ms 362 | } 363 | return mi.MessageOf(x) 364 | } 365 | 366 | // Deprecated: Use AppendEntriesResponse.ProtoReflect.Descriptor instead. 367 | func (*AppendEntriesResponse) Descriptor() ([]byte, []int) { 368 | return file_transport_proto_rawDescGZIP(), []int{3} 369 | } 370 | 371 | func (x *AppendEntriesResponse) GetRpcHeader() *RPCHeader { 372 | if x != nil { 373 | return x.RpcHeader 374 | } 375 | return nil 376 | } 377 | 378 | func (x *AppendEntriesResponse) GetTerm() uint64 { 379 | if x != nil { 380 | return x.Term 381 | } 382 | return 0 383 | } 384 | 385 | func (x *AppendEntriesResponse) GetLastLog() uint64 { 386 | if x != nil { 387 | return x.LastLog 388 | } 389 | return 0 390 | } 391 | 392 | func (x *AppendEntriesResponse) GetSuccess() bool { 393 | if x != nil { 394 | return x.Success 395 | } 396 | return false 397 | } 398 | 399 | func (x *AppendEntriesResponse) GetNoRetryBackoff() bool { 400 | if x != nil { 401 | return x.NoRetryBackoff 402 | } 403 | return false 404 | } 405 | 406 | type RequestVoteRequest struct { 407 | state protoimpl.MessageState 408 | sizeCache protoimpl.SizeCache 409 | unknownFields protoimpl.UnknownFields 410 | 411 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 412 | Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` 413 | Candidate []byte `protobuf:"bytes,3,opt,name=candidate,proto3" json:"candidate,omitempty"` 414 | LastLogIndex uint64 `protobuf:"varint,4,opt,name=last_log_index,json=lastLogIndex,proto3" json:"last_log_index,omitempty"` 415 | LastLogTerm uint64 `protobuf:"varint,5,opt,name=last_log_term,json=lastLogTerm,proto3" json:"last_log_term,omitempty"` 416 | LeadershipTransfer bool `protobuf:"varint,6,opt,name=leadership_transfer,json=leadershipTransfer,proto3" json:"leadership_transfer,omitempty"` 417 | } 418 | 419 | func (x *RequestVoteRequest) Reset() { 420 | *x = RequestVoteRequest{} 421 | if protoimpl.UnsafeEnabled { 422 | mi := &file_transport_proto_msgTypes[4] 423 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 424 | ms.StoreMessageInfo(mi) 425 | } 426 | } 427 | 428 | func (x *RequestVoteRequest) String() string { 429 | return protoimpl.X.MessageStringOf(x) 430 | } 431 | 432 | func (*RequestVoteRequest) ProtoMessage() {} 433 | 434 | func (x *RequestVoteRequest) ProtoReflect() protoreflect.Message { 435 | mi := &file_transport_proto_msgTypes[4] 436 | if protoimpl.UnsafeEnabled && x != nil { 437 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 438 | if ms.LoadMessageInfo() == nil { 439 | ms.StoreMessageInfo(mi) 440 | } 441 | return ms 442 | } 443 | return mi.MessageOf(x) 444 | } 445 | 446 | // Deprecated: Use RequestVoteRequest.ProtoReflect.Descriptor instead. 447 | func (*RequestVoteRequest) Descriptor() ([]byte, []int) { 448 | return file_transport_proto_rawDescGZIP(), []int{4} 449 | } 450 | 451 | func (x *RequestVoteRequest) GetRpcHeader() *RPCHeader { 452 | if x != nil { 453 | return x.RpcHeader 454 | } 455 | return nil 456 | } 457 | 458 | func (x *RequestVoteRequest) GetTerm() uint64 { 459 | if x != nil { 460 | return x.Term 461 | } 462 | return 0 463 | } 464 | 465 | func (x *RequestVoteRequest) GetCandidate() []byte { 466 | if x != nil { 467 | return x.Candidate 468 | } 469 | return nil 470 | } 471 | 472 | func (x *RequestVoteRequest) GetLastLogIndex() uint64 { 473 | if x != nil { 474 | return x.LastLogIndex 475 | } 476 | return 0 477 | } 478 | 479 | func (x *RequestVoteRequest) GetLastLogTerm() uint64 { 480 | if x != nil { 481 | return x.LastLogTerm 482 | } 483 | return 0 484 | } 485 | 486 | func (x *RequestVoteRequest) GetLeadershipTransfer() bool { 487 | if x != nil { 488 | return x.LeadershipTransfer 489 | } 490 | return false 491 | } 492 | 493 | type RequestVoteResponse struct { 494 | state protoimpl.MessageState 495 | sizeCache protoimpl.SizeCache 496 | unknownFields protoimpl.UnknownFields 497 | 498 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 499 | Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` 500 | Peers []byte `protobuf:"bytes,3,opt,name=peers,proto3" json:"peers,omitempty"` 501 | Granted bool `protobuf:"varint,4,opt,name=granted,proto3" json:"granted,omitempty"` 502 | } 503 | 504 | func (x *RequestVoteResponse) Reset() { 505 | *x = RequestVoteResponse{} 506 | if protoimpl.UnsafeEnabled { 507 | mi := &file_transport_proto_msgTypes[5] 508 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 509 | ms.StoreMessageInfo(mi) 510 | } 511 | } 512 | 513 | func (x *RequestVoteResponse) String() string { 514 | return protoimpl.X.MessageStringOf(x) 515 | } 516 | 517 | func (*RequestVoteResponse) ProtoMessage() {} 518 | 519 | func (x *RequestVoteResponse) ProtoReflect() protoreflect.Message { 520 | mi := &file_transport_proto_msgTypes[5] 521 | if protoimpl.UnsafeEnabled && x != nil { 522 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 523 | if ms.LoadMessageInfo() == nil { 524 | ms.StoreMessageInfo(mi) 525 | } 526 | return ms 527 | } 528 | return mi.MessageOf(x) 529 | } 530 | 531 | // Deprecated: Use RequestVoteResponse.ProtoReflect.Descriptor instead. 532 | func (*RequestVoteResponse) Descriptor() ([]byte, []int) { 533 | return file_transport_proto_rawDescGZIP(), []int{5} 534 | } 535 | 536 | func (x *RequestVoteResponse) GetRpcHeader() *RPCHeader { 537 | if x != nil { 538 | return x.RpcHeader 539 | } 540 | return nil 541 | } 542 | 543 | func (x *RequestVoteResponse) GetTerm() uint64 { 544 | if x != nil { 545 | return x.Term 546 | } 547 | return 0 548 | } 549 | 550 | func (x *RequestVoteResponse) GetPeers() []byte { 551 | if x != nil { 552 | return x.Peers 553 | } 554 | return nil 555 | } 556 | 557 | func (x *RequestVoteResponse) GetGranted() bool { 558 | if x != nil { 559 | return x.Granted 560 | } 561 | return false 562 | } 563 | 564 | type TimeoutNowRequest struct { 565 | state protoimpl.MessageState 566 | sizeCache protoimpl.SizeCache 567 | unknownFields protoimpl.UnknownFields 568 | 569 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 570 | } 571 | 572 | func (x *TimeoutNowRequest) Reset() { 573 | *x = TimeoutNowRequest{} 574 | if protoimpl.UnsafeEnabled { 575 | mi := &file_transport_proto_msgTypes[6] 576 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 577 | ms.StoreMessageInfo(mi) 578 | } 579 | } 580 | 581 | func (x *TimeoutNowRequest) String() string { 582 | return protoimpl.X.MessageStringOf(x) 583 | } 584 | 585 | func (*TimeoutNowRequest) ProtoMessage() {} 586 | 587 | func (x *TimeoutNowRequest) ProtoReflect() protoreflect.Message { 588 | mi := &file_transport_proto_msgTypes[6] 589 | if protoimpl.UnsafeEnabled && x != nil { 590 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 591 | if ms.LoadMessageInfo() == nil { 592 | ms.StoreMessageInfo(mi) 593 | } 594 | return ms 595 | } 596 | return mi.MessageOf(x) 597 | } 598 | 599 | // Deprecated: Use TimeoutNowRequest.ProtoReflect.Descriptor instead. 600 | func (*TimeoutNowRequest) Descriptor() ([]byte, []int) { 601 | return file_transport_proto_rawDescGZIP(), []int{6} 602 | } 603 | 604 | func (x *TimeoutNowRequest) GetRpcHeader() *RPCHeader { 605 | if x != nil { 606 | return x.RpcHeader 607 | } 608 | return nil 609 | } 610 | 611 | type TimeoutNowResponse struct { 612 | state protoimpl.MessageState 613 | sizeCache protoimpl.SizeCache 614 | unknownFields protoimpl.UnknownFields 615 | 616 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 617 | } 618 | 619 | func (x *TimeoutNowResponse) Reset() { 620 | *x = TimeoutNowResponse{} 621 | if protoimpl.UnsafeEnabled { 622 | mi := &file_transport_proto_msgTypes[7] 623 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 624 | ms.StoreMessageInfo(mi) 625 | } 626 | } 627 | 628 | func (x *TimeoutNowResponse) String() string { 629 | return protoimpl.X.MessageStringOf(x) 630 | } 631 | 632 | func (*TimeoutNowResponse) ProtoMessage() {} 633 | 634 | func (x *TimeoutNowResponse) ProtoReflect() protoreflect.Message { 635 | mi := &file_transport_proto_msgTypes[7] 636 | if protoimpl.UnsafeEnabled && x != nil { 637 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 638 | if ms.LoadMessageInfo() == nil { 639 | ms.StoreMessageInfo(mi) 640 | } 641 | return ms 642 | } 643 | return mi.MessageOf(x) 644 | } 645 | 646 | // Deprecated: Use TimeoutNowResponse.ProtoReflect.Descriptor instead. 647 | func (*TimeoutNowResponse) Descriptor() ([]byte, []int) { 648 | return file_transport_proto_rawDescGZIP(), []int{7} 649 | } 650 | 651 | func (x *TimeoutNowResponse) GetRpcHeader() *RPCHeader { 652 | if x != nil { 653 | return x.RpcHeader 654 | } 655 | return nil 656 | } 657 | 658 | // The first InstallSnapshotRequest on the stream contains all the metadata. 659 | // All further messages contain only data. 660 | type InstallSnapshotRequest struct { 661 | state protoimpl.MessageState 662 | sizeCache protoimpl.SizeCache 663 | unknownFields protoimpl.UnknownFields 664 | 665 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 666 | SnapshotVersion int64 `protobuf:"varint,11,opt,name=snapshot_version,json=snapshotVersion,proto3" json:"snapshot_version,omitempty"` 667 | Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` 668 | Leader []byte `protobuf:"bytes,3,opt,name=leader,proto3" json:"leader,omitempty"` 669 | LastLogIndex uint64 `protobuf:"varint,4,opt,name=last_log_index,json=lastLogIndex,proto3" json:"last_log_index,omitempty"` 670 | LastLogTerm uint64 `protobuf:"varint,5,opt,name=last_log_term,json=lastLogTerm,proto3" json:"last_log_term,omitempty"` 671 | Peers []byte `protobuf:"bytes,6,opt,name=peers,proto3" json:"peers,omitempty"` 672 | Configuration []byte `protobuf:"bytes,7,opt,name=configuration,proto3" json:"configuration,omitempty"` 673 | ConfigurationIndex uint64 `protobuf:"varint,8,opt,name=configuration_index,json=configurationIndex,proto3" json:"configuration_index,omitempty"` 674 | Size int64 `protobuf:"varint,9,opt,name=size,proto3" json:"size,omitempty"` 675 | Data []byte `protobuf:"bytes,10,opt,name=data,proto3" json:"data,omitempty"` 676 | } 677 | 678 | func (x *InstallSnapshotRequest) Reset() { 679 | *x = InstallSnapshotRequest{} 680 | if protoimpl.UnsafeEnabled { 681 | mi := &file_transport_proto_msgTypes[8] 682 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 683 | ms.StoreMessageInfo(mi) 684 | } 685 | } 686 | 687 | func (x *InstallSnapshotRequest) String() string { 688 | return protoimpl.X.MessageStringOf(x) 689 | } 690 | 691 | func (*InstallSnapshotRequest) ProtoMessage() {} 692 | 693 | func (x *InstallSnapshotRequest) ProtoReflect() protoreflect.Message { 694 | mi := &file_transport_proto_msgTypes[8] 695 | if protoimpl.UnsafeEnabled && x != nil { 696 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 697 | if ms.LoadMessageInfo() == nil { 698 | ms.StoreMessageInfo(mi) 699 | } 700 | return ms 701 | } 702 | return mi.MessageOf(x) 703 | } 704 | 705 | // Deprecated: Use InstallSnapshotRequest.ProtoReflect.Descriptor instead. 706 | func (*InstallSnapshotRequest) Descriptor() ([]byte, []int) { 707 | return file_transport_proto_rawDescGZIP(), []int{8} 708 | } 709 | 710 | func (x *InstallSnapshotRequest) GetRpcHeader() *RPCHeader { 711 | if x != nil { 712 | return x.RpcHeader 713 | } 714 | return nil 715 | } 716 | 717 | func (x *InstallSnapshotRequest) GetSnapshotVersion() int64 { 718 | if x != nil { 719 | return x.SnapshotVersion 720 | } 721 | return 0 722 | } 723 | 724 | func (x *InstallSnapshotRequest) GetTerm() uint64 { 725 | if x != nil { 726 | return x.Term 727 | } 728 | return 0 729 | } 730 | 731 | func (x *InstallSnapshotRequest) GetLeader() []byte { 732 | if x != nil { 733 | return x.Leader 734 | } 735 | return nil 736 | } 737 | 738 | func (x *InstallSnapshotRequest) GetLastLogIndex() uint64 { 739 | if x != nil { 740 | return x.LastLogIndex 741 | } 742 | return 0 743 | } 744 | 745 | func (x *InstallSnapshotRequest) GetLastLogTerm() uint64 { 746 | if x != nil { 747 | return x.LastLogTerm 748 | } 749 | return 0 750 | } 751 | 752 | func (x *InstallSnapshotRequest) GetPeers() []byte { 753 | if x != nil { 754 | return x.Peers 755 | } 756 | return nil 757 | } 758 | 759 | func (x *InstallSnapshotRequest) GetConfiguration() []byte { 760 | if x != nil { 761 | return x.Configuration 762 | } 763 | return nil 764 | } 765 | 766 | func (x *InstallSnapshotRequest) GetConfigurationIndex() uint64 { 767 | if x != nil { 768 | return x.ConfigurationIndex 769 | } 770 | return 0 771 | } 772 | 773 | func (x *InstallSnapshotRequest) GetSize() int64 { 774 | if x != nil { 775 | return x.Size 776 | } 777 | return 0 778 | } 779 | 780 | func (x *InstallSnapshotRequest) GetData() []byte { 781 | if x != nil { 782 | return x.Data 783 | } 784 | return nil 785 | } 786 | 787 | type InstallSnapshotResponse struct { 788 | state protoimpl.MessageState 789 | sizeCache protoimpl.SizeCache 790 | unknownFields protoimpl.UnknownFields 791 | 792 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 793 | Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` 794 | Success bool `protobuf:"varint,3,opt,name=success,proto3" json:"success,omitempty"` 795 | } 796 | 797 | func (x *InstallSnapshotResponse) Reset() { 798 | *x = InstallSnapshotResponse{} 799 | if protoimpl.UnsafeEnabled { 800 | mi := &file_transport_proto_msgTypes[9] 801 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 802 | ms.StoreMessageInfo(mi) 803 | } 804 | } 805 | 806 | func (x *InstallSnapshotResponse) String() string { 807 | return protoimpl.X.MessageStringOf(x) 808 | } 809 | 810 | func (*InstallSnapshotResponse) ProtoMessage() {} 811 | 812 | func (x *InstallSnapshotResponse) ProtoReflect() protoreflect.Message { 813 | mi := &file_transport_proto_msgTypes[9] 814 | if protoimpl.UnsafeEnabled && x != nil { 815 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 816 | if ms.LoadMessageInfo() == nil { 817 | ms.StoreMessageInfo(mi) 818 | } 819 | return ms 820 | } 821 | return mi.MessageOf(x) 822 | } 823 | 824 | // Deprecated: Use InstallSnapshotResponse.ProtoReflect.Descriptor instead. 825 | func (*InstallSnapshotResponse) Descriptor() ([]byte, []int) { 826 | return file_transport_proto_rawDescGZIP(), []int{9} 827 | } 828 | 829 | func (x *InstallSnapshotResponse) GetRpcHeader() *RPCHeader { 830 | if x != nil { 831 | return x.RpcHeader 832 | } 833 | return nil 834 | } 835 | 836 | func (x *InstallSnapshotResponse) GetTerm() uint64 { 837 | if x != nil { 838 | return x.Term 839 | } 840 | return 0 841 | } 842 | 843 | func (x *InstallSnapshotResponse) GetSuccess() bool { 844 | if x != nil { 845 | return x.Success 846 | } 847 | return false 848 | } 849 | 850 | type RequestPreVoteRequest struct { 851 | state protoimpl.MessageState 852 | sizeCache protoimpl.SizeCache 853 | unknownFields protoimpl.UnknownFields 854 | 855 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 856 | Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` 857 | LastLogIndex uint64 `protobuf:"varint,3,opt,name=last_log_index,json=lastLogIndex,proto3" json:"last_log_index,omitempty"` 858 | LastLogTerm uint64 `protobuf:"varint,4,opt,name=last_log_term,json=lastLogTerm,proto3" json:"last_log_term,omitempty"` 859 | } 860 | 861 | func (x *RequestPreVoteRequest) Reset() { 862 | *x = RequestPreVoteRequest{} 863 | if protoimpl.UnsafeEnabled { 864 | mi := &file_transport_proto_msgTypes[10] 865 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 866 | ms.StoreMessageInfo(mi) 867 | } 868 | } 869 | 870 | func (x *RequestPreVoteRequest) String() string { 871 | return protoimpl.X.MessageStringOf(x) 872 | } 873 | 874 | func (*RequestPreVoteRequest) ProtoMessage() {} 875 | 876 | func (x *RequestPreVoteRequest) ProtoReflect() protoreflect.Message { 877 | mi := &file_transport_proto_msgTypes[10] 878 | if protoimpl.UnsafeEnabled && x != nil { 879 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 880 | if ms.LoadMessageInfo() == nil { 881 | ms.StoreMessageInfo(mi) 882 | } 883 | return ms 884 | } 885 | return mi.MessageOf(x) 886 | } 887 | 888 | // Deprecated: Use RequestPreVoteRequest.ProtoReflect.Descriptor instead. 889 | func (*RequestPreVoteRequest) Descriptor() ([]byte, []int) { 890 | return file_transport_proto_rawDescGZIP(), []int{10} 891 | } 892 | 893 | func (x *RequestPreVoteRequest) GetRpcHeader() *RPCHeader { 894 | if x != nil { 895 | return x.RpcHeader 896 | } 897 | return nil 898 | } 899 | 900 | func (x *RequestPreVoteRequest) GetTerm() uint64 { 901 | if x != nil { 902 | return x.Term 903 | } 904 | return 0 905 | } 906 | 907 | func (x *RequestPreVoteRequest) GetLastLogIndex() uint64 { 908 | if x != nil { 909 | return x.LastLogIndex 910 | } 911 | return 0 912 | } 913 | 914 | func (x *RequestPreVoteRequest) GetLastLogTerm() uint64 { 915 | if x != nil { 916 | return x.LastLogTerm 917 | } 918 | return 0 919 | } 920 | 921 | type RequestPreVoteResponse struct { 922 | state protoimpl.MessageState 923 | sizeCache protoimpl.SizeCache 924 | unknownFields protoimpl.UnknownFields 925 | 926 | RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` 927 | Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` 928 | Granted bool `protobuf:"varint,3,opt,name=granted,proto3" json:"granted,omitempty"` 929 | } 930 | 931 | func (x *RequestPreVoteResponse) Reset() { 932 | *x = RequestPreVoteResponse{} 933 | if protoimpl.UnsafeEnabled { 934 | mi := &file_transport_proto_msgTypes[11] 935 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 936 | ms.StoreMessageInfo(mi) 937 | } 938 | } 939 | 940 | func (x *RequestPreVoteResponse) String() string { 941 | return protoimpl.X.MessageStringOf(x) 942 | } 943 | 944 | func (*RequestPreVoteResponse) ProtoMessage() {} 945 | 946 | func (x *RequestPreVoteResponse) ProtoReflect() protoreflect.Message { 947 | mi := &file_transport_proto_msgTypes[11] 948 | if protoimpl.UnsafeEnabled && x != nil { 949 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 950 | if ms.LoadMessageInfo() == nil { 951 | ms.StoreMessageInfo(mi) 952 | } 953 | return ms 954 | } 955 | return mi.MessageOf(x) 956 | } 957 | 958 | // Deprecated: Use RequestPreVoteResponse.ProtoReflect.Descriptor instead. 959 | func (*RequestPreVoteResponse) Descriptor() ([]byte, []int) { 960 | return file_transport_proto_rawDescGZIP(), []int{11} 961 | } 962 | 963 | func (x *RequestPreVoteResponse) GetRpcHeader() *RPCHeader { 964 | if x != nil { 965 | return x.RpcHeader 966 | } 967 | return nil 968 | } 969 | 970 | func (x *RequestPreVoteResponse) GetTerm() uint64 { 971 | if x != nil { 972 | return x.Term 973 | } 974 | return 0 975 | } 976 | 977 | func (x *RequestPreVoteResponse) GetGranted() bool { 978 | if x != nil { 979 | return x.Granted 980 | } 981 | return false 982 | } 983 | 984 | var File_transport_proto protoreflect.FileDescriptor 985 | 986 | var file_transport_proto_rawDesc = []byte{ 987 | 0x0a, 0x0f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 988 | 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 989 | 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 990 | 0x74, 0x6f, 0x22, 0x5a, 0x0a, 0x09, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 991 | 0x29, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x5f, 0x76, 0x65, 0x72, 0x73, 992 | 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 993 | 0x63, 0x6f, 0x6c, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 994 | 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x61, 0x64, 995 | 0x64, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x61, 0x64, 0x64, 0x72, 0x22, 0xd2, 996 | 0x02, 0x0a, 0x03, 0x4c, 0x6f, 0x67, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 997 | 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x12, 0x0a, 0x04, 998 | 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 999 | 0x12, 0x20, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0c, 1000 | 0x2e, 0x4c, 0x6f, 0x67, 0x2e, 0x4c, 0x6f, 0x67, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 1001 | 0x70, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 1002 | 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1e, 0x0a, 0x0a, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 1003 | 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x65, 0x78, 0x74, 0x65, 1004 | 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3b, 0x0a, 0x0b, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 1005 | 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 1006 | 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 1007 | 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x65, 1008 | 0x64, 0x41, 0x74, 0x22, 0x8d, 0x01, 0x0a, 0x07, 0x4c, 0x6f, 0x67, 0x54, 0x79, 0x70, 0x65, 0x12, 1009 | 0x0f, 0x0a, 0x0b, 0x4c, 0x4f, 0x47, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x10, 0x00, 1010 | 0x12, 0x0c, 0x0a, 0x08, 0x4c, 0x4f, 0x47, 0x5f, 0x4e, 0x4f, 0x4f, 0x50, 0x10, 0x01, 0x12, 0x1b, 1011 | 0x0a, 0x17, 0x4c, 0x4f, 0x47, 0x5f, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x45, 0x45, 0x52, 0x5f, 0x44, 1012 | 0x45, 0x50, 0x52, 0x45, 0x43, 0x41, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x4c, 1013 | 0x4f, 0x47, 0x5f, 0x52, 0x45, 0x4d, 0x4f, 0x56, 0x45, 0x5f, 0x50, 0x45, 0x45, 0x52, 0x5f, 0x44, 1014 | 0x45, 0x50, 0x52, 0x45, 0x43, 0x41, 0x54, 0x45, 0x44, 0x10, 0x03, 0x12, 0x0f, 0x0a, 0x0b, 0x4c, 1015 | 0x4f, 0x47, 0x5f, 0x42, 0x41, 0x52, 0x52, 0x49, 0x45, 0x52, 0x10, 0x04, 0x12, 0x15, 0x0a, 0x11, 1016 | 0x4c, 0x4f, 0x47, 0x5f, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x55, 0x52, 0x41, 0x54, 0x49, 0x4f, 1017 | 0x4e, 0x10, 0x05, 0x22, 0x87, 0x02, 0x0a, 0x14, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 1018 | 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x0a, 1019 | 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 1020 | 0x32, 0x0a, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 1021 | 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 1022 | 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x16, 0x0a, 0x06, 0x6c, 1023 | 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x6c, 0x65, 0x61, 1024 | 0x64, 0x65, 0x72, 0x12, 0x24, 0x0a, 0x0e, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 1025 | 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x70, 0x72, 0x65, 1026 | 0x76, 0x4c, 0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x22, 0x0a, 0x0d, 0x70, 0x72, 0x65, 1027 | 0x76, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 1028 | 0x52, 0x0b, 0x70, 0x72, 0x65, 0x76, 0x4c, 0x6f, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x12, 0x1e, 0x0a, 1029 | 0x07, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x04, 1030 | 0x2e, 0x4c, 0x6f, 0x67, 0x52, 0x07, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x12, 0x2e, 0x0a, 1031 | 0x13, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x69, 1032 | 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x11, 0x6c, 0x65, 0x61, 0x64, 1033 | 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x22, 0xb5, 0x01, 1034 | 0x0a, 0x15, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 1035 | 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 1036 | 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x52, 0x50, 1037 | 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 1038 | 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 1039 | 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x19, 0x0a, 0x08, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6c, 1040 | 0x6f, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x6f, 1041 | 0x67, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 1042 | 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x6e, 1043 | 0x6f, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x18, 1044 | 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6e, 0x6f, 0x52, 0x65, 0x74, 0x72, 0x79, 0x42, 0x61, 1045 | 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x22, 0xec, 0x01, 0x0a, 0x12, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 1046 | 0x74, 0x56, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x0a, 1047 | 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 1048 | 0x32, 0x0a, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 1049 | 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 1050 | 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x1c, 0x0a, 0x09, 0x63, 1051 | 0x61, 0x6e, 0x64, 0x69, 0x64, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 1052 | 0x63, 0x61, 0x6e, 0x64, 0x69, 0x64, 0x61, 0x74, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 1053 | 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, 1054 | 0x04, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 1055 | 0x22, 0x0a, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x74, 0x65, 0x72, 0x6d, 1056 | 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x54, 1057 | 0x65, 0x72, 0x6d, 0x12, 0x2f, 0x0a, 0x13, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 1058 | 0x70, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 1059 | 0x52, 0x12, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x54, 0x72, 0x61, 0x6e, 1060 | 0x73, 0x66, 0x65, 0x72, 0x22, 0x84, 0x01, 0x0a, 0x13, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 1061 | 0x56, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x0a, 1062 | 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 1063 | 0x32, 0x0a, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 1064 | 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 1065 | 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x14, 0x0a, 0x05, 0x70, 1066 | 0x65, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x70, 0x65, 0x65, 0x72, 1067 | 0x73, 0x12, 0x18, 0x0a, 0x07, 0x67, 0x72, 0x61, 0x6e, 0x74, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 1068 | 0x28, 0x08, 0x52, 0x07, 0x67, 0x72, 0x61, 0x6e, 0x74, 0x65, 0x64, 0x22, 0x3e, 0x0a, 0x11, 0x54, 1069 | 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4e, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 1070 | 0x12, 0x29, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 1071 | 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 1072 | 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x3f, 0x0a, 0x12, 0x54, 1073 | 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4e, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 1074 | 0x65, 0x12, 0x29, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 1075 | 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 1076 | 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0xf9, 0x02, 0x0a, 1077 | 0x16, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 1078 | 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 1079 | 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x52, 0x50, 1080 | 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 1081 | 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x76, 1082 | 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x73, 0x6e, 1083 | 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 1084 | 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 1085 | 0x6d, 0x12, 0x16, 0x0a, 0x06, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 1086 | 0x0c, 0x52, 0x06, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x24, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 1087 | 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, 1088 | 0x04, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 1089 | 0x22, 0x0a, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x74, 0x65, 0x72, 0x6d, 1090 | 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x54, 1091 | 0x65, 0x72, 0x6d, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x06, 0x20, 0x01, 1092 | 0x28, 0x0c, 0x52, 0x05, 0x70, 0x65, 0x65, 0x72, 0x73, 0x12, 0x24, 0x0a, 0x0d, 0x63, 0x6f, 0x6e, 1093 | 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 1094 | 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 1095 | 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 1096 | 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x63, 0x6f, 1097 | 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x64, 0x65, 0x78, 1098 | 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 1099 | 0x73, 0x69, 0x7a, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0a, 0x20, 0x01, 1100 | 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x72, 0x0a, 0x17, 0x49, 0x6e, 0x73, 0x74, 1101 | 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 1102 | 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 1103 | 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 1104 | 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x12, 1105 | 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 1106 | 0x72, 0x6d, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 1107 | 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x22, 0xa0, 0x01, 0x0a, 1108 | 0x15, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x50, 0x72, 0x65, 0x56, 0x6f, 0x74, 0x65, 0x52, 1109 | 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 1110 | 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x52, 0x50, 0x43, 1111 | 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 1112 | 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 1113 | 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x24, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 1114 | 0x67, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6c, 1115 | 0x61, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x22, 0x0a, 0x0d, 0x6c, 1116 | 0x61, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x04, 0x20, 0x01, 1117 | 0x28, 0x04, 0x52, 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x22, 1118 | 0x71, 0x0a, 0x16, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x50, 0x72, 0x65, 0x56, 0x6f, 0x74, 1119 | 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x0a, 0x72, 0x70, 0x63, 1120 | 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 1121 | 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 1122 | 0x61, 0x64, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 1123 | 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x18, 0x0a, 0x07, 0x67, 0x72, 0x61, 0x6e, 1124 | 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x67, 0x72, 0x61, 0x6e, 0x74, 1125 | 0x65, 0x64, 0x32, 0xa3, 0x03, 0x0a, 0x0d, 0x52, 0x61, 0x66, 0x74, 0x54, 0x72, 0x61, 0x6e, 0x73, 1126 | 0x70, 0x6f, 0x72, 0x74, 0x12, 0x4c, 0x0a, 0x15, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 1127 | 0x74, 0x72, 0x69, 0x65, 0x73, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x15, 0x2e, 1128 | 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x71, 1129 | 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 1130 | 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 1131 | 0x30, 0x01, 0x12, 0x40, 0x0a, 0x0d, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 1132 | 0x69, 0x65, 0x73, 0x12, 0x15, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 1133 | 0x69, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x41, 0x70, 0x70, 1134 | 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 1135 | 0x73, 0x65, 0x22, 0x00, 0x12, 0x3a, 0x0a, 0x0b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 1136 | 0x6f, 0x74, 0x65, 0x12, 0x13, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x6f, 0x74, 1137 | 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 1138 | 0x73, 0x74, 0x56, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 1139 | 0x12, 0x37, 0x0a, 0x0a, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4e, 0x6f, 0x77, 0x12, 0x12, 1140 | 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4e, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 1141 | 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4e, 0x6f, 0x77, 0x52, 1142 | 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0f, 0x49, 0x6e, 0x73, 1143 | 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x17, 0x2e, 0x49, 1144 | 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 1145 | 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 1146 | 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 1147 | 0x00, 0x28, 0x01, 0x12, 0x43, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x50, 0x72, 1148 | 0x65, 0x56, 0x6f, 0x74, 0x65, 0x12, 0x16, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x50, 1149 | 0x72, 0x65, 0x56, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 1150 | 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x50, 0x72, 0x65, 0x56, 0x6f, 0x74, 0x65, 0x52, 0x65, 1151 | 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x2c, 0x5a, 0x2a, 0x67, 0x69, 0x74, 0x68, 1152 | 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4a, 0x69, 0x6c, 0x6c, 0x65, 0x2f, 0x72, 0x61, 0x66, 1153 | 0x74, 0x2d, 0x67, 0x72, 0x70, 0x63, 0x2d, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 1154 | 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, 1155 | } 1156 | 1157 | var ( 1158 | file_transport_proto_rawDescOnce sync.Once 1159 | file_transport_proto_rawDescData = file_transport_proto_rawDesc 1160 | ) 1161 | 1162 | func file_transport_proto_rawDescGZIP() []byte { 1163 | file_transport_proto_rawDescOnce.Do(func() { 1164 | file_transport_proto_rawDescData = protoimpl.X.CompressGZIP(file_transport_proto_rawDescData) 1165 | }) 1166 | return file_transport_proto_rawDescData 1167 | } 1168 | 1169 | var file_transport_proto_enumTypes = make([]protoimpl.EnumInfo, 1) 1170 | var file_transport_proto_msgTypes = make([]protoimpl.MessageInfo, 12) 1171 | var file_transport_proto_goTypes = []interface{}{ 1172 | (Log_LogType)(0), // 0: Log.LogType 1173 | (*RPCHeader)(nil), // 1: RPCHeader 1174 | (*Log)(nil), // 2: Log 1175 | (*AppendEntriesRequest)(nil), // 3: AppendEntriesRequest 1176 | (*AppendEntriesResponse)(nil), // 4: AppendEntriesResponse 1177 | (*RequestVoteRequest)(nil), // 5: RequestVoteRequest 1178 | (*RequestVoteResponse)(nil), // 6: RequestVoteResponse 1179 | (*TimeoutNowRequest)(nil), // 7: TimeoutNowRequest 1180 | (*TimeoutNowResponse)(nil), // 8: TimeoutNowResponse 1181 | (*InstallSnapshotRequest)(nil), // 9: InstallSnapshotRequest 1182 | (*InstallSnapshotResponse)(nil), // 10: InstallSnapshotResponse 1183 | (*RequestPreVoteRequest)(nil), // 11: RequestPreVoteRequest 1184 | (*RequestPreVoteResponse)(nil), // 12: RequestPreVoteResponse 1185 | (*timestamppb.Timestamp)(nil), // 13: google.protobuf.Timestamp 1186 | } 1187 | var file_transport_proto_depIdxs = []int32{ 1188 | 0, // 0: Log.type:type_name -> Log.LogType 1189 | 13, // 1: Log.appended_at:type_name -> google.protobuf.Timestamp 1190 | 1, // 2: AppendEntriesRequest.rpc_header:type_name -> RPCHeader 1191 | 2, // 3: AppendEntriesRequest.entries:type_name -> Log 1192 | 1, // 4: AppendEntriesResponse.rpc_header:type_name -> RPCHeader 1193 | 1, // 5: RequestVoteRequest.rpc_header:type_name -> RPCHeader 1194 | 1, // 6: RequestVoteResponse.rpc_header:type_name -> RPCHeader 1195 | 1, // 7: TimeoutNowRequest.rpc_header:type_name -> RPCHeader 1196 | 1, // 8: TimeoutNowResponse.rpc_header:type_name -> RPCHeader 1197 | 1, // 9: InstallSnapshotRequest.rpc_header:type_name -> RPCHeader 1198 | 1, // 10: InstallSnapshotResponse.rpc_header:type_name -> RPCHeader 1199 | 1, // 11: RequestPreVoteRequest.rpc_header:type_name -> RPCHeader 1200 | 1, // 12: RequestPreVoteResponse.rpc_header:type_name -> RPCHeader 1201 | 3, // 13: RaftTransport.AppendEntriesPipeline:input_type -> AppendEntriesRequest 1202 | 3, // 14: RaftTransport.AppendEntries:input_type -> AppendEntriesRequest 1203 | 5, // 15: RaftTransport.RequestVote:input_type -> RequestVoteRequest 1204 | 7, // 16: RaftTransport.TimeoutNow:input_type -> TimeoutNowRequest 1205 | 9, // 17: RaftTransport.InstallSnapshot:input_type -> InstallSnapshotRequest 1206 | 11, // 18: RaftTransport.RequestPreVote:input_type -> RequestPreVoteRequest 1207 | 4, // 19: RaftTransport.AppendEntriesPipeline:output_type -> AppendEntriesResponse 1208 | 4, // 20: RaftTransport.AppendEntries:output_type -> AppendEntriesResponse 1209 | 6, // 21: RaftTransport.RequestVote:output_type -> RequestVoteResponse 1210 | 8, // 22: RaftTransport.TimeoutNow:output_type -> TimeoutNowResponse 1211 | 10, // 23: RaftTransport.InstallSnapshot:output_type -> InstallSnapshotResponse 1212 | 12, // 24: RaftTransport.RequestPreVote:output_type -> RequestPreVoteResponse 1213 | 19, // [19:25] is the sub-list for method output_type 1214 | 13, // [13:19] is the sub-list for method input_type 1215 | 13, // [13:13] is the sub-list for extension type_name 1216 | 13, // [13:13] is the sub-list for extension extendee 1217 | 0, // [0:13] is the sub-list for field type_name 1218 | } 1219 | 1220 | func init() { file_transport_proto_init() } 1221 | func file_transport_proto_init() { 1222 | if File_transport_proto != nil { 1223 | return 1224 | } 1225 | if !protoimpl.UnsafeEnabled { 1226 | file_transport_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { 1227 | switch v := v.(*RPCHeader); i { 1228 | case 0: 1229 | return &v.state 1230 | case 1: 1231 | return &v.sizeCache 1232 | case 2: 1233 | return &v.unknownFields 1234 | default: 1235 | return nil 1236 | } 1237 | } 1238 | file_transport_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { 1239 | switch v := v.(*Log); i { 1240 | case 0: 1241 | return &v.state 1242 | case 1: 1243 | return &v.sizeCache 1244 | case 2: 1245 | return &v.unknownFields 1246 | default: 1247 | return nil 1248 | } 1249 | } 1250 | file_transport_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { 1251 | switch v := v.(*AppendEntriesRequest); i { 1252 | case 0: 1253 | return &v.state 1254 | case 1: 1255 | return &v.sizeCache 1256 | case 2: 1257 | return &v.unknownFields 1258 | default: 1259 | return nil 1260 | } 1261 | } 1262 | file_transport_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { 1263 | switch v := v.(*AppendEntriesResponse); i { 1264 | case 0: 1265 | return &v.state 1266 | case 1: 1267 | return &v.sizeCache 1268 | case 2: 1269 | return &v.unknownFields 1270 | default: 1271 | return nil 1272 | } 1273 | } 1274 | file_transport_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { 1275 | switch v := v.(*RequestVoteRequest); i { 1276 | case 0: 1277 | return &v.state 1278 | case 1: 1279 | return &v.sizeCache 1280 | case 2: 1281 | return &v.unknownFields 1282 | default: 1283 | return nil 1284 | } 1285 | } 1286 | file_transport_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { 1287 | switch v := v.(*RequestVoteResponse); i { 1288 | case 0: 1289 | return &v.state 1290 | case 1: 1291 | return &v.sizeCache 1292 | case 2: 1293 | return &v.unknownFields 1294 | default: 1295 | return nil 1296 | } 1297 | } 1298 | file_transport_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { 1299 | switch v := v.(*TimeoutNowRequest); i { 1300 | case 0: 1301 | return &v.state 1302 | case 1: 1303 | return &v.sizeCache 1304 | case 2: 1305 | return &v.unknownFields 1306 | default: 1307 | return nil 1308 | } 1309 | } 1310 | file_transport_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { 1311 | switch v := v.(*TimeoutNowResponse); i { 1312 | case 0: 1313 | return &v.state 1314 | case 1: 1315 | return &v.sizeCache 1316 | case 2: 1317 | return &v.unknownFields 1318 | default: 1319 | return nil 1320 | } 1321 | } 1322 | file_transport_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { 1323 | switch v := v.(*InstallSnapshotRequest); i { 1324 | case 0: 1325 | return &v.state 1326 | case 1: 1327 | return &v.sizeCache 1328 | case 2: 1329 | return &v.unknownFields 1330 | default: 1331 | return nil 1332 | } 1333 | } 1334 | file_transport_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { 1335 | switch v := v.(*InstallSnapshotResponse); i { 1336 | case 0: 1337 | return &v.state 1338 | case 1: 1339 | return &v.sizeCache 1340 | case 2: 1341 | return &v.unknownFields 1342 | default: 1343 | return nil 1344 | } 1345 | } 1346 | file_transport_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { 1347 | switch v := v.(*RequestPreVoteRequest); i { 1348 | case 0: 1349 | return &v.state 1350 | case 1: 1351 | return &v.sizeCache 1352 | case 2: 1353 | return &v.unknownFields 1354 | default: 1355 | return nil 1356 | } 1357 | } 1358 | file_transport_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { 1359 | switch v := v.(*RequestPreVoteResponse); i { 1360 | case 0: 1361 | return &v.state 1362 | case 1: 1363 | return &v.sizeCache 1364 | case 2: 1365 | return &v.unknownFields 1366 | default: 1367 | return nil 1368 | } 1369 | } 1370 | } 1371 | type x struct{} 1372 | out := protoimpl.TypeBuilder{ 1373 | File: protoimpl.DescBuilder{ 1374 | GoPackagePath: reflect.TypeOf(x{}).PkgPath(), 1375 | RawDescriptor: file_transport_proto_rawDesc, 1376 | NumEnums: 1, 1377 | NumMessages: 12, 1378 | NumExtensions: 0, 1379 | NumServices: 1, 1380 | }, 1381 | GoTypes: file_transport_proto_goTypes, 1382 | DependencyIndexes: file_transport_proto_depIdxs, 1383 | EnumInfos: file_transport_proto_enumTypes, 1384 | MessageInfos: file_transport_proto_msgTypes, 1385 | }.Build() 1386 | File_transport_proto = out.File 1387 | file_transport_proto_rawDesc = nil 1388 | file_transport_proto_goTypes = nil 1389 | file_transport_proto_depIdxs = nil 1390 | } 1391 | --------------------------------------------------------------------------------