From 5a628e73396b097f72c31bd52cf456a230ce97da Mon Sep 17 00:00:00 2001 From: Matt Keeler Date: Thu, 12 Jan 2023 11:54:01 -0500 Subject: [PATCH 1/2] Export the DeferError future and its Init/Respond methods This is needed to allow for other transports to live in other packages --- api.go | 20 +++++------ fsm.go | 14 ++++---- future.go | 30 ++++++++-------- future_test.go | 18 +++++----- inmem_transport.go | 6 ++-- net_transport.go | 4 +-- raft.go | 90 +++++++++++++++++++++++----------------------- raft_test.go | 8 ++--- replication.go | 10 +++--- snapshot.go | 6 ++-- 10 files changed, 103 insertions(+), 103 deletions(-) diff --git a/api.go b/api.go index eda33c3f0..f17abe4e0 100644 --- a/api.go +++ b/api.go @@ -747,7 +747,7 @@ func (r *Raft) ReloadableConfig() ReloadableConfig { // servers to the cluster. func (r *Raft) BootstrapCluster(configuration Configuration) Future { bootstrapReq := &bootstrapFuture{} - bootstrapReq.init() + bootstrapReq.Init() bootstrapReq.configuration = configuration select { case <-r.shutdownCh: @@ -807,7 +807,7 @@ func (r *Raft) ApplyLog(log Log, timeout time.Duration) ApplyFuture { Extensions: log.Extensions, }, } - logFuture.init() + logFuture.Init() select { case <-timer: @@ -833,7 +833,7 @@ func (r *Raft) Barrier(timeout time.Duration) Future { // Create a log future, no index or term yet logFuture := &logFuture{log: Log{Type: LogBarrier}} - logFuture.init() + logFuture.Init() select { case <-timer: @@ -851,7 +851,7 @@ func (r *Raft) Barrier(timeout time.Duration) Future { func (r *Raft) VerifyLeader() Future { metrics.IncrCounter([]string{"raft", "verify_leader"}, 1) verifyFuture := &verifyFuture{} - verifyFuture.init() + verifyFuture.Init() select { case <-r.shutdownCh: return errorFuture{ErrRaftShutdown} @@ -864,9 +864,9 @@ func (r *Raft) VerifyLeader() Future { // committed. The main loop can access this directly. func (r *Raft) GetConfiguration() ConfigurationFuture { configReq := &configurationsFuture{} - configReq.init() + configReq.Init() configReq.configurations = configurations{latest: r.getLatestConfiguration()} - configReq.respond(nil) + configReq.Respond(nil) return configReq } @@ -997,12 +997,12 @@ func (r *Raft) Shutdown() Future { // can be used to open the snapshot. func (r *Raft) Snapshot() SnapshotFuture { future := &userSnapshotFuture{} - future.init() + future.Init() select { case r.userSnapshotCh <- future: return future case <-r.shutdownCh: - future.respond(ErrRaftShutdown) + future.Respond(ErrRaftShutdown) return future } } @@ -1033,7 +1033,7 @@ func (r *Raft) Restore(meta *SnapshotMeta, reader io.Reader, timeout time.Durati meta: meta, reader: reader, } - restore.init() + restore.Init() select { case <-timer: return ErrEnqueueTimeout @@ -1055,7 +1055,7 @@ func (r *Raft) Restore(meta *SnapshotMeta, reader io.Reader, timeout time.Durati Type: LogNoop, }, } - noop.init() + noop.Init() select { case <-timer: return ErrEnqueueTimeout diff --git a/fsm.go b/fsm.go index 6d26a9a28..9a50ec55d 100644 --- a/fsm.go +++ b/fsm.go @@ -88,7 +88,7 @@ func (r *Raft) runFSM() { // Invoke the future if given if req.future != nil { req.future.response = resp - req.future.respond(nil) + req.future.Respond(nil) } }() @@ -171,7 +171,7 @@ func (r *Raft) runFSM() { if req.future != nil { req.future.response = resp - req.future.respond(nil) + req.future.Respond(nil) } } } @@ -180,7 +180,7 @@ func (r *Raft) runFSM() { // Open the snapshot meta, source, err := r.snapshots.Open(req.ID) if err != nil { - req.respond(fmt.Errorf("failed to open snapshot %v: %v", req.ID, err)) + req.Respond(fmt.Errorf("failed to open snapshot %v: %v", req.ID, err)) return } defer source.Close() @@ -194,20 +194,20 @@ func (r *Raft) runFSM() { // Attempt to restore if err := fsmRestoreAndMeasure(snapLogger, r.fsm, source, meta.Size); err != nil { - req.respond(fmt.Errorf("failed to restore snapshot %v: %v", req.ID, err)) + req.Respond(fmt.Errorf("failed to restore snapshot %v: %v", req.ID, err)) return } // Update the last index and term lastIndex = meta.Index lastTerm = meta.Term - req.respond(nil) + req.Respond(nil) } snapshot := func(req *reqSnapshotFuture) { // Is there something to snapshot? if lastIndex == 0 { - req.respond(ErrNothingNewToSnapshot) + req.Respond(ErrNothingNewToSnapshot) return } @@ -220,7 +220,7 @@ func (r *Raft) runFSM() { req.index = lastIndex req.term = lastTerm req.snapshot = snap - req.respond(err) + req.Respond(err) } saturation := newSaturationMetric([]string{"raft", "thread", "fsm", "saturation"}, 1*time.Second) diff --git a/future.go b/future.go index 1411ae219..92de9630b 100644 --- a/future.go +++ b/future.go @@ -84,20 +84,20 @@ func (e errorFuture) Index() uint64 { return 0 } -// deferError can be embedded to allow a future +// DeferError can be embedded to allow a future // to provide an error in the future. -type deferError struct { +type DeferError struct { err error errCh chan error responded bool ShutdownCh chan struct{} } -func (d *deferError) init() { +func (d *DeferError) Init() { d.errCh = make(chan error, 1) } -func (d *deferError) Error() error { +func (d *DeferError) Error() error { if d.err != nil { // Note that when we've received a nil error, this // won't trigger, but the channel is closed after @@ -115,7 +115,7 @@ func (d *deferError) Error() error { return d.err } -func (d *deferError) respond(err error) { +func (d *DeferError) Respond(err error) { if d.errCh == nil { return } @@ -138,7 +138,7 @@ type configurationChangeFuture struct { // bootstrapFuture is used to attempt a live bootstrap of the cluster. See the // Raft object's BootstrapCluster member function for more details. type bootstrapFuture struct { - deferError + DeferError // configuration is the proposed bootstrap configuration to apply. configuration Configuration @@ -147,7 +147,7 @@ type bootstrapFuture struct { // logFuture is used to apply a log entry and waits until // the log is considered committed. type logFuture struct { - deferError + DeferError log Log response interface{} dispatch time.Time @@ -179,7 +179,7 @@ func (s *shutdownFuture) Error() error { // userSnapshotFuture is used for waiting on a user-triggered snapshot to // complete. type userSnapshotFuture struct { - deferError + DeferError // opener is a function used to open the snapshot. This is filled in // once the future returns with no error. @@ -203,7 +203,7 @@ func (u *userSnapshotFuture) Open() (*SnapshotMeta, io.ReadCloser, error) { // userRestoreFuture is used for waiting on a user-triggered restore of an // external snapshot to complete. type userRestoreFuture struct { - deferError + DeferError // meta is the metadata that belongs with the snapshot. meta *SnapshotMeta @@ -215,7 +215,7 @@ type userRestoreFuture struct { // reqSnapshotFuture is used for requesting a snapshot start. // It is only used internally. type reqSnapshotFuture struct { - deferError + DeferError // snapshot details provided by the FSM runner before responding index uint64 @@ -226,14 +226,14 @@ type reqSnapshotFuture struct { // restoreFuture is used for requesting an FSM to perform a // snapshot restore. Used internally only. type restoreFuture struct { - deferError + DeferError ID string } // verifyFuture is used to verify the current node is still // the leader. This is to prevent a stale read. type verifyFuture struct { - deferError + DeferError notifyCh chan *verifyFuture quorumSize int votes int @@ -243,7 +243,7 @@ type verifyFuture struct { // leadershipTransferFuture is used to track the progress of a leadership // transfer internally. type leadershipTransferFuture struct { - deferError + DeferError ID *ServerID Address *ServerAddress @@ -252,7 +252,7 @@ type leadershipTransferFuture struct { // configurationsFuture is used to retrieve the current configurations. This is // used to allow safe access to this information outside of the main thread. type configurationsFuture struct { - deferError + DeferError configurations configurations } @@ -292,7 +292,7 @@ func (v *verifyFuture) vote(leader bool) { // appendFuture is used for waiting on a pipelined append // entries RPC. type appendFuture struct { - deferError + DeferError start time.Time args *AppendEntriesRequest resp *AppendEntriesResponse diff --git a/future_test.go b/future_test.go index 8bb958329..8d7ac8824 100644 --- a/future_test.go +++ b/future_test.go @@ -6,9 +6,9 @@ import ( ) func TestDeferFutureSuccess(t *testing.T) { - var f deferError - f.init() - f.respond(nil) + var f DeferError + f.Init() + f.Respond(nil) if err := f.Error(); err != nil { t.Fatalf("unexpected error result; got %#v want nil", err) } @@ -19,9 +19,9 @@ func TestDeferFutureSuccess(t *testing.T) { func TestDeferFutureError(t *testing.T) { want := errors.New("x") - var f deferError - f.init() - f.respond(want) + var f DeferError + f.Init() + f.Respond(want) if got := f.Error(); got != want { t.Fatalf("unexpected error result; got %#v want %#v", got, want) } @@ -33,9 +33,9 @@ func TestDeferFutureError(t *testing.T) { func TestDeferFutureConcurrent(t *testing.T) { // Food for the race detector. want := errors.New("x") - var f deferError - f.init() - go f.respond(want) + var f DeferError + f.Init() + go f.Respond(want) if got := f.Error(); got != want { t.Errorf("unexpected error result; got %#v want %#v", got, want) } diff --git a/inmem_transport.go b/inmem_transport.go index b5bdecc73..7d84c097e 100644 --- a/inmem_transport.go +++ b/inmem_transport.go @@ -268,7 +268,7 @@ func (i *inmemPipeline) decodeResponses() { case rpcResp := <-inp.respCh: // Copy the result back *inp.future.resp = *rpcResp.Response.(*AppendEntriesResponse) - inp.future.respond(rpcResp.Error) + inp.future.Respond(rpcResp.Error) select { case i.doneCh <- inp.future: @@ -277,7 +277,7 @@ func (i *inmemPipeline) decodeResponses() { } case <-timeoutCh: - inp.future.respond(fmt.Errorf("command timed out")) + inp.future.Respond(fmt.Errorf("command timed out")) select { case i.doneCh <- inp.future: case <-i.shutdownCh: @@ -300,7 +300,7 @@ func (i *inmemPipeline) AppendEntries(args *AppendEntriesRequest, resp *AppendEn args: args, resp: resp, } - future.init() + future.Init() // Handle a timeout var timeout <-chan time.Time diff --git a/net_transport.go b/net_transport.go index 2e98bd36d..c4730f27b 100644 --- a/net_transport.go +++ b/net_transport.go @@ -747,7 +747,7 @@ func (n *netPipeline) decodeResponses() { } _, err := decodeResponse(n.conn, future.resp) - future.respond(err) + future.Respond(err) select { case n.doneCh <- future: case <-n.shutdownCh: @@ -767,7 +767,7 @@ func (n *netPipeline) AppendEntries(args *AppendEntriesRequest, resp *AppendEntr args: args, resp: resp, } - future.init() + future.Init() // Add a send timeout if timeout := n.trans.timeout; timeout > 0 { diff --git a/raft.go b/raft.go index 5ad23a04b..9d7b787b4 100644 --- a/raft.go +++ b/raft.go @@ -116,7 +116,7 @@ func (r *Raft) requestConfigChange(req configurationChangeRequest, timeout time. future := &configurationChangeFuture{ req: req, } - future.init() + future.Init() select { case <-timer: return errorFuture{ErrEnqueueTimeout} @@ -169,36 +169,36 @@ func (r *Raft) runFollower() { case c := <-r.configurationChangeCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader - c.respond(ErrNotLeader) + c.Respond(ErrNotLeader) case a := <-r.applyCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader - a.respond(ErrNotLeader) + a.Respond(ErrNotLeader) case v := <-r.verifyCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader - v.respond(ErrNotLeader) + v.Respond(ErrNotLeader) case ur := <-r.userRestoreCh: r.mainThreadSaturation.working() // Reject any restores since we are not the leader - ur.respond(ErrNotLeader) + ur.Respond(ErrNotLeader) case l := <-r.leadershipTransferCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader - l.respond(ErrNotLeader) + l.Respond(ErrNotLeader) case c := <-r.configurationsCh: r.mainThreadSaturation.working() c.configurations = r.configurations.Clone() - c.respond(nil) + c.Respond(nil) case b := <-r.bootstrapCh: r.mainThreadSaturation.working() - b.respond(r.liveBootstrap(b.configuration)) + b.Respond(r.liveBootstrap(b.configuration)) case <-r.leaderNotifyCh: // Ignore since we are not the leader @@ -336,36 +336,36 @@ func (r *Raft) runCandidate() { case c := <-r.configurationChangeCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader - c.respond(ErrNotLeader) + c.Respond(ErrNotLeader) case a := <-r.applyCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader - a.respond(ErrNotLeader) + a.Respond(ErrNotLeader) case v := <-r.verifyCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader - v.respond(ErrNotLeader) + v.Respond(ErrNotLeader) case ur := <-r.userRestoreCh: r.mainThreadSaturation.working() // Reject any restores since we are not the leader - ur.respond(ErrNotLeader) + ur.Respond(ErrNotLeader) case l := <-r.leadershipTransferCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader - l.respond(ErrNotLeader) + l.Respond(ErrNotLeader) case c := <-r.configurationsCh: r.mainThreadSaturation.working() c.configurations = r.configurations.Clone() - c.respond(nil) + c.Respond(nil) case b := <-r.bootstrapCh: r.mainThreadSaturation.working() - b.respond(ErrCantBootstrap) + b.Respond(ErrCantBootstrap) case <-r.leaderNotifyCh: // Ignore since we are not the leader @@ -461,12 +461,12 @@ func (r *Raft) runLeader() { // Respond to all inflight operations for e := r.leaderState.inflight.Front(); e != nil; e = e.Next() { - e.Value.(*logFuture).respond(ErrLeadershipLost) + e.Value.(*logFuture).Respond(ErrLeadershipLost) } // Respond to any pending verify requests for future := range r.leaderState.notify { - future.respond(ErrLeadershipLost) + future.Respond(ErrLeadershipLost) } // Clear all the state @@ -544,7 +544,7 @@ func (r *Raft) startStopReplication() { commitment: r.leaderState.commitment, stopCh: make(chan uint64, 1), triggerCh: make(chan struct{}, 1), - triggerDeferErrorCh: make(chan *deferError, 1), + triggerDeferErrorCh: make(chan *DeferError, 1), currentTerm: r.getCurrentTerm(), nextIndex: lastIdx + 1, lastContact: time.Now(), @@ -637,7 +637,7 @@ func (r *Raft) leaderLoop() { r.mainThreadSaturation.working() if r.getLeadershipTransferInProgress() { r.logger.Debug(ErrLeadershipTransferInProgress.Error()) - future.respond(ErrLeadershipTransferInProgress) + future.Respond(ErrLeadershipTransferInProgress) continue } @@ -674,19 +674,19 @@ func (r *Raft) leaderLoop() { close(stopCh) err := fmt.Errorf("leadership transfer timeout") r.logger.Debug(err.Error()) - future.respond(err) + future.Respond(err) <-doneCh case <-leftLeaderLoop: close(stopCh) err := fmt.Errorf("lost leadership during transfer (expected)") r.logger.Debug(err.Error()) - future.respond(nil) + future.Respond(nil) <-doneCh case err := <-doneCh: if err != nil { r.logger.Debug(err.Error()) } - future.respond(err) + future.Respond(err) } }() @@ -791,7 +791,7 @@ func (r *Raft) leaderLoop() { for _, repl := range r.leaderState.replState { repl.cleanNotify(v) } - v.respond(ErrNotLeader) + v.Respond(ErrNotLeader) } else { // Quorum of members agree, we are still leader @@ -799,47 +799,47 @@ func (r *Raft) leaderLoop() { for _, repl := range r.leaderState.replState { repl.cleanNotify(v) } - v.respond(nil) + v.Respond(nil) } case future := <-r.userRestoreCh: r.mainThreadSaturation.working() if r.getLeadershipTransferInProgress() { r.logger.Debug(ErrLeadershipTransferInProgress.Error()) - future.respond(ErrLeadershipTransferInProgress) + future.Respond(ErrLeadershipTransferInProgress) continue } err := r.restoreUserSnapshot(future.meta, future.reader) - future.respond(err) + future.Respond(err) case future := <-r.configurationsCh: r.mainThreadSaturation.working() if r.getLeadershipTransferInProgress() { r.logger.Debug(ErrLeadershipTransferInProgress.Error()) - future.respond(ErrLeadershipTransferInProgress) + future.Respond(ErrLeadershipTransferInProgress) continue } future.configurations = r.configurations.Clone() - future.respond(nil) + future.Respond(nil) case future := <-r.configurationChangeChIfStable(): r.mainThreadSaturation.working() if r.getLeadershipTransferInProgress() { r.logger.Debug(ErrLeadershipTransferInProgress.Error()) - future.respond(ErrLeadershipTransferInProgress) + future.Respond(ErrLeadershipTransferInProgress) continue } r.appendConfigurationEntry(future) case b := <-r.bootstrapCh: r.mainThreadSaturation.working() - b.respond(ErrCantBootstrap) + b.Respond(ErrCantBootstrap) case newLog := <-r.applyCh: r.mainThreadSaturation.working() if r.getLeadershipTransferInProgress() { r.logger.Debug(ErrLeadershipTransferInProgress.Error()) - newLog.respond(ErrLeadershipTransferInProgress) + newLog.Respond(ErrLeadershipTransferInProgress) continue } // Group commit, gather all the ready commits @@ -858,7 +858,7 @@ func (r *Raft) leaderLoop() { if stepDown { // we're in the process of stepping down as leader, don't process anything new for i := range ready { - ready[i].respond(ErrNotLeader) + ready[i].Respond(ErrNotLeader) } } else { r.dispatchLogs(ready) @@ -902,7 +902,7 @@ func (r *Raft) verifyLeader(v *verifyFuture) { // Set the quorum size, hot-path for single node v.quorumSize = r.quorumSize() if v.quorumSize == 1 { - v.respond(nil) + v.Respond(nil) return } @@ -930,8 +930,8 @@ func (r *Raft) leadershipTransfer(id ServerID, address ServerAddress, repl *foll } for atomic.LoadUint64(&repl.nextIndex) <= r.getLastIndex() { - err := &deferError{} - err.init() + err := &DeferError{} + err.Init() repl.triggerDeferErrorCh <- err select { case err := <-err.errCh: @@ -1059,7 +1059,7 @@ func (r *Raft) restoreUserSnapshot(meta *SnapshotMeta, reader io.Reader) error { if e == nil { break } - e.Value.(*logFuture).respond(ErrAbortedByRestore) + e.Value.(*logFuture).Respond(ErrAbortedByRestore) r.leaderState.inflight.Remove(e) } @@ -1100,7 +1100,7 @@ func (r *Raft) restoreUserSnapshot(meta *SnapshotMeta, reader io.Reader) error { // bad state so we panic to take ourselves out. fsm := &restoreFuture{ID: sink.ID()} fsm.ShutdownCh = r.shutdownCh - fsm.init() + fsm.Init() select { case r.fsmMutateCh <- fsm: case <-r.shutdownCh: @@ -1129,7 +1129,7 @@ func (r *Raft) restoreUserSnapshot(meta *SnapshotMeta, reader io.Reader) error { func (r *Raft) appendConfigurationEntry(future *configurationChangeFuture) { configuration, err := nextConfiguration(r.configurations.latest, r.configurations.latestIndex, future.req) if err != nil { - future.respond(err) + future.Respond(err) return } @@ -1192,7 +1192,7 @@ func (r *Raft) dispatchLogs(applyLogs []*logFuture) { if err := r.logs.StoreLogs(logs); err != nil { r.logger.Error("failed to commit logs", "error", err) for _, applyLog := range applyLogs { - applyLog.respond(err) + applyLog.Respond(err) } r.setState(Follower) return @@ -1229,7 +1229,7 @@ func (r *Raft) processLogs(index uint64, futures map[uint64]*logFuture) { case <-r.shutdownCh: for _, cl := range batch { if cl.future != nil { - cl.future.respond(ErrRaftShutdown) + cl.future.Respond(ErrRaftShutdown) } } } @@ -1271,7 +1271,7 @@ func (r *Raft) processLogs(index uint64, futures map[uint64]*logFuture) { case futureOk: // Invoke the future if given. - future.respond(nil) + future.Respond(nil) } } @@ -1762,11 +1762,11 @@ func (r *Raft) installSnapshot(rpc RPC, req *InstallSnapshotRequest) { // Restore snapshot future := &restoreFuture{ID: sink.ID()} future.ShutdownCh = r.shutdownCh - future.init() + future.Init() select { case r.fsmMutateCh <- future: case <-r.shutdownCh: - future.respond(ErrRaftShutdown) + future.Respond(ErrRaftShutdown) return } @@ -1940,12 +1940,12 @@ func (r *Raft) pickServer() *Server { // mainloop. func (r *Raft) initiateLeadershipTransfer(id *ServerID, address *ServerAddress) LeadershipTransferFuture { future := &leadershipTransferFuture{ID: id, Address: address} - future.init() + future.Init() if id != nil && *id == r.localID { err := fmt.Errorf("cannot transfer leadership to itself") r.logger.Info(err.Error()) - future.respond(err) + future.Respond(err) return future } diff --git a/raft_test.go b/raft_test.go index 84234e7f1..6d3c71eed 100644 --- a/raft_test.go +++ b/raft_test.go @@ -888,7 +888,7 @@ func TestRaft_AddKnownPeer(t *testing.T) { followers := c.GetInState(Follower) configReq := &configurationsFuture{} - configReq.init() + configReq.Init() leader.configurationsCh <- configReq if err := configReq.Error(); err != nil { t.Fatalf("err: %v", err) @@ -902,7 +902,7 @@ func TestRaft_AddKnownPeer(t *testing.T) { t.Fatalf("AddVoter() err: %v", err) } configReq = &configurationsFuture{} - configReq.init() + configReq.Init() leader.configurationsCh <- configReq if err := configReq.Error(); err != nil { t.Fatalf("err: %v", err) @@ -925,7 +925,7 @@ func TestRaft_RemoveUnknownPeer(t *testing.T) { // Get the leader leader := c.Leader() configReq := &configurationsFuture{} - configReq.init() + configReq.Init() leader.configurationsCh <- configReq if err := configReq.Error(); err != nil { t.Fatalf("err: %v", err) @@ -941,7 +941,7 @@ func TestRaft_RemoveUnknownPeer(t *testing.T) { t.Fatalf("RemoveServer() err: %v", err) } configReq = &configurationsFuture{} - configReq.init() + configReq.Init() leader.configurationsCh <- configReq if err := configReq.Error(); err != nil { t.Fatalf("err: %v", err) diff --git a/replication.go b/replication.go index efe46e4e4..598f2890e 100644 --- a/replication.go +++ b/replication.go @@ -61,7 +61,7 @@ type followerReplication struct { // triggerDeferErrorCh is used to provide a backchannel. By sending a // deferErr, the sender can be notifed when the replication is done. - triggerDeferErrorCh chan *deferError + triggerDeferErrorCh chan *DeferError // lastContact is updated to the current time whenever any response is // received from the follower (successful or not). This is used to check @@ -151,9 +151,9 @@ RPC: lastLogIdx, _ := r.getLastLog() shouldStop = r.replicateTo(s, lastLogIdx) if !shouldStop { - deferErr.respond(nil) + deferErr.Respond(nil) } else { - deferErr.respond(fmt.Errorf("replication failed")) + deferErr.Respond(fmt.Errorf("replication failed")) } case <-s.triggerCh: lastLogIdx, _ := r.getLastLog() @@ -473,9 +473,9 @@ SEND: lastLogIdx, _ := r.getLastLog() shouldStop = r.pipelineSend(s, pipeline, &nextIndex, lastLogIdx) if !shouldStop { - deferErr.respond(nil) + deferErr.Respond(nil) } else { - deferErr.respond(fmt.Errorf("replication failed")) + deferErr.Respond(fmt.Errorf("replication failed")) } case <-s.triggerCh: lastLogIdx, _ := r.getLastLog() diff --git a/snapshot.go b/snapshot.go index d6b267963..3d7c00800 100644 --- a/snapshot.go +++ b/snapshot.go @@ -90,7 +90,7 @@ func (r *Raft) runSnapshots() { return r.snapshots.Open(id) } } - future.respond(err) + future.Respond(err) case <-r.shutdownCh: return @@ -124,7 +124,7 @@ func (r *Raft) takeSnapshot() (string, error) { // Create a request for the FSM to perform a snapshot. snapReq := &reqSnapshotFuture{} - snapReq.init() + snapReq.Init() // Wait for dispatch or shutdown. select { @@ -147,7 +147,7 @@ func (r *Raft) takeSnapshot() (string, error) { // it is owned by the main thread. configReq := &configurationsFuture{} configReq.ShutdownCh = r.shutdownCh - configReq.init() + configReq.Init() select { case r.configurationsCh <- configReq: case <-r.shutdownCh: From 4b9a6aabbb4e05bad3c095886e34b3a33e6efd1a Mon Sep 17 00:00:00 2001 From: Matt Keeler Date: Thu, 12 Jan 2023 12:42:55 -0500 Subject: [PATCH 2/2] Implement a gRPC transport --- Makefile | 15 + buf.gen.yaml | 14 + buf.yaml | 17 + go.mod | 4 +- go.sum | 145 ++- proto/transport/v1/common.gen.go | 44 + proto/transport/v1/conversion.go | 15 + proto/transport/v1/service.gen.go | 216 +++++ proto/transport/v1/service.pb.go | 1203 +++++++++++++++++++++++++ proto/transport/v1/service.proto | 204 +++++ proto/transport/v1/service_grpc.pb.go | 320 +++++++ proto/transport/v1/types.pb.go | 341 +++++++ proto/transport/v1/types.proto | 73 ++ transports/grpc/future.go | 27 + transports/grpc/pipeline.go | 101 +++ transports/grpc/server.go | 295 ++++++ transports/grpc/transport.go | 256 ++++++ 17 files changed, 3288 insertions(+), 2 deletions(-) create mode 100644 buf.gen.yaml create mode 100644 buf.yaml create mode 100644 proto/transport/v1/common.gen.go create mode 100644 proto/transport/v1/conversion.go create mode 100644 proto/transport/v1/service.gen.go create mode 100644 proto/transport/v1/service.pb.go create mode 100644 proto/transport/v1/service.proto create mode 100644 proto/transport/v1/service_grpc.pb.go create mode 100644 proto/transport/v1/types.pb.go create mode 100644 proto/transport/v1/types.proto create mode 100644 transports/grpc/future.go create mode 100644 transports/grpc/pipeline.go create mode 100644 transports/grpc/server.go create mode 100644 transports/grpc/transport.go diff --git a/Makefile b/Makefile index c988f0ab2..ea4d0effc 100644 --- a/Makefile +++ b/Makefile @@ -42,4 +42,19 @@ cov: INTEG_TESTS=yes gocov test github.com/hashicorp/raft | gocov-html > /tmp/coverage.html open /tmp/coverage.html +.PHONY: proto +proto: proto-tools proto-no-tools + +.PHONY: proto-no-tools +proto-no-tools: + buf generate + mog -source './proto/transport/v1/*.pb.go' + +.PHONY: proto-tools +proto-tools: + go install github.com/bufbuild/buf/cmd/buf@v1.11.0 + go install google.golang.org/protobuf/cmd/protoc-gen-go@$(shell grep google.golang.org/protobuf go.mod | awk '{print $$2}') + go install google.golang.org/grpc/cmd/protoc-gen-go-grpc@v1.2.0 + go install github.com/hashicorp/mog@v0.3.0 + .PHONY: test cov integ deps dep-linter lint diff --git a/buf.gen.yaml b/buf.gen.yaml new file mode 100644 index 000000000..9bb7ef827 --- /dev/null +++ b/buf.gen.yaml @@ -0,0 +1,14 @@ +version: v1 +managed: + enabled: true + go_package_prefix: + default: github.com/hashicorp/raft +plugins: + - name: go + out: . + opt: paths=source_relative + - name: go-grpc + out: . + opt: + - paths=source_relative + - require_unimplemented_servers=false diff --git a/buf.yaml b/buf.yaml new file mode 100644 index 000000000..f0f694d7a --- /dev/null +++ b/buf.yaml @@ -0,0 +1,17 @@ +version: v1 +lint: + use: + - DEFAULT + except: + # we want to enable our Go packages to have a pb prefix to make goimports more + # intelligently handle fixing up imports and hopefully getting it right. + - PACKAGE_DIRECTORY_MATCH + + # if we ever need a v2 we can have a second version with the .v2 version in the package name + - PACKAGE_VERSION_SUFFIX + + service_suffix: Service + allow_comment_ignores: true +breaking: + use: + - FILE diff --git a/go.mod b/go.mod index e088fa2ec..1e7beac1d 100644 --- a/go.mod +++ b/go.mod @@ -8,5 +8,7 @@ require ( github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878 github.com/hashicorp/go-hclog v0.9.1 github.com/hashicorp/go-msgpack v0.5.5 - github.com/stretchr/testify v1.3.0 + github.com/stretchr/testify v1.7.0 + google.golang.org/grpc v1.51.0 + google.golang.org/protobuf v1.28.1 ) diff --git a/go.sum b/go.sum index f087772cc..dc68dc89b 100644 --- a/go.sum +++ b/go.sum @@ -1,13 +1,58 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/DataDog/datadog-go v2.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= +github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878 h1:EFSB7Zo9Eg91v7MJPVsifUysc/wPdN+NOnVe6bWbdBM= github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878/go.mod h1:3AMJUQhVx52RsWOnlkpikZr01T/yAVN2gn0861vByNg= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag= github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= +github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20211001041855-01bcc9b48dfe/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.10.2-0.20220325020618-49ff273808a1/go.mod h1:KJwIaB5Mv44NWtYuAOFCVOjcI94vtpEz2JU/D2v6IjE= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.6 h1:BKbKCqvP6I+rmFHt06ZmyQtvB8xAkWdhFyr0ZUNZcxQ= +github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-hclog v0.9.1 h1:9PZfAcVEvez4yhLH2TBU64/h/z4xlFI80cWXRrxuKuM= github.com/hashicorp/go-hclog v0.9.1/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrjA0H7acj2lQ= @@ -28,12 +73,110 @@ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZb github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= +github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b h1:PxfKdU9lEEDYjdIzOtC4qFWgkU2rGHdKlKowJSMN9h0= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f h1:v4INt8xihDGvnrfjMDVXGxw9wrfxYyCjk0KbXjhR55s= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.4.0 h1:BrVqGRd7+k1DiOgtnFvAkoQEWQvBc25ouMJM6429SFg= +golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013 h1:+kGHl1aib/qcwaRi1CbqBZ1rk19r85MNUf8HaBghugY= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.51.0 h1:E1eGv1FTqoLIdnBCZufiSHgKjlqG6fKFf6pPWtMTh8U= +google.golang.org/grpc v1.51.0/go.mod h1:wgNDFcnuBGmxLKI/qn4T+m5BtEBYXJPvibbUPsAIPww= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w= +google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/proto/transport/v1/common.gen.go b/proto/transport/v1/common.gen.go new file mode 100644 index 000000000..11bc28738 --- /dev/null +++ b/proto/transport/v1/common.gen.go @@ -0,0 +1,44 @@ +// Code generated by mog. DO NOT EDIT. + +package transportv1 + +import "github.com/hashicorp/raft" + +func LogToStruct(s *Log, t *raft.Log) { + if s == nil { + return + } + t.Index = s.Index + t.Term = s.Term + t.Type = raft.LogType(s.Type) + t.Data = s.Data + t.Extensions = s.Extensions + t.AppendedAt = TimeFromProto(s.AppendedAt) +} +func LogFromStruct(t *raft.Log, s *Log) { + if s == nil { + return + } + s.Index = t.Index + s.Term = t.Term + s.Type = uint32(t.Type) + s.Data = t.Data + s.Extensions = t.Extensions + s.AppendedAt = TimeToProto(t.AppendedAt) +} +func RPCHeaderToStruct(s *RPCHeader, t *raft.RPCHeader) { + if s == nil { + return + } + t.ProtocolVersion = raft.ProtocolVersion(s.ProtocolVersion) + t.ID = s.Id + t.Addr = s.Addr +} +func RPCHeaderFromStruct(t *raft.RPCHeader, s *RPCHeader) { + if s == nil { + return + } + s.ProtocolVersion = int32(t.ProtocolVersion) + s.Id = t.ID + s.Addr = t.Addr +} diff --git a/proto/transport/v1/conversion.go b/proto/transport/v1/conversion.go new file mode 100644 index 000000000..b5493820e --- /dev/null +++ b/proto/transport/v1/conversion.go @@ -0,0 +1,15 @@ +package transportv1 + +import ( + "time" + + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +func TimeFromProto(t *timestamppb.Timestamp) time.Time { + return t.AsTime() +} + +func TimeToProto(t time.Time) *timestamppb.Timestamp { + return timestamppb.New(t) +} diff --git a/proto/transport/v1/service.gen.go b/proto/transport/v1/service.gen.go new file mode 100644 index 000000000..64a39cb5f --- /dev/null +++ b/proto/transport/v1/service.gen.go @@ -0,0 +1,216 @@ +// Code generated by mog. DO NOT EDIT. + +package transportv1 + +import "github.com/hashicorp/raft" + +func AppendEntriesRequestToStruct(s *AppendEntriesRequest, t *raft.AppendEntriesRequest) { + if s == nil { + return + } + if s.RpcHeader != nil { + RPCHeaderToStruct(s.RpcHeader, &t.RPCHeader) + } + t.Term = s.Term + t.PrevLogEntry = s.PreviousLogEntry + t.PrevLogTerm = s.PreviousLogTerm + { + t.Entries = make([]*raft.Log, len(s.Entries)) + for i := range s.Entries { + if s.Entries[i] != nil { + var x raft.Log + LogToStruct(s.Entries[i], &x) + t.Entries[i] = &x + } + } + } + t.LeaderCommitIndex = s.LeaderCommitIndex +} +func AppendEntriesRequestFromStruct(t *raft.AppendEntriesRequest, s *AppendEntriesRequest) { + if s == nil { + return + } + { + var x RPCHeader + RPCHeaderFromStruct(&t.RPCHeader, &x) + s.RpcHeader = &x + } + s.Term = t.Term + s.PreviousLogEntry = t.PrevLogEntry + s.PreviousLogTerm = t.PrevLogTerm + { + s.Entries = make([]*Log, len(t.Entries)) + for i := range t.Entries { + if t.Entries[i] != nil { + var x Log + LogFromStruct(t.Entries[i], &x) + s.Entries[i] = &x + } + } + } + s.LeaderCommitIndex = t.LeaderCommitIndex +} +func AppendEntriesResponseToStruct(s *AppendEntriesResponse, t *raft.AppendEntriesResponse) { + if s == nil { + return + } + if s.RpcHeader != nil { + RPCHeaderToStruct(s.RpcHeader, &t.RPCHeader) + } + t.Term = s.Term + t.LastLog = s.LastLog + t.Success = s.Success + t.NoRetryBackoff = s.NoRetryBackoff +} +func AppendEntriesResponseFromStruct(t *raft.AppendEntriesResponse, s *AppendEntriesResponse) { + if s == nil { + return + } + { + var x RPCHeader + RPCHeaderFromStruct(&t.RPCHeader, &x) + s.RpcHeader = &x + } + s.Term = t.Term + s.LastLog = t.LastLog + s.Success = t.Success + s.NoRetryBackoff = t.NoRetryBackoff +} +func InstallSnapshotMetadataToStruct(s *InstallSnapshotMetadata, t *raft.InstallSnapshotRequest) { + if s == nil { + return + } + if s.RpcHeader != nil { + RPCHeaderToStruct(s.RpcHeader, &t.RPCHeader) + } + t.SnapshotVersion = raft.SnapshotVersion(s.SnapshotVersion) + t.Term = s.Term + t.LastLogIndex = s.LastLogIndex + t.LastLogTerm = s.LastLogTerm + t.Configuration = s.Configuration + t.ConfigurationIndex = s.ConfigurationIndex + t.Size = s.Size +} +func InstallSnapshotMetadataFromStruct(t *raft.InstallSnapshotRequest, s *InstallSnapshotMetadata) { + if s == nil { + return + } + { + var x RPCHeader + RPCHeaderFromStruct(&t.RPCHeader, &x) + s.RpcHeader = &x + } + s.SnapshotVersion = int32(t.SnapshotVersion) + s.Term = t.Term + s.LastLogIndex = t.LastLogIndex + s.LastLogTerm = t.LastLogTerm + s.Configuration = t.Configuration + s.ConfigurationIndex = t.ConfigurationIndex + s.Size = t.Size +} +func InstallSnapshotResponseToStruct(s *InstallSnapshotResponse, t *raft.InstallSnapshotResponse) { + if s == nil { + return + } + if s.RpcHeader != nil { + RPCHeaderToStruct(s.RpcHeader, &t.RPCHeader) + } + t.Term = s.Term + t.Success = s.Success +} +func InstallSnapshotResponseFromStruct(t *raft.InstallSnapshotResponse, s *InstallSnapshotResponse) { + if s == nil { + return + } + { + var x RPCHeader + RPCHeaderFromStruct(&t.RPCHeader, &x) + s.RpcHeader = &x + } + s.Term = t.Term + s.Success = t.Success +} +func RequestVoteRequestToStruct(s *RequestVoteRequest, t *raft.RequestVoteRequest) { + if s == nil { + return + } + if s.RpcHeader != nil { + RPCHeaderToStruct(s.RpcHeader, &t.RPCHeader) + } + t.Term = s.Term + t.LastLogIndex = s.LastLogIndex + t.LastLogTerm = s.LastLogTerm + t.LeadershipTransfer = s.LeadershipTransfer +} +func RequestVoteRequestFromStruct(t *raft.RequestVoteRequest, s *RequestVoteRequest) { + if s == nil { + return + } + { + var x RPCHeader + RPCHeaderFromStruct(&t.RPCHeader, &x) + s.RpcHeader = &x + } + s.Term = t.Term + s.LastLogIndex = t.LastLogIndex + s.LastLogTerm = t.LastLogTerm + s.LeadershipTransfer = t.LeadershipTransfer +} +func RequestVoteResponseToStruct(s *RequestVoteResponse, t *raft.RequestVoteResponse) { + if s == nil { + return + } + if s.RpcHeader != nil { + RPCHeaderToStruct(s.RpcHeader, &t.RPCHeader) + } + t.Term = s.Term + t.Granted = s.Granted +} +func RequestVoteResponseFromStruct(t *raft.RequestVoteResponse, s *RequestVoteResponse) { + if s == nil { + return + } + { + var x RPCHeader + RPCHeaderFromStruct(&t.RPCHeader, &x) + s.RpcHeader = &x + } + s.Term = t.Term + s.Granted = t.Granted +} +func TimeoutNowRequestToStruct(s *TimeoutNowRequest, t *raft.TimeoutNowRequest) { + if s == nil { + return + } + if s.RpcHeader != nil { + RPCHeaderToStruct(s.RpcHeader, &t.RPCHeader) + } +} +func TimeoutNowRequestFromStruct(t *raft.TimeoutNowRequest, s *TimeoutNowRequest) { + if s == nil { + return + } + { + var x RPCHeader + RPCHeaderFromStruct(&t.RPCHeader, &x) + s.RpcHeader = &x + } +} +func TimeoutNowResponseToStruct(s *TimeoutNowResponse, t *raft.TimeoutNowResponse) { + if s == nil { + return + } + if s.RpcHeader != nil { + RPCHeaderToStruct(s.RpcHeader, &t.RPCHeader) + } +} +func TimeoutNowResponseFromStruct(t *raft.TimeoutNowResponse, s *TimeoutNowResponse) { + if s == nil { + return + } + { + var x RPCHeader + RPCHeaderFromStruct(&t.RPCHeader, &x) + s.RpcHeader = &x + } +} diff --git a/proto/transport/v1/service.pb.go b/proto/transport/v1/service.pb.go new file mode 100644 index 000000000..2742232aa --- /dev/null +++ b/proto/transport/v1/service.pb.go @@ -0,0 +1,1203 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.28.1 +// protoc (unknown) +// source: proto/transport/v1/service.proto + +package transportv1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// AppendEntriesRequest is the command used to append entries to the +// replicated log. +// mog annotation: +// +// target=github.com/hashicorp/raft.AppendEntriesRequest +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields,Leader +type AppendEntriesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // mog: target=RPCHeader + RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` + // Provide the current term and leader + Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` + // Provide the previous entries index for integrity checking + // mog: target=PrevLogEntry + PreviousLogEntry uint64 `protobuf:"varint,3,opt,name=previous_log_entry,json=previousLogEntry,proto3" json:"previous_log_entry,omitempty"` + // PreviousLogTerm is the previous logs term value for integrity checking + // mog: target=PrevLogTerm + PreviousLogTerm uint64 `protobuf:"varint,4,opt,name=previous_log_term,json=previousLogTerm,proto3" json:"previous_log_term,omitempty"` + // New entries to commit + Entries []*Log `protobuf:"bytes,5,rep,name=entries,proto3" json:"entries,omitempty"` + // Commit index on the leader + LeaderCommitIndex uint64 `protobuf:"varint,6,opt,name=leader_commit_index,json=leaderCommitIndex,proto3" json:"leader_commit_index,omitempty"` +} + +func (x *AppendEntriesRequest) Reset() { + *x = AppendEntriesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AppendEntriesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AppendEntriesRequest) ProtoMessage() {} + +func (x *AppendEntriesRequest) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AppendEntriesRequest.ProtoReflect.Descriptor instead. +func (*AppendEntriesRequest) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{0} +} + +func (x *AppendEntriesRequest) GetRpcHeader() *RPCHeader { + if x != nil { + return x.RpcHeader + } + return nil +} + +func (x *AppendEntriesRequest) GetTerm() uint64 { + if x != nil { + return x.Term + } + return 0 +} + +func (x *AppendEntriesRequest) GetPreviousLogEntry() uint64 { + if x != nil { + return x.PreviousLogEntry + } + return 0 +} + +func (x *AppendEntriesRequest) GetPreviousLogTerm() uint64 { + if x != nil { + return x.PreviousLogTerm + } + return 0 +} + +func (x *AppendEntriesRequest) GetEntries() []*Log { + if x != nil { + return x.Entries + } + return nil +} + +func (x *AppendEntriesRequest) GetLeaderCommitIndex() uint64 { + if x != nil { + return x.LeaderCommitIndex + } + return 0 +} + +// AppendEntriesResponse is the response returned from an +// AppendEntriesRequest. +// mog annotation: +// +// target=github.com/hashicorp/raft.AppendEntriesResponse +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +type AppendEntriesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // mog: target=RPCHeader + RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` + // Newer term if leader is out of date + Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` + // Last Log is a hint to help accelerate rebuilding slow nodes + LastLog uint64 `protobuf:"varint,3,opt,name=last_log,json=lastLog,proto3" json:"last_log,omitempty"` + // We may not succeed if we have a conflicting entry + Success bool `protobuf:"varint,4,opt,name=success,proto3" json:"success,omitempty"` + // There are scenarios where this request didn't succeed + // but there's no need to wait/back-off the next attempt. + NoRetryBackoff bool `protobuf:"varint,5,opt,name=no_retry_backoff,json=noRetryBackoff,proto3" json:"no_retry_backoff,omitempty"` +} + +func (x *AppendEntriesResponse) Reset() { + *x = AppendEntriesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AppendEntriesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AppendEntriesResponse) ProtoMessage() {} + +func (x *AppendEntriesResponse) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AppendEntriesResponse.ProtoReflect.Descriptor instead. +func (*AppendEntriesResponse) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{1} +} + +func (x *AppendEntriesResponse) GetRpcHeader() *RPCHeader { + if x != nil { + return x.RpcHeader + } + return nil +} + +func (x *AppendEntriesResponse) GetTerm() uint64 { + if x != nil { + return x.Term + } + return 0 +} + +func (x *AppendEntriesResponse) GetLastLog() uint64 { + if x != nil { + return x.LastLog + } + return 0 +} + +func (x *AppendEntriesResponse) GetSuccess() bool { + if x != nil { + return x.Success + } + return false +} + +func (x *AppendEntriesResponse) GetNoRetryBackoff() bool { + if x != nil { + return x.NoRetryBackoff + } + return false +} + +// RequestVoteRequest is the command used to request a vote. +// mog annotation: +// +// target=github.com/hashicorp/raft.RequestVoteRequest +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields,Candidate +type RequestVoteRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // mog: target=RPCHeader + RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` + // Provide the term + Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` + // The requesting servers last log index + LastLogIndex uint64 `protobuf:"varint,3,opt,name=last_log_index,json=lastLogIndex,proto3" json:"last_log_index,omitempty"` + // The requesting servers last log term + LastLogTerm uint64 `protobuf:"varint,4,opt,name=last_log_term,json=lastLogTerm,proto3" json:"last_log_term,omitempty"` + // Used to indicate to peers if this vote was triggered by a leadership + // transfer. It is required for leadership transfer to work, because servers + // wouldn't vote otherwise if they are aware of an existing leader. + LeadershipTransfer bool `protobuf:"varint,5,opt,name=leadership_transfer,json=leadershipTransfer,proto3" json:"leadership_transfer,omitempty"` +} + +func (x *RequestVoteRequest) Reset() { + *x = RequestVoteRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RequestVoteRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RequestVoteRequest) ProtoMessage() {} + +func (x *RequestVoteRequest) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RequestVoteRequest.ProtoReflect.Descriptor instead. +func (*RequestVoteRequest) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{2} +} + +func (x *RequestVoteRequest) GetRpcHeader() *RPCHeader { + if x != nil { + return x.RpcHeader + } + return nil +} + +func (x *RequestVoteRequest) GetTerm() uint64 { + if x != nil { + return x.Term + } + return 0 +} + +func (x *RequestVoteRequest) GetLastLogIndex() uint64 { + if x != nil { + return x.LastLogIndex + } + return 0 +} + +func (x *RequestVoteRequest) GetLastLogTerm() uint64 { + if x != nil { + return x.LastLogTerm + } + return 0 +} + +func (x *RequestVoteRequest) GetLeadershipTransfer() bool { + if x != nil { + return x.LeadershipTransfer + } + return false +} + +// RequestVoteResponse is the data returned from RequestVote RPC +// mog annotation: +// +// target=github.com/hashicorp/raft.RequestVoteResponse +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields,Peers +type RequestVoteResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // mog: target=RPCHeader + RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` + // Provide the term + Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` + // Was the vote granted + Granted bool `protobuf:"varint,3,opt,name=granted,proto3" json:"granted,omitempty"` +} + +func (x *RequestVoteResponse) Reset() { + *x = RequestVoteResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RequestVoteResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RequestVoteResponse) ProtoMessage() {} + +func (x *RequestVoteResponse) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RequestVoteResponse.ProtoReflect.Descriptor instead. +func (*RequestVoteResponse) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{3} +} + +func (x *RequestVoteResponse) GetRpcHeader() *RPCHeader { + if x != nil { + return x.RpcHeader + } + return nil +} + +func (x *RequestVoteResponse) GetTerm() uint64 { + if x != nil { + return x.Term + } + return 0 +} + +func (x *RequestVoteResponse) GetGranted() bool { + if x != nil { + return x.Granted + } + return false +} + +type InstallSnapshotRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Message: + // + // *InstallSnapshotRequest_Metadata + // *InstallSnapshotRequest_Chunk + Message isInstallSnapshotRequest_Message `protobuf_oneof:"message"` +} + +func (x *InstallSnapshotRequest) Reset() { + *x = InstallSnapshotRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InstallSnapshotRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InstallSnapshotRequest) ProtoMessage() {} + +func (x *InstallSnapshotRequest) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use InstallSnapshotRequest.ProtoReflect.Descriptor instead. +func (*InstallSnapshotRequest) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{4} +} + +func (m *InstallSnapshotRequest) GetMessage() isInstallSnapshotRequest_Message { + if m != nil { + return m.Message + } + return nil +} + +func (x *InstallSnapshotRequest) GetMetadata() *InstallSnapshotMetadata { + if x, ok := x.GetMessage().(*InstallSnapshotRequest_Metadata); ok { + return x.Metadata + } + return nil +} + +func (x *InstallSnapshotRequest) GetChunk() *InstallSnapshotChunk { + if x, ok := x.GetMessage().(*InstallSnapshotRequest_Chunk); ok { + return x.Chunk + } + return nil +} + +type isInstallSnapshotRequest_Message interface { + isInstallSnapshotRequest_Message() +} + +type InstallSnapshotRequest_Metadata struct { + Metadata *InstallSnapshotMetadata `protobuf:"bytes,1,opt,name=metadata,proto3,oneof"` +} + +type InstallSnapshotRequest_Chunk struct { + Chunk *InstallSnapshotChunk `protobuf:"bytes,2,opt,name=chunk,proto3,oneof"` +} + +func (*InstallSnapshotRequest_Metadata) isInstallSnapshotRequest_Message() {} + +func (*InstallSnapshotRequest_Chunk) isInstallSnapshotRequest_Message() {} + +// InstallSnapshotMetadata is the initial install snapshot request +// arguments. This will be the first message sent in the InstallSnapshot RPC. +// mog annotation: +// +// target=github.com/hashicorp/raft.InstallSnapshotRequest +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields,Leader,Peers +type InstallSnapshotMetadata struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // mog: target=RPCHeader + RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` + SnapshotVersion int32 `protobuf:"varint,2,opt,name=snapshot_version,json=snapshotVersion,proto3" json:"snapshot_version,omitempty"` + // Term is the current term of the leader + Term uint64 `protobuf:"varint,3,opt,name=term,proto3" json:"term,omitempty"` + // LastLogIndex is the last log index represented in the snapshot + LastLogIndex uint64 `protobuf:"varint,4,opt,name=last_log_index,json=lastLogIndex,proto3" json:"last_log_index,omitempty"` + // LastLogTerm is the last log term represented in the snapshot + LastLogTerm uint64 `protobuf:"varint,5,opt,name=last_log_term,json=lastLogTerm,proto3" json:"last_log_term,omitempty"` + // Configuration contains cluster membership information. + Configuration []byte `protobuf:"bytes,6,opt,name=configuration,proto3" json:"configuration,omitempty"` + // ConfigurationIndex is the log index where the Configuration data was + // originally written. + ConfigurationIndex uint64 `protobuf:"varint,7,opt,name=configuration_index,json=configurationIndex,proto3" json:"configuration_index,omitempty"` + // Size is the number of bytes of snapshot data to be sent. + Size int64 `protobuf:"varint,8,opt,name=size,proto3" json:"size,omitempty"` +} + +func (x *InstallSnapshotMetadata) Reset() { + *x = InstallSnapshotMetadata{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InstallSnapshotMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InstallSnapshotMetadata) ProtoMessage() {} + +func (x *InstallSnapshotMetadata) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use InstallSnapshotMetadata.ProtoReflect.Descriptor instead. +func (*InstallSnapshotMetadata) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{5} +} + +func (x *InstallSnapshotMetadata) GetRpcHeader() *RPCHeader { + if x != nil { + return x.RpcHeader + } + return nil +} + +func (x *InstallSnapshotMetadata) GetSnapshotVersion() int32 { + if x != nil { + return x.SnapshotVersion + } + return 0 +} + +func (x *InstallSnapshotMetadata) GetTerm() uint64 { + if x != nil { + return x.Term + } + return 0 +} + +func (x *InstallSnapshotMetadata) GetLastLogIndex() uint64 { + if x != nil { + return x.LastLogIndex + } + return 0 +} + +func (x *InstallSnapshotMetadata) GetLastLogTerm() uint64 { + if x != nil { + return x.LastLogTerm + } + return 0 +} + +func (x *InstallSnapshotMetadata) GetConfiguration() []byte { + if x != nil { + return x.Configuration + } + return nil +} + +func (x *InstallSnapshotMetadata) GetConfigurationIndex() uint64 { + if x != nil { + return x.ConfigurationIndex + } + return 0 +} + +func (x *InstallSnapshotMetadata) GetSize() int64 { + if x != nil { + return x.Size + } + return 0 +} + +type InstallSnapshotChunk struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SnapshotData []byte `protobuf:"bytes,1,opt,name=snapshot_data,json=snapshotData,proto3" json:"snapshot_data,omitempty"` +} + +func (x *InstallSnapshotChunk) Reset() { + *x = InstallSnapshotChunk{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InstallSnapshotChunk) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InstallSnapshotChunk) ProtoMessage() {} + +func (x *InstallSnapshotChunk) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use InstallSnapshotChunk.ProtoReflect.Descriptor instead. +func (*InstallSnapshotChunk) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{6} +} + +func (x *InstallSnapshotChunk) GetSnapshotData() []byte { + if x != nil { + return x.SnapshotData + } + return nil +} + +// InstallSnapshotResponse is the data returned from RequestVote RPC +// mog annotation: +// +// target=github.com/hashicorp/raft.InstallSnapshotResponse +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +type InstallSnapshotResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // mog: target=RPCHeader + RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` + // Provide the term + Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` + // Success represents whether the snapshot was installed successfully. + Success bool `protobuf:"varint,3,opt,name=success,proto3" json:"success,omitempty"` +} + +func (x *InstallSnapshotResponse) Reset() { + *x = InstallSnapshotResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *InstallSnapshotResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InstallSnapshotResponse) ProtoMessage() {} + +func (x *InstallSnapshotResponse) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use InstallSnapshotResponse.ProtoReflect.Descriptor instead. +func (*InstallSnapshotResponse) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{7} +} + +func (x *InstallSnapshotResponse) GetRpcHeader() *RPCHeader { + if x != nil { + return x.RpcHeader + } + return nil +} + +func (x *InstallSnapshotResponse) GetTerm() uint64 { + if x != nil { + return x.Term + } + return 0 +} + +func (x *InstallSnapshotResponse) GetSuccess() bool { + if x != nil { + return x.Success + } + return false +} + +// TimeoutNowRequest is the arguments for the TimeoutNow RPC request +// mog annotation: +// +// target=github.com/hashicorp/raft.TimeoutNowRequest +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +type TimeoutNowRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // mog: target=RPCHeader + RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` +} + +func (x *TimeoutNowRequest) Reset() { + *x = TimeoutNowRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimeoutNowRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimeoutNowRequest) ProtoMessage() {} + +func (x *TimeoutNowRequest) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimeoutNowRequest.ProtoReflect.Descriptor instead. +func (*TimeoutNowRequest) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{8} +} + +func (x *TimeoutNowRequest) GetRpcHeader() *RPCHeader { + if x != nil { + return x.RpcHeader + } + return nil +} + +// TimeoutNowResponse is the response message for a TimeoutNow RPC request +// mog annotation: +// +// target=github.com/hashicorp/raft.TimeoutNowResponse +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +type TimeoutNowResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // mog: target=RPCHeader + RpcHeader *RPCHeader `protobuf:"bytes,1,opt,name=rpc_header,json=rpcHeader,proto3" json:"rpc_header,omitempty"` +} + +func (x *TimeoutNowResponse) Reset() { + *x = TimeoutNowResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_service_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimeoutNowResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimeoutNowResponse) ProtoMessage() {} + +func (x *TimeoutNowResponse) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_service_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimeoutNowResponse.ProtoReflect.Descriptor instead. +func (*TimeoutNowResponse) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_service_proto_rawDescGZIP(), []int{9} +} + +func (x *TimeoutNowResponse) GetRpcHeader() *RPCHeader { + if x != nil { + return x.RpcHeader + } + return nil +} + +var File_proto_transport_v1_service_proto protoreflect.FileDescriptor + +var file_proto_transport_v1_service_proto_rawDesc = []byte{ + 0x0a, 0x20, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, + 0x74, 0x2f, 0x76, 0x31, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x12, 0x1b, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, + 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x1a, + 0x1e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, + 0x2f, 0x76, 0x31, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, + 0xb7, 0x02, 0x0a, 0x14, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x45, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, + 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x68, + 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, + 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, + 0x65, 0x72, 0x6d, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, + 0x6c, 0x6f, 0x67, 0x5f, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x10, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x4c, 0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x2a, 0x0a, 0x11, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, 0x6c, 0x6f, + 0x67, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x70, 0x72, + 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x4c, 0x6f, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x12, 0x3a, 0x0a, + 0x07, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, + 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, + 0x52, 0x07, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x6c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x11, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x43, 0x6f, + 0x6d, 0x6d, 0x69, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x22, 0xd1, 0x01, 0x0a, 0x15, 0x41, 0x70, + 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x45, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, + 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, + 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, + 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, + 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x19, + 0x0a, 0x08, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x07, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, + 0x63, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x6e, 0x6f, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, + 0x62, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6e, + 0x6f, 0x52, 0x65, 0x74, 0x72, 0x79, 0x42, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x22, 0xea, 0x01, + 0x0a, 0x12, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x45, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, + 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, + 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, + 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, + 0x24, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x6f, 0x67, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x22, 0x0a, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6c, 0x6f, + 0x67, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x6c, 0x61, + 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x12, 0x2f, 0x0a, 0x13, 0x6c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, + 0x69, 0x70, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x22, 0x8a, 0x01, 0x0a, 0x13, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x45, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, + 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, + 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, + 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, + 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x18, 0x0a, + 0x07, 0x67, 0x72, 0x61, 0x6e, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, + 0x67, 0x72, 0x61, 0x6e, 0x74, 0x65, 0x64, 0x22, 0xc2, 0x01, 0x0a, 0x16, 0x49, 0x6e, 0x73, 0x74, + 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x52, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, + 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, + 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, + 0x6f, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x08, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x49, 0x0a, 0x05, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, + 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, + 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 0x74, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x48, 0x00, 0x52, 0x05, 0x63, 0x68, 0x75, 0x6e, + 0x6b, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xd4, 0x02, 0x0a, + 0x17, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x45, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, + 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x68, + 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, + 0x29, 0x0a, 0x10, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x76, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x73, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, + 0x72, 0x6d, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x24, + 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x12, 0x22, 0x0a, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x67, + 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x6c, 0x61, 0x73, + 0x74, 0x4c, 0x6f, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x12, 0x24, 0x0a, 0x0d, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x0d, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2f, + 0x0a, 0x13, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x73, + 0x69, 0x7a, 0x65, 0x22, 0x3b, 0x0a, 0x14, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, + 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x12, 0x23, 0x0a, 0x0d, 0x73, + 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x44, 0x61, 0x74, 0x61, + 0x22, 0x8e, 0x01, 0x0a, 0x17, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, + 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x45, 0x0a, 0x0a, + 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x26, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, + 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, + 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x22, 0x5a, 0x0a, 0x11, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4e, 0x6f, 0x77, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x45, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x68, 0x61, 0x73, + 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x52, 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x5b, 0x0a, + 0x12, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4e, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x45, 0x0a, 0x0a, 0x72, 0x70, 0x63, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, + 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, + 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, + 0x09, 0x72, 0x70, 0x63, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x32, 0xf7, 0x04, 0x0a, 0x14, 0x52, + 0x61, 0x66, 0x74, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x12, 0x76, 0x0a, 0x0d, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, + 0x72, 0x69, 0x65, 0x73, 0x12, 0x31, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, + 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, + 0x76, 0x31, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, + 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, + 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, + 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x82, 0x01, 0x0a, 0x15, + 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x50, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x31, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, + 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, + 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, + 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, + 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x45, 0x6e, 0x74, + 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x30, 0x01, + 0x12, 0x70, 0x0a, 0x0b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x6f, 0x74, 0x65, 0x12, + 0x2f, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, + 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x30, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, + 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x80, 0x01, 0x0a, 0x0f, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, + 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x33, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, + 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, + 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x53, 0x6e, 0x61, 0x70, + 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x34, 0x2e, 0x68, 0x61, + 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6c, + 0x6c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x28, 0x01, 0x30, 0x01, 0x12, 0x6d, 0x0a, 0x0a, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x4e, 0x6f, 0x77, 0x12, 0x2e, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, + 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, + 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4e, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, + 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, + 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4e, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x42, 0xf8, 0x01, 0x0a, 0x1f, 0x63, 0x6f, 0x6d, 0x2e, 0x68, 0x61, 0x73, + 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x38, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, + 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x72, + 0x61, 0x66, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, + 0x6f, 0x72, 0x74, 0x2f, 0x76, 0x31, 0x3b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, + 0x76, 0x31, 0xa2, 0x02, 0x03, 0x48, 0x52, 0x54, 0xaa, 0x02, 0x1b, 0x48, 0x61, 0x73, 0x68, 0x69, + 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, + 0x6f, 0x72, 0x74, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x1b, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, + 0x72, 0x70, 0x5c, 0x52, 0x61, 0x66, 0x74, 0x5c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, + 0x74, 0x5c, 0x56, 0x31, 0xe2, 0x02, 0x27, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, + 0x5c, 0x52, 0x61, 0x66, 0x74, 0x5c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x5c, + 0x56, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, + 0x1e, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x3a, 0x3a, 0x52, 0x61, 0x66, 0x74, + 0x3a, 0x3a, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x3a, 0x3a, 0x56, 0x31, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_proto_transport_v1_service_proto_rawDescOnce sync.Once + file_proto_transport_v1_service_proto_rawDescData = file_proto_transport_v1_service_proto_rawDesc +) + +func file_proto_transport_v1_service_proto_rawDescGZIP() []byte { + file_proto_transport_v1_service_proto_rawDescOnce.Do(func() { + file_proto_transport_v1_service_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_transport_v1_service_proto_rawDescData) + }) + return file_proto_transport_v1_service_proto_rawDescData +} + +var file_proto_transport_v1_service_proto_msgTypes = make([]protoimpl.MessageInfo, 10) +var file_proto_transport_v1_service_proto_goTypes = []interface{}{ + (*AppendEntriesRequest)(nil), // 0: hashicorp.raft.transport.v1.AppendEntriesRequest + (*AppendEntriesResponse)(nil), // 1: hashicorp.raft.transport.v1.AppendEntriesResponse + (*RequestVoteRequest)(nil), // 2: hashicorp.raft.transport.v1.RequestVoteRequest + (*RequestVoteResponse)(nil), // 3: hashicorp.raft.transport.v1.RequestVoteResponse + (*InstallSnapshotRequest)(nil), // 4: hashicorp.raft.transport.v1.InstallSnapshotRequest + (*InstallSnapshotMetadata)(nil), // 5: hashicorp.raft.transport.v1.InstallSnapshotMetadata + (*InstallSnapshotChunk)(nil), // 6: hashicorp.raft.transport.v1.InstallSnapshotChunk + (*InstallSnapshotResponse)(nil), // 7: hashicorp.raft.transport.v1.InstallSnapshotResponse + (*TimeoutNowRequest)(nil), // 8: hashicorp.raft.transport.v1.TimeoutNowRequest + (*TimeoutNowResponse)(nil), // 9: hashicorp.raft.transport.v1.TimeoutNowResponse + (*RPCHeader)(nil), // 10: hashicorp.raft.transport.v1.RPCHeader + (*Log)(nil), // 11: hashicorp.raft.transport.v1.Log +} +var file_proto_transport_v1_service_proto_depIdxs = []int32{ + 10, // 0: hashicorp.raft.transport.v1.AppendEntriesRequest.rpc_header:type_name -> hashicorp.raft.transport.v1.RPCHeader + 11, // 1: hashicorp.raft.transport.v1.AppendEntriesRequest.entries:type_name -> hashicorp.raft.transport.v1.Log + 10, // 2: hashicorp.raft.transport.v1.AppendEntriesResponse.rpc_header:type_name -> hashicorp.raft.transport.v1.RPCHeader + 10, // 3: hashicorp.raft.transport.v1.RequestVoteRequest.rpc_header:type_name -> hashicorp.raft.transport.v1.RPCHeader + 10, // 4: hashicorp.raft.transport.v1.RequestVoteResponse.rpc_header:type_name -> hashicorp.raft.transport.v1.RPCHeader + 5, // 5: hashicorp.raft.transport.v1.InstallSnapshotRequest.metadata:type_name -> hashicorp.raft.transport.v1.InstallSnapshotMetadata + 6, // 6: hashicorp.raft.transport.v1.InstallSnapshotRequest.chunk:type_name -> hashicorp.raft.transport.v1.InstallSnapshotChunk + 10, // 7: hashicorp.raft.transport.v1.InstallSnapshotMetadata.rpc_header:type_name -> hashicorp.raft.transport.v1.RPCHeader + 10, // 8: hashicorp.raft.transport.v1.InstallSnapshotResponse.rpc_header:type_name -> hashicorp.raft.transport.v1.RPCHeader + 10, // 9: hashicorp.raft.transport.v1.TimeoutNowRequest.rpc_header:type_name -> hashicorp.raft.transport.v1.RPCHeader + 10, // 10: hashicorp.raft.transport.v1.TimeoutNowResponse.rpc_header:type_name -> hashicorp.raft.transport.v1.RPCHeader + 0, // 11: hashicorp.raft.transport.v1.RaftTransportService.AppendEntries:input_type -> hashicorp.raft.transport.v1.AppendEntriesRequest + 0, // 12: hashicorp.raft.transport.v1.RaftTransportService.AppendEntriesPipeline:input_type -> hashicorp.raft.transport.v1.AppendEntriesRequest + 2, // 13: hashicorp.raft.transport.v1.RaftTransportService.RequestVote:input_type -> hashicorp.raft.transport.v1.RequestVoteRequest + 4, // 14: hashicorp.raft.transport.v1.RaftTransportService.InstallSnapshot:input_type -> hashicorp.raft.transport.v1.InstallSnapshotRequest + 8, // 15: hashicorp.raft.transport.v1.RaftTransportService.TimeoutNow:input_type -> hashicorp.raft.transport.v1.TimeoutNowRequest + 1, // 16: hashicorp.raft.transport.v1.RaftTransportService.AppendEntries:output_type -> hashicorp.raft.transport.v1.AppendEntriesResponse + 1, // 17: hashicorp.raft.transport.v1.RaftTransportService.AppendEntriesPipeline:output_type -> hashicorp.raft.transport.v1.AppendEntriesResponse + 3, // 18: hashicorp.raft.transport.v1.RaftTransportService.RequestVote:output_type -> hashicorp.raft.transport.v1.RequestVoteResponse + 7, // 19: hashicorp.raft.transport.v1.RaftTransportService.InstallSnapshot:output_type -> hashicorp.raft.transport.v1.InstallSnapshotResponse + 9, // 20: hashicorp.raft.transport.v1.RaftTransportService.TimeoutNow:output_type -> hashicorp.raft.transport.v1.TimeoutNowResponse + 16, // [16:21] is the sub-list for method output_type + 11, // [11:16] is the sub-list for method input_type + 11, // [11:11] is the sub-list for extension type_name + 11, // [11:11] is the sub-list for extension extendee + 0, // [0:11] is the sub-list for field type_name +} + +func init() { file_proto_transport_v1_service_proto_init() } +func file_proto_transport_v1_service_proto_init() { + if File_proto_transport_v1_service_proto != nil { + return + } + file_proto_transport_v1_types_proto_init() + if !protoimpl.UnsafeEnabled { + file_proto_transport_v1_service_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AppendEntriesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_service_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AppendEntriesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_service_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RequestVoteRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_service_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RequestVoteResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_service_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InstallSnapshotRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_service_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InstallSnapshotMetadata); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_service_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InstallSnapshotChunk); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_service_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*InstallSnapshotResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_service_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TimeoutNowRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_service_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TimeoutNowResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_proto_transport_v1_service_proto_msgTypes[4].OneofWrappers = []interface{}{ + (*InstallSnapshotRequest_Metadata)(nil), + (*InstallSnapshotRequest_Chunk)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_proto_transport_v1_service_proto_rawDesc, + NumEnums: 0, + NumMessages: 10, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_proto_transport_v1_service_proto_goTypes, + DependencyIndexes: file_proto_transport_v1_service_proto_depIdxs, + MessageInfos: file_proto_transport_v1_service_proto_msgTypes, + }.Build() + File_proto_transport_v1_service_proto = out.File + file_proto_transport_v1_service_proto_rawDesc = nil + file_proto_transport_v1_service_proto_goTypes = nil + file_proto_transport_v1_service_proto_depIdxs = nil +} diff --git a/proto/transport/v1/service.proto b/proto/transport/v1/service.proto new file mode 100644 index 000000000..d0e846464 --- /dev/null +++ b/proto/transport/v1/service.proto @@ -0,0 +1,204 @@ +syntax = "proto3"; + +package hashicorp.raft.transport.v1; + +import "proto/transport/v1/types.proto"; + +service RaftTransportService { + // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE + rpc AppendEntries(AppendEntriesRequest) returns (AppendEntriesResponse); + // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE + // buf:lint:ignore RPC_REQUEST_STANDARD_NAME + // buf:lint:ignore RPC_RESPONSE_STANDARD_NAME + rpc AppendEntriesPipeline(stream AppendEntriesRequest) returns (stream AppendEntriesResponse); + rpc RequestVote(RequestVoteRequest) returns (RequestVoteResponse); + rpc InstallSnapshot(stream InstallSnapshotRequest) returns (stream InstallSnapshotResponse); + rpc TimeoutNow(TimeoutNowRequest) returns (TimeoutNowResponse); +} + +// AppendEntriesRequest is the command used to append entries to the +// replicated log. +// mog annotation: +// +// target=github.com/hashicorp/raft.AppendEntriesRequest +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields,Leader +message AppendEntriesRequest { + // mog: target=RPCHeader + RPCHeader rpc_header = 1; + + // Provide the current term and leader + uint64 term = 2; + + // Provide the previous entries index for integrity checking + // mog: target=PrevLogEntry + uint64 previous_log_entry = 3; + + // PreviousLogTerm is the previous logs term value for integrity checking + // mog: target=PrevLogTerm + uint64 previous_log_term = 4; + + // New entries to commit + repeated Log entries = 5; + + // Commit index on the leader + uint64 leader_commit_index = 6; +} + +// AppendEntriesResponse is the response returned from an +// AppendEntriesRequest. +// mog annotation: +// +// target=github.com/hashicorp/raft.AppendEntriesResponse +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +message AppendEntriesResponse { + // mog: target=RPCHeader + RPCHeader rpc_header = 1; + + // Newer term if leader is out of date + uint64 term = 2; + + // Last Log is a hint to help accelerate rebuilding slow nodes + uint64 last_log = 3; + + // We may not succeed if we have a conflicting entry + bool success = 4; + + // There are scenarios where this request didn't succeed + // but there's no need to wait/back-off the next attempt. + bool no_retry_backoff = 5; +} + +// RequestVoteRequest is the command used to request a vote. +// mog annotation: +// +// target=github.com/hashicorp/raft.RequestVoteRequest +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields,Candidate +message RequestVoteRequest { + // mog: target=RPCHeader + RPCHeader rpc_header = 1; + + // Provide the term + uint64 term = 2; + + // The requesting servers last log index + uint64 last_log_index = 3; + + // The requesting servers last log term + uint64 last_log_term = 4; + + // Used to indicate to peers if this vote was triggered by a leadership + // transfer. It is required for leadership transfer to work, because servers + // wouldn't vote otherwise if they are aware of an existing leader. + bool leadership_transfer = 5; +} + +// RequestVoteResponse is the data returned from RequestVote RPC +// mog annotation: +// +// target=github.com/hashicorp/raft.RequestVoteResponse +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields,Peers +message RequestVoteResponse { + // mog: target=RPCHeader + RPCHeader rpc_header = 1; + + // Provide the term + uint64 term = 2; + + // Was the vote granted + bool granted = 3; +} + +message InstallSnapshotRequest { + oneof message { + InstallSnapshotMetadata metadata = 1; + InstallSnapshotChunk chunk = 2; + } +} + +// InstallSnapshotMetadata is the initial install snapshot request +// arguments. This will be the first message sent in the InstallSnapshot RPC. +// mog annotation: +// +// target=github.com/hashicorp/raft.InstallSnapshotRequest +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields,Leader,Peers +message InstallSnapshotMetadata { + // mog: target=RPCHeader + RPCHeader rpc_header = 1; + + int32 snapshot_version = 2; + + // Term is the current term of the leader + uint64 term = 3; + + // LastLogIndex is the last log index represented in the snapshot + uint64 last_log_index = 4; + + // LastLogTerm is the last log term represented in the snapshot + uint64 last_log_term = 5; + + // Configuration contains cluster membership information. + bytes configuration = 6; + + // ConfigurationIndex is the log index where the Configuration data was + // originally written. + uint64 configuration_index = 7; + + // Size is the number of bytes of snapshot data to be sent. + int64 size = 8; +} + +message InstallSnapshotChunk { + bytes snapshot_data = 1; +} + +// InstallSnapshotResponse is the data returned from RequestVote RPC +// mog annotation: +// +// target=github.com/hashicorp/raft.InstallSnapshotResponse +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +message InstallSnapshotResponse { + // mog: target=RPCHeader + RPCHeader rpc_header = 1; + + // Provide the term + uint64 term = 2; + + // Success represents whether the snapshot was installed successfully. + bool success = 3; +} + +// TimeoutNowRequest is the arguments for the TimeoutNow RPC request +// mog annotation: +// +// target=github.com/hashicorp/raft.TimeoutNowRequest +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +message TimeoutNowRequest { + // mog: target=RPCHeader + RPCHeader rpc_header = 1; +} + +// TimeoutNowResponse is the response message for a TimeoutNow RPC request +// mog annotation: +// +// target=github.com/hashicorp/raft.TimeoutNowResponse +// output=service.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +message TimeoutNowResponse { + // mog: target=RPCHeader + RPCHeader rpc_header = 1; +} diff --git a/proto/transport/v1/service_grpc.pb.go b/proto/transport/v1/service_grpc.pb.go new file mode 100644 index 000000000..24f61762a --- /dev/null +++ b/proto/transport/v1/service_grpc.pb.go @@ -0,0 +1,320 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.2.0 +// - protoc (unknown) +// source: proto/transport/v1/service.proto + +package transportv1 + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// RaftTransportServiceClient is the client API for RaftTransportService service. +// +// 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. +type RaftTransportServiceClient interface { + // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE + AppendEntries(ctx context.Context, in *AppendEntriesRequest, opts ...grpc.CallOption) (*AppendEntriesResponse, error) + // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE + // buf:lint:ignore RPC_REQUEST_STANDARD_NAME + // buf:lint:ignore RPC_RESPONSE_STANDARD_NAME + AppendEntriesPipeline(ctx context.Context, opts ...grpc.CallOption) (RaftTransportService_AppendEntriesPipelineClient, error) + RequestVote(ctx context.Context, in *RequestVoteRequest, opts ...grpc.CallOption) (*RequestVoteResponse, error) + InstallSnapshot(ctx context.Context, opts ...grpc.CallOption) (RaftTransportService_InstallSnapshotClient, error) + TimeoutNow(ctx context.Context, in *TimeoutNowRequest, opts ...grpc.CallOption) (*TimeoutNowResponse, error) +} + +type raftTransportServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewRaftTransportServiceClient(cc grpc.ClientConnInterface) RaftTransportServiceClient { + return &raftTransportServiceClient{cc} +} + +func (c *raftTransportServiceClient) AppendEntries(ctx context.Context, in *AppendEntriesRequest, opts ...grpc.CallOption) (*AppendEntriesResponse, error) { + out := new(AppendEntriesResponse) + err := c.cc.Invoke(ctx, "/hashicorp.raft.transport.v1.RaftTransportService/AppendEntries", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *raftTransportServiceClient) AppendEntriesPipeline(ctx context.Context, opts ...grpc.CallOption) (RaftTransportService_AppendEntriesPipelineClient, error) { + stream, err := c.cc.NewStream(ctx, &RaftTransportService_ServiceDesc.Streams[0], "/hashicorp.raft.transport.v1.RaftTransportService/AppendEntriesPipeline", opts...) + if err != nil { + return nil, err + } + x := &raftTransportServiceAppendEntriesPipelineClient{stream} + return x, nil +} + +type RaftTransportService_AppendEntriesPipelineClient interface { + Send(*AppendEntriesRequest) error + Recv() (*AppendEntriesResponse, error) + grpc.ClientStream +} + +type raftTransportServiceAppendEntriesPipelineClient struct { + grpc.ClientStream +} + +func (x *raftTransportServiceAppendEntriesPipelineClient) Send(m *AppendEntriesRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *raftTransportServiceAppendEntriesPipelineClient) Recv() (*AppendEntriesResponse, error) { + m := new(AppendEntriesResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *raftTransportServiceClient) RequestVote(ctx context.Context, in *RequestVoteRequest, opts ...grpc.CallOption) (*RequestVoteResponse, error) { + out := new(RequestVoteResponse) + err := c.cc.Invoke(ctx, "/hashicorp.raft.transport.v1.RaftTransportService/RequestVote", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *raftTransportServiceClient) InstallSnapshot(ctx context.Context, opts ...grpc.CallOption) (RaftTransportService_InstallSnapshotClient, error) { + stream, err := c.cc.NewStream(ctx, &RaftTransportService_ServiceDesc.Streams[1], "/hashicorp.raft.transport.v1.RaftTransportService/InstallSnapshot", opts...) + if err != nil { + return nil, err + } + x := &raftTransportServiceInstallSnapshotClient{stream} + return x, nil +} + +type RaftTransportService_InstallSnapshotClient interface { + Send(*InstallSnapshotRequest) error + Recv() (*InstallSnapshotResponse, error) + grpc.ClientStream +} + +type raftTransportServiceInstallSnapshotClient struct { + grpc.ClientStream +} + +func (x *raftTransportServiceInstallSnapshotClient) Send(m *InstallSnapshotRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *raftTransportServiceInstallSnapshotClient) Recv() (*InstallSnapshotResponse, error) { + m := new(InstallSnapshotResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *raftTransportServiceClient) TimeoutNow(ctx context.Context, in *TimeoutNowRequest, opts ...grpc.CallOption) (*TimeoutNowResponse, error) { + out := new(TimeoutNowResponse) + err := c.cc.Invoke(ctx, "/hashicorp.raft.transport.v1.RaftTransportService/TimeoutNow", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// RaftTransportServiceServer is the server API for RaftTransportService service. +// All implementations should embed UnimplementedRaftTransportServiceServer +// for forward compatibility +type RaftTransportServiceServer interface { + // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE + AppendEntries(context.Context, *AppendEntriesRequest) (*AppendEntriesResponse, error) + // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE + // buf:lint:ignore RPC_REQUEST_STANDARD_NAME + // buf:lint:ignore RPC_RESPONSE_STANDARD_NAME + AppendEntriesPipeline(RaftTransportService_AppendEntriesPipelineServer) error + RequestVote(context.Context, *RequestVoteRequest) (*RequestVoteResponse, error) + InstallSnapshot(RaftTransportService_InstallSnapshotServer) error + TimeoutNow(context.Context, *TimeoutNowRequest) (*TimeoutNowResponse, error) +} + +// UnimplementedRaftTransportServiceServer should be embedded to have forward compatible implementations. +type UnimplementedRaftTransportServiceServer struct { +} + +func (UnimplementedRaftTransportServiceServer) AppendEntries(context.Context, *AppendEntriesRequest) (*AppendEntriesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AppendEntries not implemented") +} +func (UnimplementedRaftTransportServiceServer) AppendEntriesPipeline(RaftTransportService_AppendEntriesPipelineServer) error { + return status.Errorf(codes.Unimplemented, "method AppendEntriesPipeline not implemented") +} +func (UnimplementedRaftTransportServiceServer) RequestVote(context.Context, *RequestVoteRequest) (*RequestVoteResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RequestVote not implemented") +} +func (UnimplementedRaftTransportServiceServer) InstallSnapshot(RaftTransportService_InstallSnapshotServer) error { + return status.Errorf(codes.Unimplemented, "method InstallSnapshot not implemented") +} +func (UnimplementedRaftTransportServiceServer) TimeoutNow(context.Context, *TimeoutNowRequest) (*TimeoutNowResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method TimeoutNow not implemented") +} + +// UnsafeRaftTransportServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to RaftTransportServiceServer will +// result in compilation errors. +type UnsafeRaftTransportServiceServer interface { + mustEmbedUnimplementedRaftTransportServiceServer() +} + +func RegisterRaftTransportServiceServer(s grpc.ServiceRegistrar, srv RaftTransportServiceServer) { + s.RegisterService(&RaftTransportService_ServiceDesc, srv) +} + +func _RaftTransportService_AppendEntries_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AppendEntriesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RaftTransportServiceServer).AppendEntries(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/hashicorp.raft.transport.v1.RaftTransportService/AppendEntries", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RaftTransportServiceServer).AppendEntries(ctx, req.(*AppendEntriesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RaftTransportService_AppendEntriesPipeline_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(RaftTransportServiceServer).AppendEntriesPipeline(&raftTransportServiceAppendEntriesPipelineServer{stream}) +} + +type RaftTransportService_AppendEntriesPipelineServer interface { + Send(*AppendEntriesResponse) error + Recv() (*AppendEntriesRequest, error) + grpc.ServerStream +} + +type raftTransportServiceAppendEntriesPipelineServer struct { + grpc.ServerStream +} + +func (x *raftTransportServiceAppendEntriesPipelineServer) Send(m *AppendEntriesResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *raftTransportServiceAppendEntriesPipelineServer) Recv() (*AppendEntriesRequest, error) { + m := new(AppendEntriesRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _RaftTransportService_RequestVote_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestVoteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RaftTransportServiceServer).RequestVote(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/hashicorp.raft.transport.v1.RaftTransportService/RequestVote", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RaftTransportServiceServer).RequestVote(ctx, req.(*RequestVoteRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RaftTransportService_InstallSnapshot_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(RaftTransportServiceServer).InstallSnapshot(&raftTransportServiceInstallSnapshotServer{stream}) +} + +type RaftTransportService_InstallSnapshotServer interface { + Send(*InstallSnapshotResponse) error + Recv() (*InstallSnapshotRequest, error) + grpc.ServerStream +} + +type raftTransportServiceInstallSnapshotServer struct { + grpc.ServerStream +} + +func (x *raftTransportServiceInstallSnapshotServer) Send(m *InstallSnapshotResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *raftTransportServiceInstallSnapshotServer) Recv() (*InstallSnapshotRequest, error) { + m := new(InstallSnapshotRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _RaftTransportService_TimeoutNow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(TimeoutNowRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RaftTransportServiceServer).TimeoutNow(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/hashicorp.raft.transport.v1.RaftTransportService/TimeoutNow", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RaftTransportServiceServer).TimeoutNow(ctx, req.(*TimeoutNowRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// RaftTransportService_ServiceDesc is the grpc.ServiceDesc for RaftTransportService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var RaftTransportService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "hashicorp.raft.transport.v1.RaftTransportService", + HandlerType: (*RaftTransportServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "AppendEntries", + Handler: _RaftTransportService_AppendEntries_Handler, + }, + { + MethodName: "RequestVote", + Handler: _RaftTransportService_RequestVote_Handler, + }, + { + MethodName: "TimeoutNow", + Handler: _RaftTransportService_TimeoutNow_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "AppendEntriesPipeline", + Handler: _RaftTransportService_AppendEntriesPipeline_Handler, + ServerStreams: true, + ClientStreams: true, + }, + { + StreamName: "InstallSnapshot", + Handler: _RaftTransportService_InstallSnapshot_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "proto/transport/v1/service.proto", +} diff --git a/proto/transport/v1/types.pb.go b/proto/transport/v1/types.pb.go new file mode 100644 index 000000000..e2fc25276 --- /dev/null +++ b/proto/transport/v1/types.pb.go @@ -0,0 +1,341 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.28.1 +// protoc (unknown) +// source: proto/transport/v1/types.proto + +package transportv1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// mog annotation: +// +// target=github.com/hashicorp/raft.RPCHeader +// output=common.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +type RPCHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // ProtocolVerison is the version of the protocol the sender is speaking + ProtocolVersion int32 `protobuf:"varint,1,opt,name=protocol_version,json=protocolVersion,proto3" json:"protocol_version,omitempty"` + // ID is the ServerID of the node sending the RPC Request or Response + // mog: target=ID + Id []byte `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"` + // Addr is the ServerAddr of the node sending theRPC Request or Response + Addr []byte `protobuf:"bytes,3,opt,name=addr,proto3" json:"addr,omitempty"` +} + +func (x *RPCHeader) Reset() { + *x = RPCHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_types_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RPCHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RPCHeader) ProtoMessage() {} + +func (x *RPCHeader) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_types_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RPCHeader.ProtoReflect.Descriptor instead. +func (*RPCHeader) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_types_proto_rawDescGZIP(), []int{0} +} + +func (x *RPCHeader) GetProtocolVersion() int32 { + if x != nil { + return x.ProtocolVersion + } + return 0 +} + +func (x *RPCHeader) GetId() []byte { + if x != nil { + return x.Id + } + return nil +} + +func (x *RPCHeader) GetAddr() []byte { + if x != nil { + return x.Addr + } + return nil +} + +// mog annotation: +// +// target=github.com/hashicorp/raft.Log +// output=common.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +type Log struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Index holds the index of the log entry. + Index uint64 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"` + // Term holds the election term of the log entry. + Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` + // Type holds the type of the log entry. + Type uint32 `protobuf:"varint,3,opt,name=type,proto3" json:"type,omitempty"` + // Data holds the log entries type-specific data. + Data []byte `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"` + // Extensions holds an opaque byte slice of information for middleware. It + // is up to the client of the library to properly modify this as it adds + // layers and remove those layers when appropriate. This value is a part of + // the log, so very large values could cause timing issues. + // + // N.B. It is _up to the client_ to handle upgrade paths. For instance if + // using this with go-raftchunking, the client should ensure that all Raft + // peers are using a version that can handle that extension before ever + // actually triggering chunking behavior. It is sometimes sufficient to + // ensure that non-leaders are upgraded first, then the current leader is + // upgraded, but a leader changeover during this process could lead to + // trouble, so gating extension behavior via some flag in the client + // program is also a good idea. + Extensions []byte `protobuf:"bytes,5,opt,name=extensions,proto3" json:"extensions,omitempty"` + // AppendedAt stores the time the leader first appended this log to it's + // LogStore. Followers will observe the leader's time. It is not used for + // coordination or as part of the replication protocol at all. It exists only + // to provide operational information for example how many seconds worth of + // logs are present on the leader which might impact follower's ability to + // catch up after restoring a large snapshot. We should never rely on this + // being in the past when appending on a follower or reading a log back since + // the clock skew can mean a follower could see a log with a future timestamp. + // In general too the leader is not required to persist the log before + // delivering to followers although the current implementation happens to do + // this. + // Time validity bounds. + // mog: func-to=TimeFromProto func-from=TimeToProto + AppendedAt *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=appended_at,json=appendedAt,proto3" json:"appended_at,omitempty"` +} + +func (x *Log) Reset() { + *x = Log{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_transport_v1_types_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Log) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Log) ProtoMessage() {} + +func (x *Log) ProtoReflect() protoreflect.Message { + mi := &file_proto_transport_v1_types_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Log.ProtoReflect.Descriptor instead. +func (*Log) Descriptor() ([]byte, []int) { + return file_proto_transport_v1_types_proto_rawDescGZIP(), []int{1} +} + +func (x *Log) GetIndex() uint64 { + if x != nil { + return x.Index + } + return 0 +} + +func (x *Log) GetTerm() uint64 { + if x != nil { + return x.Term + } + return 0 +} + +func (x *Log) GetType() uint32 { + if x != nil { + return x.Type + } + return 0 +} + +func (x *Log) GetData() []byte { + if x != nil { + return x.Data + } + return nil +} + +func (x *Log) GetExtensions() []byte { + if x != nil { + return x.Extensions + } + return nil +} + +func (x *Log) GetAppendedAt() *timestamppb.Timestamp { + if x != nil { + return x.AppendedAt + } + return nil +} + +var File_proto_transport_v1_types_proto protoreflect.FileDescriptor + +var file_proto_transport_v1_types_proto_rawDesc = []byte{ + 0x0a, 0x1e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, + 0x74, 0x2f, 0x76, 0x31, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x12, 0x1b, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, + 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x1a, 0x1f, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x5a, + 0x0a, 0x09, 0x52, 0x50, 0x43, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x61, 0x64, 0x64, 0x72, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x61, 0x64, 0x64, 0x72, 0x22, 0xb4, 0x01, 0x0a, 0x03, 0x4c, + 0x6f, 0x67, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x12, 0x12, 0x0a, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, + 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, + 0x64, 0x61, 0x74, 0x61, 0x12, 0x1e, 0x0a, 0x0a, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3b, 0x0a, 0x0b, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x64, + 0x5f, 0x61, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x41, + 0x74, 0x42, 0xf6, 0x01, 0x0a, 0x1f, 0x63, 0x6f, 0x6d, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, + 0x6f, 0x72, 0x70, 0x2e, 0x72, 0x61, 0x66, 0x74, 0x2e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, + 0x72, 0x74, 0x2e, 0x76, 0x31, 0x42, 0x0a, 0x54, 0x79, 0x70, 0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, + 0x6f, 0x50, 0x01, 0x5a, 0x38, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, + 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x72, 0x61, 0x66, 0x74, 0x2f, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2f, 0x76, + 0x31, 0x3b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x76, 0x31, 0xa2, 0x02, 0x03, + 0x48, 0x52, 0x54, 0xaa, 0x02, 0x1b, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, + 0x52, 0x61, 0x66, 0x74, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x56, + 0x31, 0xca, 0x02, 0x1b, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x52, 0x61, + 0x66, 0x74, 0x5c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x5c, 0x56, 0x31, 0xe2, + 0x02, 0x27, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x52, 0x61, 0x66, 0x74, + 0x5c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x5c, 0x56, 0x31, 0x5c, 0x47, 0x50, + 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x1e, 0x48, 0x61, 0x73, 0x68, + 0x69, 0x63, 0x6f, 0x72, 0x70, 0x3a, 0x3a, 0x52, 0x61, 0x66, 0x74, 0x3a, 0x3a, 0x54, 0x72, 0x61, + 0x6e, 0x73, 0x70, 0x6f, 0x72, 0x74, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, +} + +var ( + file_proto_transport_v1_types_proto_rawDescOnce sync.Once + file_proto_transport_v1_types_proto_rawDescData = file_proto_transport_v1_types_proto_rawDesc +) + +func file_proto_transport_v1_types_proto_rawDescGZIP() []byte { + file_proto_transport_v1_types_proto_rawDescOnce.Do(func() { + file_proto_transport_v1_types_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_transport_v1_types_proto_rawDescData) + }) + return file_proto_transport_v1_types_proto_rawDescData +} + +var file_proto_transport_v1_types_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_proto_transport_v1_types_proto_goTypes = []interface{}{ + (*RPCHeader)(nil), // 0: hashicorp.raft.transport.v1.RPCHeader + (*Log)(nil), // 1: hashicorp.raft.transport.v1.Log + (*timestamppb.Timestamp)(nil), // 2: google.protobuf.Timestamp +} +var file_proto_transport_v1_types_proto_depIdxs = []int32{ + 2, // 0: hashicorp.raft.transport.v1.Log.appended_at:type_name -> google.protobuf.Timestamp + 1, // [1:1] is the sub-list for method output_type + 1, // [1:1] is the sub-list for method input_type + 1, // [1:1] is the sub-list for extension type_name + 1, // [1:1] is the sub-list for extension extendee + 0, // [0:1] is the sub-list for field type_name +} + +func init() { file_proto_transport_v1_types_proto_init() } +func file_proto_transport_v1_types_proto_init() { + if File_proto_transport_v1_types_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_proto_transport_v1_types_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RPCHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_transport_v1_types_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Log); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_proto_transport_v1_types_proto_rawDesc, + NumEnums: 0, + NumMessages: 2, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_proto_transport_v1_types_proto_goTypes, + DependencyIndexes: file_proto_transport_v1_types_proto_depIdxs, + MessageInfos: file_proto_transport_v1_types_proto_msgTypes, + }.Build() + File_proto_transport_v1_types_proto = out.File + file_proto_transport_v1_types_proto_rawDesc = nil + file_proto_transport_v1_types_proto_goTypes = nil + file_proto_transport_v1_types_proto_depIdxs = nil +} diff --git a/proto/transport/v1/types.proto b/proto/transport/v1/types.proto new file mode 100644 index 000000000..848dd36e8 --- /dev/null +++ b/proto/transport/v1/types.proto @@ -0,0 +1,73 @@ +syntax = "proto3"; + +package hashicorp.raft.transport.v1; + +import "google/protobuf/timestamp.proto"; + +// mog annotation: +// +// target=github.com/hashicorp/raft.RPCHeader +// output=common.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +message RPCHeader { + // ProtocolVerison is the version of the protocol the sender is speaking + int32 protocol_version = 1; + + // ID is the ServerID of the node sending the RPC Request or Response + // mog: target=ID + bytes id = 2; + + // Addr is the ServerAddr of the node sending theRPC Request or Response + bytes addr = 3; +} + +// mog annotation: +// +// target=github.com/hashicorp/raft.Log +// output=common.gen.go +// name=Struct +// ignore-fields=state,sizeCache,unknownFields +message Log { + // Index holds the index of the log entry. + uint64 index = 1; + + // Term holds the election term of the log entry. + uint64 term = 2; + + // Type holds the type of the log entry. + uint32 type = 3; + + // Data holds the log entries type-specific data. + bytes data = 4; + + // Extensions holds an opaque byte slice of information for middleware. It + // is up to the client of the library to properly modify this as it adds + // layers and remove those layers when appropriate. This value is a part of + // the log, so very large values could cause timing issues. + // + // N.B. It is _up to the client_ to handle upgrade paths. For instance if + // using this with go-raftchunking, the client should ensure that all Raft + // peers are using a version that can handle that extension before ever + // actually triggering chunking behavior. It is sometimes sufficient to + // ensure that non-leaders are upgraded first, then the current leader is + // upgraded, but a leader changeover during this process could lead to + // trouble, so gating extension behavior via some flag in the client + // program is also a good idea. + bytes extensions = 5; + + // AppendedAt stores the time the leader first appended this log to it's + // LogStore. Followers will observe the leader's time. It is not used for + // coordination or as part of the replication protocol at all. It exists only + // to provide operational information for example how many seconds worth of + // logs are present on the leader which might impact follower's ability to + // catch up after restoring a large snapshot. We should never rely on this + // being in the past when appending on a follower or reading a log back since + // the clock skew can mean a follower could see a log with a future timestamp. + // In general too the leader is not required to persist the log before + // delivering to followers although the current implementation happens to do + // this. + // Time validity bounds. + // mog: func-to=TimeFromProto func-from=TimeToProto + google.protobuf.Timestamp appended_at = 6; +} diff --git a/transports/grpc/future.go b/transports/grpc/future.go new file mode 100644 index 000000000..dffa7e4af --- /dev/null +++ b/transports/grpc/future.go @@ -0,0 +1,27 @@ +package grpctrans + +import ( + "time" + + "github.com/hashicorp/raft" +) + +type appendFuture struct { + raft.DeferError + + start time.Time + req *raft.AppendEntriesRequest + resp *raft.AppendEntriesResponse +} + +func (f *appendFuture) Start() time.Time { + return f.start +} + +func (f *appendFuture) Request() *raft.AppendEntriesRequest { + return f.req +} + +func (f *appendFuture) Response() *raft.AppendEntriesResponse { + return f.resp +} diff --git a/transports/grpc/pipeline.go b/transports/grpc/pipeline.go new file mode 100644 index 000000000..bc36a71ab --- /dev/null +++ b/transports/grpc/pipeline.go @@ -0,0 +1,101 @@ +package grpctrans + +import ( + "fmt" + "time" + + "github.com/hashicorp/go-hclog" + "github.com/hashicorp/raft" + + transportv1 "github.com/hashicorp/raft/proto/transport/v1" +) + +const ( + // rpcMaxPipeline controls the maximum number of outstanding + // AppendEntries RPC calls. + rpcMaxPipeline = 128 +) + +type gRPCPipeline struct { + cancel func() + + logger hclog.Logger + + doneCh chan raft.AppendFuture + inprogressCh chan *appendFuture + + stream transportv1.RaftTransportService_AppendEntriesPipelineClient +} + +func newGRPCPipeline(logger hclog.Logger, stream transportv1.RaftTransportService_AppendEntriesPipelineClient, cancel func()) *gRPCPipeline { + pipeline := &gRPCPipeline{ + logger: logger, + stream: stream, + + doneCh: make(chan raft.AppendFuture, rpcMaxPipeline), + inprogressCh: make(chan *appendFuture, rpcMaxPipeline), + cancel: cancel, + } + + go pipeline.processResponses() + return pipeline +} + +func (p *gRPCPipeline) processResponses() { + for { + select { + case <-p.stream.Context().Done(): + return + case future := <-p.inprogressCh: + resp, err := p.stream.Recv() + transportv1.AppendEntriesResponseToStruct(resp, future.resp) + future.Respond(err) + + select { + case p.doneCh <- future: + case <-p.stream.Context().Done(): + return + } + } + } +} + +func (p *gRPCPipeline) AppendEntries(args *raft.AppendEntriesRequest, out *raft.AppendEntriesResponse) (raft.AppendFuture, error) { + select { + case <-p.stream.Context().Done(): + p.logger.Info("AppenedEntries called on a closed pipeline") + return nil, raft.ErrPipelineShutdown + default: + } + future := &appendFuture{ + start: time.Now(), + req: args, + resp: out, + } + + future.Init() + + var req transportv1.AppendEntriesRequest + transportv1.AppendEntriesRequestFromStruct(args, &req) + if err := p.stream.Send(&req); err != nil { + return nil, fmt.Errorf("failed to send the AppendEntries RPC: %w", err) + } + + select { + case p.inprogressCh <- future: + return future, nil + case <-p.stream.Context().Done(): + p.logger.Info("pipeline cancelled which waiting to queue the future") + return nil, raft.ErrPipelineShutdown + } +} + +func (p *gRPCPipeline) Consumer() <-chan raft.AppendFuture { + return p.doneCh +} + +func (p *gRPCPipeline) Close() error { + p.cancel() + p.logger.Info("closing the grpc pipeline") + return p.stream.CloseSend() +} diff --git a/transports/grpc/server.go b/transports/grpc/server.go new file mode 100644 index 000000000..ce5fb8519 --- /dev/null +++ b/transports/grpc/server.go @@ -0,0 +1,295 @@ +package grpctrans + +import ( + "context" + "io" + "sync" + + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "github.com/hashicorp/go-hclog" + "github.com/hashicorp/raft" + transportv1 "github.com/hashicorp/raft/proto/transport/v1" +) + +var _ transportv1.RaftTransportServiceServer = &gRPCRaftService{} + +type gRPCRaftService struct { + rpcCh chan raft.RPC + logger hclog.Logger + + heartbeatFn func(raft.RPC) + heartbeatFnLock sync.Mutex +} + +func (g *gRPCRaftService) AppendEntries(ctx context.Context, req *transportv1.AppendEntriesRequest) (*transportv1.AppendEntriesResponse, error) { + var raftReq raft.AppendEntriesRequest + transportv1.AppendEntriesRequestToStruct(req, &raftReq) + + respCh := make(chan raft.RPCResponse, 1) + rpc := raft.RPC{ + Command: &raftReq, + RespChan: respCh, + } + + fastpathed := false + if isHeartbeat(req) { + g.heartbeatFnLock.Lock() + fn := g.heartbeatFn + g.heartbeatFnLock.Unlock() + + if fn != nil { + // fastpath heartbeat processing + fn(rpc) + fastpathed = true + } + } + + if !fastpathed { + select { + case g.rpcCh <- rpc: + case <-ctx.Done(): + return nil, status.FromContextError(ctx.Err()).Err() + } + } + + select { + case rpcResp := <-respCh: + if rpcResp.Error != nil { + return nil, status.Error(codes.Internal, rpcResp.Error.Error()) + } + + var resp transportv1.AppendEntriesResponse + transportv1.AppendEntriesResponseFromStruct(rpcResp.Response.(*raft.AppendEntriesResponse), &resp) + return &resp, nil + case <-ctx.Done(): + return nil, status.FromContextError(ctx.Err()).Err() + } +} + +func (g *gRPCRaftService) AppendEntriesPipeline(srv transportv1.RaftTransportService_AppendEntriesPipelineServer) error { + g.logger.Info("handling an append entries pipeline") + // This is not as efficient as it could be. To improve this we could + // queue up the append entries requests in one go routine and wait + // on their completion and send results back in a second go routine. + // If sending responses is experiencing extra latency then we would + // be better pipelining the requests. However the existing net_transport + // processes commands for a singular connection serially waiting on one + // RPC to finish before processing more commands. Therefore this should + // be at least as performant as that transport. + for { + req, err := srv.Recv() + if err != nil { + g.logger.Info("failed to recv pipelined entries", "error", err) + return err + } + resp, err := g.AppendEntries(srv.Context(), req) + if err != nil { + g.logger.Info("failed to append pipelined entries", "error", err) + return err + } + + err = srv.Send(resp) + if err != nil { + g.logger.Info("failed to send pipelined response") + return err + } + } +} + +func (g *gRPCRaftService) RequestVote(ctx context.Context, req *transportv1.RequestVoteRequest) (*transportv1.RequestVoteResponse, error) { + var raftReq raft.RequestVoteRequest + transportv1.RequestVoteRequestToStruct(req, &raftReq) + + respCh := make(chan raft.RPCResponse, 1) + rpc := raft.RPC{ + Command: &raftReq, + RespChan: respCh, + } + + select { + case g.rpcCh <- rpc: + case <-ctx.Done(): + return nil, status.FromContextError(ctx.Err()).Err() + } + + select { + case rpcResp := <-respCh: + if rpcResp.Error != nil { + return nil, status.Error(codes.Internal, rpcResp.Error.Error()) + } + + var resp transportv1.RequestVoteResponse + transportv1.RequestVoteResponseFromStruct(rpcResp.Response.(*raft.RequestVoteResponse), &resp) + return &resp, nil + case <-ctx.Done(): + return nil, status.FromContextError(ctx.Err()).Err() + } +} + +func (g *gRPCRaftService) InstallSnapshot(srv transportv1.RaftTransportService_InstallSnapshotServer) error { + var raftReq raft.InstallSnapshotRequest + + // receive the initial snapshot metadata + req, err := srv.Recv() + if err != nil { + return err + } + + meta := req.GetMetadata() + if meta != nil { + return status.Error(codes.InvalidArgument, "first message in the InstallSnapshot stream must include snapshot metadata") + } + + transportv1.InstallSnapshotMetadataToStruct(meta, &raftReq) + + errCh := make(chan error, 1) + dataCh := make(chan []byte) + respCh := make(chan raft.RPCResponse) + rdr := snapshotStreamReader{ + chunks: dataCh, + } + + go g.processSnapshotChunks(srv, dataCh, errCh, meta.Size) + + rpc := raft.RPC{ + Command: raftReq, + Reader: &rdr, + RespChan: respCh, + } + + select { + case g.rpcCh <- rpc: + case <-srv.Context().Done(): + return status.FromContextError(srv.Context().Err()).Err() + } + + select { + case rpcResp := <-respCh: + if rpcResp.Error != nil { + return status.Error(codes.Internal, rpcResp.Error.Error()) + } + + var resp transportv1.InstallSnapshotResponse + transportv1.InstallSnapshotResponseFromStruct(rpcResp.Response.(*raft.InstallSnapshotResponse), &resp) + + srv.Send(&resp) + return nil + case err := <-errCh: + return err + case <-srv.Context().Done(): + return status.FromContextError(srv.Context().Err()).Err() + } +} + +func (g *gRPCRaftService) processSnapshotChunks(srv transportv1.RaftTransportService_InstallSnapshotServer, dataCh chan<- []byte, errCh chan<- error, size int64) { + defer close(dataCh) + for size > 0 { + req, err := srv.Recv() + if err != nil { + errCh <- err + return + } + + chunk := req.GetChunk() + if chunk == nil { + errCh <- status.Error(codes.FailedPrecondition, "All InstallSnapshot stream messages after initial metadata must contain snapshot chunk data") + return + } + + if int64(len(chunk.SnapshotData)) > size { + errCh <- status.Error(codes.FailedPrecondition, "More snapshot data sent than snapshot metadata foretold") + return + } + + select { + case dataCh <- chunk.SnapshotData: + size -= int64(len(chunk.SnapshotData)) + case <-srv.Context().Done(): + return + } + } +} + +func (g *gRPCRaftService) TimeoutNow(ctx context.Context, req *transportv1.TimeoutNowRequest) (*transportv1.TimeoutNowResponse, error) { + var raftReq raft.TimeoutNowRequest + transportv1.TimeoutNowRequestToStruct(req, &raftReq) + + respCh := make(chan raft.RPCResponse, 1) + rpc := raft.RPC{ + Command: &raftReq, + RespChan: respCh, + } + + select { + case g.rpcCh <- rpc: + case <-ctx.Done(): + return nil, status.FromContextError(ctx.Err()).Err() + } + + select { + case rpcResp := <-respCh: + if rpcResp.Error != nil { + return nil, status.Error(codes.Internal, rpcResp.Error.Error()) + } + + var resp transportv1.TimeoutNowResponse + transportv1.TimeoutNowResponseFromStruct(rpcResp.Response.(*raft.TimeoutNowResponse), &resp) + return &resp, nil + case <-ctx.Done(): + return nil, status.FromContextError(ctx.Err()).Err() + } +} + +func (g *gRPCRaftService) setHeartbeatHandler(cb func(raft.RPC)) { + g.heartbeatFnLock.Lock() + g.heartbeatFn = cb + g.heartbeatFnLock.Unlock() +} + +func (g *gRPCRaftService) registerService(srv grpc.ServiceRegistrar) { + transportv1.RegisterRaftTransportServiceServer(srv, g) +} + +func isHeartbeat(req *transportv1.AppendEntriesRequest) bool { + return req.Term != 0 && req.GetRpcHeader().Addr != nil && + req.PreviousLogEntry == 0 && req.PreviousLogTerm == 0 && + len(req.Entries) == 0 && req.LeaderCommitIndex == 0 +} + +type snapshotStreamReader struct { + streamErr chan<- error + chunks <-chan []byte + extra []byte +} + +func (s *snapshotStreamReader) Read(buf []byte) (int, error) { + // Check if we need more data from the stream + if s.extra == nil { + ok := false + s.extra, ok = <-s.chunks + + if !ok { + return 0, io.EOF + } + } + + // This may only partially fill the buffer with data available. + // According to the io.Reader interface that is okay. We could + // make this more efficient by proactively populating more + // buffer and by looping to fill the provided buffer. For now + // this is simple and should work. + copied := copy(buf, s.extra) + + if copied == len(s.extra) { + // we have consumed all the buffered data so nil it out + s.extra = nil + } else { + // reslice s.extra to move past the already copied data + s.extra = s.extra[copied:] + } + + return copied, nil +} diff --git a/transports/grpc/transport.go b/transports/grpc/transport.go new file mode 100644 index 000000000..d54e00069 --- /dev/null +++ b/transports/grpc/transport.go @@ -0,0 +1,256 @@ +package grpctrans + +import ( + "context" + "fmt" + "io" + + "github.com/hashicorp/go-hclog" + transportv1 "github.com/hashicorp/raft/proto/transport/v1" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + + "github.com/hashicorp/raft" +) + +var _ raft.Transport = &GRPCTransport{} + +const ( + // snapshotBufferSize is the size of the snapshot chunks that will be + // sent when making InstallSnapshot RPCs + snapshotChunkSize = 256 * 1024 // 256KB +) + +type GRPCTransport struct { + logger hclog.Logger + + serverAddressProvider raft.ServerAddressProvider + + consumeCh chan raft.RPC + + service *gRPCRaftService + + localAddr raft.ServerAddress +} + +type GRPCTransportConfig struct { + // ServerAddressProvider is used to override the target address when establishing a connection to invoke an RPC + ServerAddressProvider raft.ServerAddressProvider + + Logger hclog.Logger + + LocalAddr raft.ServerAddress +} + +func New(conf GRPCTransportConfig) *GRPCTransport { + rpcCh := make(chan raft.RPC) + return &GRPCTransport{ + logger: conf.Logger, + localAddr: conf.LocalAddr, + serverAddressProvider: conf.ServerAddressProvider, + consumeCh: rpcCh, + service: &gRPCRaftService{rpcCh: rpcCh, logger: conf.Logger}, + } +} + +func (g *GRPCTransport) Close() error { + return nil +} + +func (g *GRPCTransport) Consumer() <-chan raft.RPC { + return g.consumeCh +} + +func (g *GRPCTransport) LocalAddr() raft.ServerAddress { + return g.localAddr +} + +func (g *GRPCTransport) getClient(ctx context.Context, target raft.ServerAddress) (transportv1.RaftTransportServiceClient, error) { + conn, err := grpc.DialContext(ctx, string(target), grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + return nil, err + } + + return transportv1.NewRaftTransportServiceClient(conn), nil +} + +func (g *GRPCTransport) AppendEntriesPipeline(id raft.ServerID, target raft.ServerAddress) (raft.AppendPipeline, error) { + ctx, cancel := context.WithCancel(context.Background()) + + client, err := g.getClient(ctx, target) + if err != nil { + cancel() + return nil, err + } + + stream, err := client.AppendEntriesPipeline(ctx) + if err != nil { + cancel() + return nil, err + } + + return newGRPCPipeline(g.logger.With("id", id, "address", target), stream, cancel), nil +} + +func (g *GRPCTransport) AppendEntries(id raft.ServerID, target raft.ServerAddress, args *raft.AppendEntriesRequest, out *raft.AppendEntriesResponse) error { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + var req transportv1.AppendEntriesRequest + transportv1.AppendEntriesRequestFromStruct(args, &req) + + client, err := g.getClient(ctx, target) + if err != nil { + return err + } + + resp, err := client.AppendEntries(ctx, &req) + if err != nil { + return err + } + transportv1.AppendEntriesResponseToStruct(resp, out) + return nil +} + +func (g *GRPCTransport) RequestVote(id raft.ServerID, target raft.ServerAddress, args *raft.RequestVoteRequest, out *raft.RequestVoteResponse) error { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + var req transportv1.RequestVoteRequest + transportv1.RequestVoteRequestFromStruct(args, &req) + + client, err := g.getClient(ctx, target) + if err != nil { + return err + } + + resp, err := client.RequestVote(ctx, &req) + if err != nil { + return err + } + transportv1.RequestVoteResponseToStruct(resp, out) + return nil +} + +func (g *GRPCTransport) InstallSnapshot(id raft.ServerID, target raft.ServerAddress, args *raft.InstallSnapshotRequest, out *raft.InstallSnapshotResponse, data io.Reader) error { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + var meta transportv1.InstallSnapshotMetadata + + transportv1.InstallSnapshotMetadataFromStruct(args, &meta) + + req := transportv1.InstallSnapshotRequest{ + Message: &transportv1.InstallSnapshotRequest_Metadata{ + Metadata: &meta, + }, + } + + client, err := g.getClient(ctx, target) + if err != nil { + return err + } + + stream, err := client.InstallSnapshot(ctx) + if err != nil { + return fmt.Errorf("failed to initiate InstallSnapshot stream") + } + + err = stream.Send(&req) + if err != nil { + return fmt.Errorf("failed to send initial snapshot metadata: %w", err) + } + + // make sure we don't read more than the snapshot size + rdr := io.LimitReader(data, args.Size) + + var written int64 + buf := make([]byte, snapshotChunkSize) + var chunk transportv1.InstallSnapshotChunk + done := false + for !done { + n, err := rdr.Read(buf) + if n > 0 { + if written+int64(n) > args.Size { + return fmt.Errorf("provided snapshots data would exceed specified size") + } + + req.Reset() + chunk.Reset() + chunk.SnapshotData = buf[:n] + req.Message = &transportv1.InstallSnapshotRequest_Chunk{Chunk: &chunk} + + err := stream.Send(&req) + if err != nil { + return fmt.Errorf("failed to send snapshot data: %w", err) + } + written += int64(n) + } + + if err == io.EOF { + if written < args.Size { + return fmt.Errorf("provided snapshot data is smaller than specified size") + } + done = true + } else if err != nil { + return fmt.Errorf("encountered an error while reading snapshot data: %w", err) + } + } + + resp, err := stream.Recv() + if err != nil { + return fmt.Errorf("error receiving InstallSnapshot response: %w", err) + } + + transportv1.InstallSnapshotResponseToStruct(resp, out) + return nil +} + +func (g *GRPCTransport) EncodePeer(id raft.ServerID, addr raft.ServerAddress) []byte { + return []byte(g.getProviderAddressOrFallback(id, addr)) +} + +func (g *GRPCTransport) DecodePeer(data []byte) raft.ServerAddress { + return raft.ServerAddress(data) +} + +func (g *GRPCTransport) SetHeartbeatHandler(cb func(rpc raft.RPC)) { + g.service.setHeartbeatHandler(cb) +} + +func (g *GRPCTransport) TimeoutNow(id raft.ServerID, target raft.ServerAddress, args *raft.TimeoutNowRequest, out *raft.TimeoutNowResponse) error { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + var req transportv1.TimeoutNowRequest + transportv1.TimeoutNowRequestFromStruct(args, &req) + + conn, err := grpc.Dial(string(target), grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + return err + } + + client := transportv1.NewRaftTransportServiceClient(conn) + resp, err := client.TimeoutNow(ctx, &req) + if err != nil { + return err + } + transportv1.TimeoutNowResponseToStruct(resp, out) + return nil +} + +func (g *GRPCTransport) getProviderAddressOrFallback(id raft.ServerID, target raft.ServerAddress) raft.ServerAddress { + if g.serverAddressProvider != nil { + serverAddressOverride, err := g.serverAddressProvider.ServerAddr(id) + if err != nil { + g.logger.Warn("unable to get address for server, using fallback address", "id", id, "fallback", target, "error", err) + } else { + return serverAddressOverride + } + } + return target +} + +func (g *GRPCTransport) RegisterRaftRPCServices(srv grpc.ServiceRegistrar) { + g.service.registerService(srv) +}