Merge pull request #151 from xiangli-cmu/refactor_proto

WIP Refactor proto
pull/820/head
Ben Johnson 2014-01-23 06:18:29 -08:00
commit ad05eb4872
32 changed files with 4984 additions and 362 deletions

View File

@ -1,10 +1,11 @@
package raft
import (
"code.google.com/p/goprotobuf/proto"
"github.com/goraft/raft/protobuf"
"io"
"io/ioutil"
"code.google.com/p/gogoprotobuf/proto"
"github.com/goraft/raft/protobuf"
)
// The request sent to a server to append entries to the log.
@ -14,43 +15,38 @@ type AppendEntriesRequest struct {
PrevLogTerm uint64
CommitIndex uint64
LeaderName string
Entries []*LogEntry
Entries []*protobuf.LogEntry
}
// Creates a new AppendEntries request.
func newAppendEntriesRequest(term uint64, prevLogIndex uint64, prevLogTerm uint64, commitIndex uint64, leaderName string, entries []*LogEntry) *AppendEntriesRequest {
func newAppendEntriesRequest(term uint64, prevLogIndex uint64, prevLogTerm uint64,
commitIndex uint64, leaderName string, entries []*LogEntry) *AppendEntriesRequest {
pbEntries := make([]*protobuf.LogEntry, len(entries))
for i := range entries {
pbEntries[i] = entries[i].pb
}
return &AppendEntriesRequest{
Term: term,
PrevLogIndex: prevLogIndex,
PrevLogTerm: prevLogTerm,
CommitIndex: commitIndex,
LeaderName: leaderName,
Entries: entries,
Entries: pbEntries,
}
}
// Encodes the AppendEntriesRequest to a buffer. Returns the number of bytes
// written and any error that may have occurred.
func (req *AppendEntriesRequest) Encode(w io.Writer) (int, error) {
protoEntries := make([]*protobuf.ProtoAppendEntriesRequest_ProtoLogEntry, len(req.Entries))
for i, entry := range req.Entries {
protoEntries[i] = &protobuf.ProtoAppendEntriesRequest_ProtoLogEntry{
Index: proto.Uint64(entry.Index),
Term: proto.Uint64(entry.Term),
CommandName: proto.String(entry.CommandName),
Command: entry.Command,
}
}
pb := &protobuf.ProtoAppendEntriesRequest{
pb := &protobuf.AppendEntriesRequest{
Term: proto.Uint64(req.Term),
PrevLogIndex: proto.Uint64(req.PrevLogIndex),
PrevLogTerm: proto.Uint64(req.PrevLogTerm),
CommitIndex: proto.Uint64(req.CommitIndex),
LeaderName: proto.String(req.LeaderName),
Entries: protoEntries,
Entries: req.Entries,
}
p, err := proto.Marshal(pb)
@ -70,9 +66,7 @@ func (req *AppendEntriesRequest) Decode(r io.Reader) (int, error) {
return -1, err
}
totalBytes := len(data)
pb := &protobuf.ProtoAppendEntriesRequest{}
pb := new(protobuf.AppendEntriesRequest)
if err := proto.Unmarshal(data, pb); err != nil {
return -1, err
}
@ -82,17 +76,7 @@ func (req *AppendEntriesRequest) Decode(r io.Reader) (int, error) {
req.PrevLogTerm = pb.GetPrevLogTerm()
req.CommitIndex = pb.GetCommitIndex()
req.LeaderName = pb.GetLeaderName()
req.Entries = pb.GetEntries()
req.Entries = make([]*LogEntry, len(pb.Entries))
for i, entry := range pb.Entries {
req.Entries[i] = &LogEntry{
Index: entry.GetIndex(),
Term: entry.GetTerm(),
CommandName: entry.GetCommandName(),
Command: entry.Command,
}
}
return totalBytes, nil
return len(data), nil
}

View File

@ -1,70 +1,73 @@
package raft
import (
"code.google.com/p/goprotobuf/proto"
"github.com/goraft/raft/protobuf"
"io"
"io/ioutil"
"code.google.com/p/gogoprotobuf/proto"
"github.com/goraft/raft/protobuf"
)
// The response returned from a server appending entries to the log.
type AppendEntriesResponse struct {
Term uint64
// the current index of the server
Index uint64
Success bool
CommitIndex uint64
peer string
append bool
pb *protobuf.AppendEntriesResponse
peer string
append bool
}
// Creates a new AppendEntries response.
func newAppendEntriesResponse(term uint64, success bool, index uint64, commitIndex uint64) *AppendEntriesResponse {
return &AppendEntriesResponse{
Term: term,
Success: success,
Index: index,
CommitIndex: commitIndex,
pb := &protobuf.AppendEntriesResponse{
Term: proto.Uint64(term),
Index: proto.Uint64(index),
Success: proto.Bool(success),
CommitIndex: proto.Uint64(commitIndex),
}
return &AppendEntriesResponse{
pb: pb,
}
}
func (aer *AppendEntriesResponse) Index() uint64 {
return aer.pb.GetIndex()
}
func (aer *AppendEntriesResponse) CommitIndex() uint64 {
return aer.pb.GetCommitIndex()
}
func (aer *AppendEntriesResponse) Term() uint64 {
return aer.pb.GetTerm()
}
func (aer *AppendEntriesResponse) Success() bool {
return aer.pb.GetSuccess()
}
// Encodes the AppendEntriesResponse to a buffer. Returns the number of bytes
// written and any error that may have occurred.
func (resp *AppendEntriesResponse) Encode(w io.Writer) (int, error) {
pb := &protobuf.ProtoAppendEntriesResponse{
Term: proto.Uint64(resp.Term),
Index: proto.Uint64(resp.Index),
CommitIndex: proto.Uint64(resp.CommitIndex),
Success: proto.Bool(resp.Success),
}
p, err := proto.Marshal(pb)
b, err := proto.Marshal(resp.pb)
if err != nil {
return -1, err
}
return w.Write(p)
return w.Write(b)
}
// Decodes the AppendEntriesResponse from a buffer. Returns the number of bytes read and
// any error that occurs.
func (resp *AppendEntriesResponse) Decode(r io.Reader) (int, error) {
data, err := ioutil.ReadAll(r)
if err != nil {
return -1, err
}
totalBytes := len(data)
pb := &protobuf.ProtoAppendEntriesResponse{}
if err := proto.Unmarshal(data, pb); err != nil {
resp.pb = new(protobuf.AppendEntriesResponse)
if err := proto.Unmarshal(data, resp.pb); err != nil {
return -1, err
}
resp.Term = pb.GetTerm()
resp.Index = pb.GetIndex()
resp.CommitIndex = pb.GetCommitIndex()
resp.Success = pb.GetSuccess()
return totalBytes, nil
return len(data), nil
}

74
log.go
View File

@ -2,13 +2,13 @@ package raft
import (
"bufio"
"code.google.com/p/goprotobuf/proto"
"errors"
"fmt"
"github.com/goraft/raft/protobuf"
"io"
"os"
"sync"
"github.com/goraft/raft/protobuf"
)
//------------------------------------------------------------------------------
@ -27,8 +27,7 @@ type Log struct {
mutex sync.RWMutex
startIndex uint64 // the index before the first entry in the Log entries
startTerm uint64
pBuffer *proto.Buffer
pLogEntry *protobuf.ProtoLogEntry
pLogEntry *protobuf.LogEntry
}
// The results of the applying a log entry.
@ -47,8 +46,7 @@ type logResult struct {
func newLog() *Log {
return &Log{
entries: make([]*LogEntry, 0),
pBuffer: proto.NewBuffer(nil),
pLogEntry: &protobuf.ProtoLogEntry{},
pLogEntry: &protobuf.LogEntry{},
}
}
@ -81,7 +79,7 @@ func (l *Log) internalCurrentIndex() uint64 {
if len(l.entries) == 0 {
return l.startIndex
}
return l.entries[len(l.entries)-1].Index
return l.entries[len(l.entries)-1].Index()
}
// The next index in the log.
@ -102,7 +100,7 @@ func (l *Log) lastCommandName() string {
defer l.mutex.RUnlock()
if len(l.entries) > 0 {
if entry := l.entries[len(l.entries)-1]; entry != nil {
return entry.CommandName
return entry.CommandName()
}
}
return ""
@ -120,7 +118,7 @@ func (l *Log) currentTerm() uint64 {
if len(l.entries) == 0 {
return l.startTerm
}
return l.entries[len(l.entries)-1].Term
return l.entries[len(l.entries)-1].Term()
}
//------------------------------------------------------------------------------
@ -175,17 +173,17 @@ func (l *Log) open(path string) error {
}
break
}
if entry.Index > l.startIndex {
if entry.Index() > l.startIndex {
// Append entry.
l.entries = append(l.entries, entry)
if entry.Index <= l.commitIndex {
command, err := newCommand(entry.CommandName, entry.Command)
if entry.Index() <= l.commitIndex {
command, err := newCommand(entry.CommandName(), entry.Command())
if err != nil {
continue
}
l.ApplyFunc(command)
}
debugln("open.log.append log index ", entry.Index)
debugln("open.log.append log index ", entry.Index())
}
readBytes += int64(n)
@ -235,7 +233,7 @@ func (l *Log) getEntry(index uint64) *LogEntry {
// Checks if the log contains a given index/term combination.
func (l *Log) containsEntry(index uint64, term uint64) bool {
entry := l.getEntry(index)
return (entry != nil && entry.Term == term)
return (entry != nil && entry.Term() == term)
}
// Retrieves a list of entries after a given index as well as the term of the
@ -271,9 +269,9 @@ func (l *Log) getEntriesAfter(index uint64, maxLogEntriesPerRequest uint64) ([]*
if uint64(length) < maxLogEntriesPerRequest {
// Determine the term at the given entry and return a subslice.
return entries, l.entries[index-1-l.startIndex].Term
return entries, l.entries[index-1-l.startIndex].Term()
} else {
return entries[:maxLogEntriesPerRequest], l.entries[index-1-l.startIndex].Term
return entries[:maxLogEntriesPerRequest], l.entries[index-1-l.startIndex].Term()
}
}
@ -298,7 +296,7 @@ func (l *Log) commitInfo() (index uint64, term uint64) {
// Return the last index & term from the last committed entry.
debugln("commitInfo.get.[", l.commitIndex, "/", l.startIndex, "]")
entry := l.entries[l.commitIndex-1-l.startIndex]
return entry.Index, entry.Term
return entry.Index(), entry.Term()
}
// Retrieves the last index and term that has been appended to the log.
@ -313,7 +311,7 @@ func (l *Log) lastInfo() (index uint64, term uint64) {
// Return the last index & term
entry := l.entries[len(l.entries)-1]
return entry.Index, entry.Term
return entry.Index(), entry.Term()
}
// Updates the commit index
@ -363,16 +361,17 @@ func (l *Log) setCommitIndex(index uint64) error {
entry := l.entries[entryIndex]
// Update commit index.
l.commitIndex = entry.Index
l.commitIndex = entry.Index()
// Decode the command.
command, err := newCommand(entry.CommandName, entry.Command)
command, err := newCommand(entry.CommandName(), entry.Command())
if err != nil {
return err
}
// Apply the changes to the state machine and store the error code.
returnValue, err := l.ApplyFunc(command)
debugf("setCommitIndex.set.result index: %v, entries index: %v", i, entryIndex)
if entry.event != nil {
entry.event.returnValue = returnValue
@ -430,9 +429,9 @@ func (l *Log) truncate(index uint64, term uint64) error {
} else {
// Do not truncate if the entry at index does not have the matching term.
entry := l.entries[index-l.startIndex-1]
if len(l.entries) > 0 && entry.Term != term {
if len(l.entries) > 0 && entry.Term() != term {
debugln("log.truncate.termMismatch")
return fmt.Errorf("raft.Log: Entry at index does not have matching term (%v): (IDX=%v, TERM=%v)", entry.Term, index, term)
return fmt.Errorf("raft.Log: Entry at index does not have matching term (%v): (IDX=%v, TERM=%v)", entry.Term(), index, term)
}
// Otherwise truncate up to the desired entry.
@ -462,7 +461,7 @@ func (l *Log) truncate(index uint64, term uint64) error {
//--------------------------------------
// Appends a series of entries to the log.
func (l *Log) appendEntries(entries []*LogEntry) error {
func (l *Log) appendEntries(entries []*protobuf.LogEntry) error {
l.mutex.Lock()
defer l.mutex.Unlock()
@ -473,12 +472,17 @@ func (l *Log) appendEntries(entries []*LogEntry) error {
var size int64
var err error
// Append each entry but exit if we hit an error.
for _, entry := range entries {
entry.log = l
if size, err = l.writeEntry(entry, w); err != nil {
for i := range entries {
logEntry := &LogEntry{
log: l,
Position: startPosition,
pb: entries[i],
}
if size, err = l.writeEntry(logEntry, w); err != nil {
return err
}
entry.Position = startPosition
startPosition += size
}
w.Flush()
@ -503,10 +507,10 @@ func (l *Log) appendEntry(entry *LogEntry) error {
// Make sure the term and index are greater than the previous.
if len(l.entries) > 0 {
lastEntry := l.entries[len(l.entries)-1]
if entry.Term < lastEntry.Term {
return fmt.Errorf("raft.Log: Cannot append entry with earlier term (%x:%x <= %x:%x)", entry.Term, entry.Index, lastEntry.Term, lastEntry.Index)
} else if entry.Term == lastEntry.Term && entry.Index <= lastEntry.Index {
return fmt.Errorf("raft.Log: Cannot append entry with earlier index in the same term (%x:%x <= %x:%x)", entry.Term, entry.Index, lastEntry.Term, lastEntry.Index)
if entry.Term() < lastEntry.Term() {
return fmt.Errorf("raft.Log: Cannot append entry with earlier term (%x:%x <= %x:%x)", entry.Term(), entry.Index(), lastEntry.Term(), lastEntry.Index())
} else if entry.Term() == lastEntry.Term() && entry.Index() <= lastEntry.Index() {
return fmt.Errorf("raft.Log: Cannot append entry with earlier index in the same term (%x:%x <= %x:%x)", entry.Term(), entry.Index(), lastEntry.Term(), lastEntry.Index())
}
}
@ -534,10 +538,10 @@ func (l *Log) writeEntry(entry *LogEntry, w io.Writer) (int64, error) {
// Make sure the term and index are greater than the previous.
if len(l.entries) > 0 {
lastEntry := l.entries[len(l.entries)-1]
if entry.Term < lastEntry.Term {
return -1, fmt.Errorf("raft.Log: Cannot append entry with earlier term (%x:%x <= %x:%x)", entry.Term, entry.Index, lastEntry.Term, lastEntry.Index)
} else if entry.Term == lastEntry.Term && entry.Index <= lastEntry.Index {
return -1, fmt.Errorf("raft.Log: Cannot append entry with earlier index in the same term (%x:%x <= %x:%x)", entry.Term, entry.Index, lastEntry.Term, lastEntry.Index)
if entry.Term() < lastEntry.Term() {
return -1, fmt.Errorf("raft.Log: Cannot append entry with earlier term (%x:%x <= %x:%x)", entry.Term(), entry.Index(), lastEntry.Term(), lastEntry.Index())
} else if entry.Term() == lastEntry.Term() && entry.Index() <= lastEntry.Index() {
return -1, fmt.Errorf("raft.Log: Cannot append entry with earlier index in the same term (%x:%x <= %x:%x)", entry.Term(), entry.Index(), lastEntry.Term(), lastEntry.Index())
}
}

View File

@ -2,22 +2,20 @@ package raft
import (
"bytes"
"code.google.com/p/goprotobuf/proto"
"encoding/json"
"fmt"
"github.com/goraft/raft/protobuf"
"io"
"code.google.com/p/gogoprotobuf/proto"
"github.com/goraft/raft/protobuf"
)
// A log entry stores a single item in the log.
type LogEntry struct {
log *Log
Index uint64
Term uint64
CommandName string
Command []byte
Position int64 // position in the log file
event *ev
pb *protobuf.LogEntry
Position int64 // position in the log file
log *Log
event *ev
}
// Creates a new log entry associated with a log.
@ -35,37 +33,51 @@ func newLogEntry(log *Log, event *ev, index uint64, term uint64, command Command
}
}
e := &LogEntry{
log: log,
Index: index,
Term: term,
CommandName: commandName,
pb := &protobuf.LogEntry{
Index: proto.Uint64(index),
Term: proto.Uint64(term),
CommandName: proto.String(commandName),
Command: buf.Bytes(),
event: event,
}
e := &LogEntry{
pb: pb,
log: log,
event: event,
}
return e, nil
}
func (e *LogEntry) Index() uint64 {
return e.pb.GetIndex()
}
func (e *LogEntry) Term() uint64 {
return e.pb.GetTerm()
}
func (e *LogEntry) CommandName() string {
return e.pb.GetCommandName()
}
func (e *LogEntry) Command() []byte {
return e.pb.GetCommand()
}
// Encodes the log entry to a buffer. Returns the number of bytes
// written and any error that may have occurred.
func (e *LogEntry) encode(w io.Writer) (int, error) {
defer e.log.pBuffer.Reset()
e.log.pLogEntry.Index = proto.Uint64(e.Index)
e.log.pLogEntry.Term = proto.Uint64(e.Term)
e.log.pLogEntry.CommandName = proto.String(e.CommandName)
e.log.pLogEntry.Command = e.Command
err := e.log.pBuffer.Marshal(e.log.pLogEntry)
b, err := proto.Marshal(e.pb)
if err != nil {
return -1, err
}
if _, err = fmt.Fprintf(w, "%8x\n", len(e.log.pBuffer.Bytes())); err != nil {
if _, err = fmt.Fprintf(w, "%8x\n", len(b)); err != nil {
return -1, err
}
return w.Write(e.log.pBuffer.Bytes())
return w.Write(b)
}
// Decodes the log entry from a buffer. Returns the number of bytes read and
@ -85,15 +97,9 @@ func (e *LogEntry) decode(r io.Reader) (int, error) {
return -1, err
}
pb := &protobuf.ProtoLogEntry{}
if err = proto.Unmarshal(data, pb); err != nil {
if err = proto.Unmarshal(data, e.pb); err != nil {
return -1, err
}
e.Term = pb.GetTerm()
e.Index = pb.GetIndex()
e.CommandName = pb.GetCommandName()
e.Command = pb.Command
return length + 8 + 1, nil
}

View File

@ -74,13 +74,13 @@ func TestLogExistingLog(t *testing.T) {
if len(log.entries) != 3 {
t.Fatalf("Expected 3 entries, got %d", len(log.entries))
}
if log.entries[0].Index != 1 || log.entries[0].Term != 1 {
if log.entries[0].Index() != 1 || log.entries[0].Term() != 1 {
t.Fatalf("Unexpected entry[0]: %v", log.entries[0])
}
if log.entries[1].Index != 2 || log.entries[1].Term != 1 {
if log.entries[1].Index() != 2 || log.entries[1].Term() != 1 {
t.Fatalf("Unexpected entry[1]: %v", log.entries[1])
}
if log.entries[2].Index != 3 || log.entries[2].Term != 2 {
if log.entries[2].Index() != 3 || log.entries[2].Term() != 2 {
t.Fatalf("Unexpected entry[2]: %v", log.entries[2])
}
}
@ -143,13 +143,13 @@ func TestLogRecovery(t *testing.T) {
if len(log.entries) != 3 {
t.Fatalf("Expected 3 entries, got %d", len(log.entries))
}
if log.entries[0].Index != 1 || log.entries[0].Term != 1 {
if log.entries[0].Index() != 1 || log.entries[0].Term() != 1 {
t.Fatalf("Unexpected entry[0]: %v", log.entries[0])
}
if log.entries[1].Index != 2 || log.entries[1].Term != 1 {
if log.entries[1].Index() != 2 || log.entries[1].Term() != 1 {
t.Fatalf("Unexpected entry[1]: %v", log.entries[1])
}
if log.entries[2].Index != 3 || log.entries[2].Term != 2 {
if log.entries[2].Index() != 3 || log.entries[2].Term() != 2 {
t.Fatalf("Unexpected entry[2]: %v", log.entries[2])
}
}
@ -220,13 +220,13 @@ func TestLogTruncate(t *testing.T) {
if len(log.entries) != 3 {
t.Fatalf("Expected 3 entries, got %d", len(log.entries))
}
if log.entries[0].Index != 1 || log.entries[0].Term != 1 {
if log.entries[0].Index() != 1 || log.entries[0].Term() != 1 {
t.Fatalf("Unexpected entry[0]: %v", log.entries[0])
}
if log.entries[1].Index != 2 || log.entries[1].Term != 1 {
if log.entries[1].Index() != 2 || log.entries[1].Term() != 1 {
t.Fatalf("Unexpected entry[1]: %v", log.entries[1])
}
if log.entries[2].Index != 3 || log.entries[2].Term != 2 {
if log.entries[2].Index() != 3 || log.entries[2].Term() != 2 {
t.Fatalf("Unexpected entry[2]: %v", log.entries[2])
}
}

18
peer.go
View File

@ -178,13 +178,13 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
// If successful then update the previous log index.
p.mutex.Lock()
if resp.Success {
if resp.Success() {
if len(req.Entries) > 0 {
p.prevLogIndex = req.Entries[len(req.Entries)-1].Index
p.prevLogIndex = req.Entries[len(req.Entries)-1].GetIndex()
// if peer append a log entry from the current term
// we set append to true
if req.Entries[len(req.Entries)-1].Term == p.server.currentTerm {
if req.Entries[len(req.Entries)-1].GetTerm() == p.server.currentTerm {
resp.append = true
}
}
@ -192,7 +192,7 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
// If it was unsuccessful then decrement the previous log index and
// we'll try again next time.
} else {
if resp.CommitIndex >= p.prevLogIndex {
if resp.CommitIndex() >= p.prevLogIndex {
// we may miss a response from peer
// so maybe the peer has committed the logs we just sent
// but we did not receive the successful reply and did not increase
@ -201,7 +201,7 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
// peer failed to truncate the log and sent a fail reply at this time
// we just need to update peer's prevLog index to commitIndex
p.prevLogIndex = resp.CommitIndex
p.prevLogIndex = resp.CommitIndex()
debugln("peer.append.resp.update: ", p.Name, "; idx =", p.prevLogIndex)
} else if p.prevLogIndex > 0 {
@ -210,8 +210,8 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
// problem.
p.prevLogIndex--
// if it not enough, we directly decrease to the index of the
if p.prevLogIndex > resp.Index {
p.prevLogIndex = resp.Index
if p.prevLogIndex > resp.Index() {
p.prevLogIndex = resp.Index()
}
debugln("peer.append.resp.decrement: ", p.Name, "; idx =", p.prevLogIndex)
@ -265,8 +265,8 @@ func (p *Peer) sendSnapshotRecoveryRequest() {
debugln("peer.snap.recovery.failed: ", p.Name)
return
}
// Send response to server for processing.
p.server.sendAsync(&AppendEntriesResponse{Term: resp.Term, Success: resp.Success, append: (resp.Term == p.server.currentTerm)})
p.server.sendAsync(resp)
}
//--------------------------------------

View File

@ -1,115 +1,677 @@
// Code generated by protoc-gen-go.
// Code generated by protoc-gen-gogo.
// source: append_entries_request.proto
// DO NOT EDIT!
package protobuf
import proto "code.google.com/p/goprotobuf/proto"
import proto "code.google.com/p/gogoprotobuf/proto"
import json "encoding/json"
import math "math"
// discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb"
import io1 "io"
import code_google_com_p_gogoprotobuf_proto2 "code.google.com/p/gogoprotobuf/proto"
import fmt3 "fmt"
import strings2 "strings"
import reflect2 "reflect"
import fmt4 "fmt"
import strings3 "strings"
import code_google_com_p_gogoprotobuf_proto3 "code.google.com/p/gogoprotobuf/proto"
import sort1 "sort"
import strconv1 "strconv"
import reflect3 "reflect"
import fmt5 "fmt"
import bytes1 "bytes"
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
var _ = proto.Marshal
var _ = &json.SyntaxError{}
var _ = math.Inf
type ProtoAppendEntriesRequest struct {
Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"`
PrevLogIndex *uint64 `protobuf:"varint,2,req" json:"PrevLogIndex,omitempty"`
PrevLogTerm *uint64 `protobuf:"varint,3,req" json:"PrevLogTerm,omitempty"`
CommitIndex *uint64 `protobuf:"varint,4,req" json:"CommitIndex,omitempty"`
LeaderName *string `protobuf:"bytes,5,req" json:"LeaderName,omitempty"`
Entries []*ProtoAppendEntriesRequest_ProtoLogEntry `protobuf:"bytes,6,rep" json:"Entries,omitempty"`
XXX_unrecognized []byte `json:"-"`
type AppendEntriesRequest struct {
Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"`
PrevLogIndex *uint64 `protobuf:"varint,2,req" json:"PrevLogIndex,omitempty"`
PrevLogTerm *uint64 `protobuf:"varint,3,req" json:"PrevLogTerm,omitempty"`
CommitIndex *uint64 `protobuf:"varint,4,req" json:"CommitIndex,omitempty"`
LeaderName *string `protobuf:"bytes,5,req" json:"LeaderName,omitempty"`
Entries []*LogEntry `protobuf:"bytes,6,rep" json:"Entries,omitempty"`
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoAppendEntriesRequest) Reset() { *m = ProtoAppendEntriesRequest{} }
func (m *ProtoAppendEntriesRequest) String() string { return proto.CompactTextString(m) }
func (*ProtoAppendEntriesRequest) ProtoMessage() {}
func (m *AppendEntriesRequest) Reset() { *m = AppendEntriesRequest{} }
func (*AppendEntriesRequest) ProtoMessage() {}
func (m *ProtoAppendEntriesRequest) GetTerm() uint64 {
func (m *AppendEntriesRequest) GetTerm() uint64 {
if m != nil && m.Term != nil {
return *m.Term
}
return 0
}
func (m *ProtoAppendEntriesRequest) GetPrevLogIndex() uint64 {
func (m *AppendEntriesRequest) GetPrevLogIndex() uint64 {
if m != nil && m.PrevLogIndex != nil {
return *m.PrevLogIndex
}
return 0
}
func (m *ProtoAppendEntriesRequest) GetPrevLogTerm() uint64 {
func (m *AppendEntriesRequest) GetPrevLogTerm() uint64 {
if m != nil && m.PrevLogTerm != nil {
return *m.PrevLogTerm
}
return 0
}
func (m *ProtoAppendEntriesRequest) GetCommitIndex() uint64 {
func (m *AppendEntriesRequest) GetCommitIndex() uint64 {
if m != nil && m.CommitIndex != nil {
return *m.CommitIndex
}
return 0
}
func (m *ProtoAppendEntriesRequest) GetLeaderName() string {
func (m *AppendEntriesRequest) GetLeaderName() string {
if m != nil && m.LeaderName != nil {
return *m.LeaderName
}
return ""
}
func (m *ProtoAppendEntriesRequest) GetEntries() []*ProtoAppendEntriesRequest_ProtoLogEntry {
func (m *AppendEntriesRequest) GetEntries() []*LogEntry {
if m != nil {
return m.Entries
}
return nil
}
type ProtoAppendEntriesRequest_ProtoLogEntry struct {
Index *uint64 `protobuf:"varint,1,req" json:"Index,omitempty"`
Term *uint64 `protobuf:"varint,2,req" json:"Term,omitempty"`
CommandName *string `protobuf:"bytes,3,req" json:"CommandName,omitempty"`
Command []byte `protobuf:"bytes,4,opt" json:"Command,omitempty"`
XXX_unrecognized []byte `json:"-"`
func init() {
}
func (m *ProtoAppendEntriesRequest_ProtoLogEntry) Reset() {
*m = ProtoAppendEntriesRequest_ProtoLogEntry{}
}
func (m *ProtoAppendEntriesRequest_ProtoLogEntry) String() string { return proto.CompactTextString(m) }
func (*ProtoAppendEntriesRequest_ProtoLogEntry) ProtoMessage() {}
func (m *ProtoAppendEntriesRequest_ProtoLogEntry) GetIndex() uint64 {
if m != nil && m.Index != nil {
return *m.Index
}
return 0
}
func (m *ProtoAppendEntriesRequest_ProtoLogEntry) GetTerm() uint64 {
if m != nil && m.Term != nil {
return *m.Term
}
return 0
}
func (m *ProtoAppendEntriesRequest_ProtoLogEntry) GetCommandName() string {
if m != nil && m.CommandName != nil {
return *m.CommandName
}
return ""
}
func (m *ProtoAppendEntriesRequest_ProtoLogEntry) GetCommand() []byte {
if m != nil {
return m.Command
func (m *AppendEntriesRequest) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io1.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io1.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.Term = &v
case 2:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io1.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.PrevLogIndex = &v
case 3:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io1.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.PrevLogTerm = &v
case 4:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io1.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.CommitIndex = &v
case 5:
if wireType != 2 {
return proto.ErrWrongType
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io1.ErrUnexpectedEOF
}
b := data[index]
index++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + int(stringLen)
if postIndex > l {
return io1.ErrUnexpectedEOF
}
s := string(data[index:postIndex])
m.LeaderName = &s
index = postIndex
case 6:
if wireType != 2 {
return proto.ErrWrongType
}
var msglen int
for shift := uint(0); ; shift += 7 {
if index >= l {
return io1.ErrUnexpectedEOF
}
b := data[index]
index++
msglen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + msglen
if postIndex > l {
return io1.ErrUnexpectedEOF
}
m.Entries = append(m.Entries, &LogEntry{})
m.Entries[len(m.Entries)-1].Unmarshal(data[index:postIndex])
index = postIndex
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto2.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func init() {
func (this *AppendEntriesRequest) String() string {
if this == nil {
return "nil"
}
s := strings2.Join([]string{`&AppendEntriesRequest{`,
`Term:` + valueToStringAppendEntriesRequest(this.Term) + `,`,
`PrevLogIndex:` + valueToStringAppendEntriesRequest(this.PrevLogIndex) + `,`,
`PrevLogTerm:` + valueToStringAppendEntriesRequest(this.PrevLogTerm) + `,`,
`CommitIndex:` + valueToStringAppendEntriesRequest(this.CommitIndex) + `,`,
`LeaderName:` + valueToStringAppendEntriesRequest(this.LeaderName) + `,`,
`Entries:` + strings2.Replace(fmt3.Sprintf("%v", this.Entries), "LogEntry", "LogEntry", 1) + `,`,
`XXX_unrecognized:` + fmt3.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func valueToStringAppendEntriesRequest(v interface{}) string {
rv := reflect2.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect2.Indirect(rv).Interface()
return fmt3.Sprintf("*%v", pv)
}
func (m *AppendEntriesRequest) Size() (n int) {
var l int
_ = l
if m.Term != nil {
n += 1 + sovAppendEntriesRequest(uint64(*m.Term))
}
if m.PrevLogIndex != nil {
n += 1 + sovAppendEntriesRequest(uint64(*m.PrevLogIndex))
}
if m.PrevLogTerm != nil {
n += 1 + sovAppendEntriesRequest(uint64(*m.PrevLogTerm))
}
if m.CommitIndex != nil {
n += 1 + sovAppendEntriesRequest(uint64(*m.CommitIndex))
}
if m.LeaderName != nil {
l = len(*m.LeaderName)
n += 1 + l + sovAppendEntriesRequest(uint64(l))
}
if len(m.Entries) > 0 {
for _, e := range m.Entries {
l = e.Size()
n += 1 + l + sovAppendEntriesRequest(uint64(l))
}
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func sovAppendEntriesRequest(x uint64) (n int) {
for {
n++
x >>= 7
if x == 0 {
break
}
}
return n
}
func sozAppendEntriesRequest(x uint64) (n int) {
return sovAppendEntriesRequest(uint64((x << 1) ^ uint64((int64(x) >> 63))))
return sovAppendEntriesRequest(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func NewPopulatedAppendEntriesRequest(r randyAppendEntriesRequest, easy bool) *AppendEntriesRequest {
this := &AppendEntriesRequest{}
v1 := uint64(r.Uint32())
this.Term = &v1
v2 := uint64(r.Uint32())
this.PrevLogIndex = &v2
v3 := uint64(r.Uint32())
this.PrevLogTerm = &v3
v4 := uint64(r.Uint32())
this.CommitIndex = &v4
v5 := randStringAppendEntriesRequest(r)
this.LeaderName = &v5
if r.Intn(10) != 0 {
v6 := r.Intn(10)
this.Entries = make([]*LogEntry, v6)
for i := 0; i < v6; i++ {
this.Entries[i] = NewPopulatedLogEntry(r, easy)
}
}
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedAppendEntriesRequest(r, 7)
}
return this
}
type randyAppendEntriesRequest interface {
Float32() float32
Float64() float64
Int63() int64
Int31() int32
Uint32() uint32
Intn(n int) int
}
func randUTF8RuneAppendEntriesRequest(r randyAppendEntriesRequest) rune {
res := rune(r.Uint32() % 1112064)
if 55296 <= res {
res += 2047
}
return res
}
func randStringAppendEntriesRequest(r randyAppendEntriesRequest) string {
v7 := r.Intn(100)
tmps := make([]rune, v7)
for i := 0; i < v7; i++ {
tmps[i] = randUTF8RuneAppendEntriesRequest(r)
}
return string(tmps)
}
func randUnrecognizedAppendEntriesRequest(r randyAppendEntriesRequest, maxFieldNumber int) (data []byte) {
l := r.Intn(5)
for i := 0; i < l; i++ {
wire := r.Intn(4)
if wire == 3 {
wire = 5
}
fieldNumber := maxFieldNumber + r.Intn(100)
data = randFieldAppendEntriesRequest(data, r, fieldNumber, wire)
}
return data
}
func randFieldAppendEntriesRequest(data []byte, r randyAppendEntriesRequest, fieldNumber int, wire int) []byte {
key := uint32(fieldNumber)<<3 | uint32(wire)
switch wire {
case 0:
data = encodeVarintPopulateAppendEntriesRequest(data, uint64(key))
data = encodeVarintPopulateAppendEntriesRequest(data, uint64(r.Int63()))
case 1:
data = encodeVarintPopulateAppendEntriesRequest(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
case 2:
data = encodeVarintPopulateAppendEntriesRequest(data, uint64(key))
ll := r.Intn(100)
data = encodeVarintPopulateAppendEntriesRequest(data, uint64(ll))
for j := 0; j < ll; j++ {
data = append(data, byte(r.Intn(256)))
}
default:
data = encodeVarintPopulateAppendEntriesRequest(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
}
return data
}
func encodeVarintPopulateAppendEntriesRequest(data []byte, v uint64) []byte {
for v >= 1<<7 {
data = append(data, uint8(uint64(v)&0x7f|0x80))
v >>= 7
}
data = append(data, uint8(v))
return data
}
func (m *AppendEntriesRequest) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *AppendEntriesRequest) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.Term != nil {
data[i] = 0x8
i++
i = encodeVarintAppendEntriesRequest(data, i, uint64(*m.Term))
}
if m.PrevLogIndex != nil {
data[i] = 0x10
i++
i = encodeVarintAppendEntriesRequest(data, i, uint64(*m.PrevLogIndex))
}
if m.PrevLogTerm != nil {
data[i] = 0x18
i++
i = encodeVarintAppendEntriesRequest(data, i, uint64(*m.PrevLogTerm))
}
if m.CommitIndex != nil {
data[i] = 0x20
i++
i = encodeVarintAppendEntriesRequest(data, i, uint64(*m.CommitIndex))
}
if m.LeaderName != nil {
data[i] = 0x2a
i++
i = encodeVarintAppendEntriesRequest(data, i, uint64(len(*m.LeaderName)))
i += copy(data[i:], *m.LeaderName)
}
if len(m.Entries) > 0 {
for _, msg := range m.Entries {
data[i] = 0x32
i++
i = encodeVarintAppendEntriesRequest(data, i, uint64(msg.Size()))
n, err := msg.MarshalTo(data[i:])
if err != nil {
return 0, err
}
i += n
}
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func encodeFixed64AppendEntriesRequest(data []byte, offset int, v uint64) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
data[offset+4] = uint8(v >> 32)
data[offset+5] = uint8(v >> 40)
data[offset+6] = uint8(v >> 48)
data[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32AppendEntriesRequest(data []byte, offset int, v uint32) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintAppendEntriesRequest(data []byte, offset int, v uint64) int {
for v >= 1<<7 {
data[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
data[offset] = uint8(v)
return offset + 1
}
func (this *AppendEntriesRequest) GoString() string {
if this == nil {
return "nil"
}
s := strings3.Join([]string{`&protobuf.AppendEntriesRequest{` + `Term:` + valueToGoStringAppendEntriesRequest(this.Term, "uint64"), `PrevLogIndex:` + valueToGoStringAppendEntriesRequest(this.PrevLogIndex, "uint64"), `PrevLogTerm:` + valueToGoStringAppendEntriesRequest(this.PrevLogTerm, "uint64"), `CommitIndex:` + valueToGoStringAppendEntriesRequest(this.CommitIndex, "uint64"), `LeaderName:` + valueToGoStringAppendEntriesRequest(this.LeaderName, "string"), `Entries:` + fmt4.Sprintf("%#v", this.Entries), `XXX_unrecognized:` + fmt4.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func valueToGoStringAppendEntriesRequest(v interface{}, typ string) string {
rv := reflect3.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect3.Indirect(rv).Interface()
return fmt4.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
}
func extensionToGoStringAppendEntriesRequest(e map[int32]code_google_com_p_gogoprotobuf_proto3.Extension) string {
if e == nil {
return "nil"
}
s := "map[int32]proto.Extension{"
keys := make([]int, 0, len(e))
for k := range e {
keys = append(keys, int(k))
}
sort1.Ints(keys)
ss := []string{}
for _, k := range keys {
ss = append(ss, strconv1.Itoa(k)+": "+e[int32(k)].GoString())
}
s += strings3.Join(ss, ",") + "}"
return s
}
func (this *AppendEntriesRequest) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt5.Errorf("that == nil && this != nil")
}
that1, ok := that.(*AppendEntriesRequest)
if !ok {
return fmt5.Errorf("that is not of type *AppendEntriesRequest")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt5.Errorf("that is type *AppendEntriesRequest but is nil && this != nil")
} else if this == nil {
return fmt5.Errorf("that is type *AppendEntriesRequestbut is not nil && this == nil")
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return fmt5.Errorf("Term this(%v) Not Equal that(%v)", *this.Term, *that1.Term)
}
} else if this.Term != nil {
return fmt5.Errorf("this.Term == nil && that.Term != nil")
} else if that1.Term != nil {
return fmt5.Errorf("Term this(%v) Not Equal that(%v)", this.Term, that1.Term)
}
if this.PrevLogIndex != nil && that1.PrevLogIndex != nil {
if *this.PrevLogIndex != *that1.PrevLogIndex {
return fmt5.Errorf("PrevLogIndex this(%v) Not Equal that(%v)", *this.PrevLogIndex, *that1.PrevLogIndex)
}
} else if this.PrevLogIndex != nil {
return fmt5.Errorf("this.PrevLogIndex == nil && that.PrevLogIndex != nil")
} else if that1.PrevLogIndex != nil {
return fmt5.Errorf("PrevLogIndex this(%v) Not Equal that(%v)", this.PrevLogIndex, that1.PrevLogIndex)
}
if this.PrevLogTerm != nil && that1.PrevLogTerm != nil {
if *this.PrevLogTerm != *that1.PrevLogTerm {
return fmt5.Errorf("PrevLogTerm this(%v) Not Equal that(%v)", *this.PrevLogTerm, *that1.PrevLogTerm)
}
} else if this.PrevLogTerm != nil {
return fmt5.Errorf("this.PrevLogTerm == nil && that.PrevLogTerm != nil")
} else if that1.PrevLogTerm != nil {
return fmt5.Errorf("PrevLogTerm this(%v) Not Equal that(%v)", this.PrevLogTerm, that1.PrevLogTerm)
}
if this.CommitIndex != nil && that1.CommitIndex != nil {
if *this.CommitIndex != *that1.CommitIndex {
return fmt5.Errorf("CommitIndex this(%v) Not Equal that(%v)", *this.CommitIndex, *that1.CommitIndex)
}
} else if this.CommitIndex != nil {
return fmt5.Errorf("this.CommitIndex == nil && that.CommitIndex != nil")
} else if that1.CommitIndex != nil {
return fmt5.Errorf("CommitIndex this(%v) Not Equal that(%v)", this.CommitIndex, that1.CommitIndex)
}
if this.LeaderName != nil && that1.LeaderName != nil {
if *this.LeaderName != *that1.LeaderName {
return fmt5.Errorf("LeaderName this(%v) Not Equal that(%v)", *this.LeaderName, *that1.LeaderName)
}
} else if this.LeaderName != nil {
return fmt5.Errorf("this.LeaderName == nil && that.LeaderName != nil")
} else if that1.LeaderName != nil {
return fmt5.Errorf("LeaderName this(%v) Not Equal that(%v)", this.LeaderName, that1.LeaderName)
}
if len(this.Entries) != len(that1.Entries) {
return fmt5.Errorf("Entries this(%v) Not Equal that(%v)", len(this.Entries), len(that1.Entries))
}
for i := range this.Entries {
if !this.Entries[i].Equal(that1.Entries[i]) {
return fmt5.Errorf("Entries this[%v](%v) Not Equal that[%v](%v)", i, this.Entries[i], i, that1.Entries[i])
}
}
if !bytes1.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt5.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *AppendEntriesRequest) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*AppendEntriesRequest)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return false
}
} else if this.Term != nil {
return false
} else if that1.Term != nil {
return false
}
if this.PrevLogIndex != nil && that1.PrevLogIndex != nil {
if *this.PrevLogIndex != *that1.PrevLogIndex {
return false
}
} else if this.PrevLogIndex != nil {
return false
} else if that1.PrevLogIndex != nil {
return false
}
if this.PrevLogTerm != nil && that1.PrevLogTerm != nil {
if *this.PrevLogTerm != *that1.PrevLogTerm {
return false
}
} else if this.PrevLogTerm != nil {
return false
} else if that1.PrevLogTerm != nil {
return false
}
if this.CommitIndex != nil && that1.CommitIndex != nil {
if *this.CommitIndex != *that1.CommitIndex {
return false
}
} else if this.CommitIndex != nil {
return false
} else if that1.CommitIndex != nil {
return false
}
if this.LeaderName != nil && that1.LeaderName != nil {
if *this.LeaderName != *that1.LeaderName {
return false
}
} else if this.LeaderName != nil {
return false
} else if that1.LeaderName != nil {
return false
}
if len(this.Entries) != len(that1.Entries) {
return false
}
for i := range this.Entries {
if !this.Entries[i].Equal(that1.Entries[i]) {
return false
}
}
if !bytes1.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}

View File

@ -1,18 +1,25 @@
package protobuf;
message ProtoAppendEntriesRequest {
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
import "log_entry.proto";
option (gogoproto.gostring_all) = true;
option (gogoproto.equal_all) = true;
option (gogoproto.verbose_equal_all) = true;
option (gogoproto.goproto_stringer_all) = false;
option (gogoproto.stringer_all) = true;
option (gogoproto.populate_all) = true;
option (gogoproto.testgen_all) = true;
option (gogoproto.benchgen_all) = true;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
message AppendEntriesRequest {
required uint64 Term=1;
required uint64 PrevLogIndex=2;
required uint64 PrevLogTerm=3;
required uint64 CommitIndex=4;
required string LeaderName=5;
message ProtoLogEntry {
required uint64 Index=1;
required uint64 Term=2;
required string CommandName=3;
optional bytes Command=4;
}
repeated ProtoLogEntry Entries=6;
repeated LogEntry Entries=6;
}

View File

@ -1,19 +1,38 @@
// Code generated by protoc-gen-go.
// Code generated by protoc-gen-gogo.
// source: append_entries_responses.proto
// DO NOT EDIT!
package protobuf
import proto "code.google.com/p/goprotobuf/proto"
import proto "code.google.com/p/gogoprotobuf/proto"
import json "encoding/json"
import math "math"
// discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb"
import io2 "io"
import code_google_com_p_gogoprotobuf_proto4 "code.google.com/p/gogoprotobuf/proto"
import fmt6 "fmt"
import strings4 "strings"
import reflect4 "reflect"
import fmt7 "fmt"
import strings5 "strings"
import code_google_com_p_gogoprotobuf_proto5 "code.google.com/p/gogoprotobuf/proto"
import sort2 "sort"
import strconv2 "strconv"
import reflect5 "reflect"
import fmt8 "fmt"
import bytes2 "bytes"
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
var _ = proto.Marshal
var _ = &json.SyntaxError{}
var _ = math.Inf
type ProtoAppendEntriesResponse struct {
type AppendEntriesResponse struct {
Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"`
Index *uint64 `protobuf:"varint,2,req" json:"Index,omitempty"`
CommitIndex *uint64 `protobuf:"varint,3,req" json:"CommitIndex,omitempty"`
@ -21,32 +40,31 @@ type ProtoAppendEntriesResponse struct {
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoAppendEntriesResponse) Reset() { *m = ProtoAppendEntriesResponse{} }
func (m *ProtoAppendEntriesResponse) String() string { return proto.CompactTextString(m) }
func (*ProtoAppendEntriesResponse) ProtoMessage() {}
func (m *AppendEntriesResponse) Reset() { *m = AppendEntriesResponse{} }
func (*AppendEntriesResponse) ProtoMessage() {}
func (m *ProtoAppendEntriesResponse) GetTerm() uint64 {
func (m *AppendEntriesResponse) GetTerm() uint64 {
if m != nil && m.Term != nil {
return *m.Term
}
return 0
}
func (m *ProtoAppendEntriesResponse) GetIndex() uint64 {
func (m *AppendEntriesResponse) GetIndex() uint64 {
if m != nil && m.Index != nil {
return *m.Index
}
return 0
}
func (m *ProtoAppendEntriesResponse) GetCommitIndex() uint64 {
func (m *AppendEntriesResponse) GetCommitIndex() uint64 {
if m != nil && m.CommitIndex != nil {
return *m.CommitIndex
}
return 0
}
func (m *ProtoAppendEntriesResponse) GetSuccess() bool {
func (m *AppendEntriesResponse) GetSuccess() bool {
if m != nil && m.Success != nil {
return *m.Success
}
@ -55,3 +73,476 @@ func (m *ProtoAppendEntriesResponse) GetSuccess() bool {
func init() {
}
func (m *AppendEntriesResponse) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io2.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io2.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.Term = &v
case 2:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io2.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.Index = &v
case 3:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io2.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.CommitIndex = &v
case 4:
if wireType != 0 {
return proto.ErrWrongType
}
var v int
for shift := uint(0); ; shift += 7 {
if index >= l {
return io2.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
b := bool(v != 0)
m.Success = &b
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto4.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func (this *AppendEntriesResponse) String() string {
if this == nil {
return "nil"
}
s := strings4.Join([]string{`&AppendEntriesResponse{`,
`Term:` + valueToStringAppendEntriesResponses(this.Term) + `,`,
`Index:` + valueToStringAppendEntriesResponses(this.Index) + `,`,
`CommitIndex:` + valueToStringAppendEntriesResponses(this.CommitIndex) + `,`,
`Success:` + valueToStringAppendEntriesResponses(this.Success) + `,`,
`XXX_unrecognized:` + fmt6.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func valueToStringAppendEntriesResponses(v interface{}) string {
rv := reflect4.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect4.Indirect(rv).Interface()
return fmt6.Sprintf("*%v", pv)
}
func (m *AppendEntriesResponse) Size() (n int) {
var l int
_ = l
if m.Term != nil {
n += 1 + sovAppendEntriesResponses(uint64(*m.Term))
}
if m.Index != nil {
n += 1 + sovAppendEntriesResponses(uint64(*m.Index))
}
if m.CommitIndex != nil {
n += 1 + sovAppendEntriesResponses(uint64(*m.CommitIndex))
}
if m.Success != nil {
n += 2
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func sovAppendEntriesResponses(x uint64) (n int) {
for {
n++
x >>= 7
if x == 0 {
break
}
}
return n
}
func sozAppendEntriesResponses(x uint64) (n int) {
return sovAppendEntriesResponses(uint64((x << 1) ^ uint64((int64(x) >> 63))))
return sovAppendEntriesResponses(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func NewPopulatedAppendEntriesResponse(r randyAppendEntriesResponses, easy bool) *AppendEntriesResponse {
this := &AppendEntriesResponse{}
v1 := uint64(r.Uint32())
this.Term = &v1
v2 := uint64(r.Uint32())
this.Index = &v2
v3 := uint64(r.Uint32())
this.CommitIndex = &v3
v4 := bool(r.Intn(2) == 0)
this.Success = &v4
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedAppendEntriesResponses(r, 5)
}
return this
}
type randyAppendEntriesResponses interface {
Float32() float32
Float64() float64
Int63() int64
Int31() int32
Uint32() uint32
Intn(n int) int
}
func randUTF8RuneAppendEntriesResponses(r randyAppendEntriesResponses) rune {
res := rune(r.Uint32() % 1112064)
if 55296 <= res {
res += 2047
}
return res
}
func randStringAppendEntriesResponses(r randyAppendEntriesResponses) string {
v5 := r.Intn(100)
tmps := make([]rune, v5)
for i := 0; i < v5; i++ {
tmps[i] = randUTF8RuneAppendEntriesResponses(r)
}
return string(tmps)
}
func randUnrecognizedAppendEntriesResponses(r randyAppendEntriesResponses, maxFieldNumber int) (data []byte) {
l := r.Intn(5)
for i := 0; i < l; i++ {
wire := r.Intn(4)
if wire == 3 {
wire = 5
}
fieldNumber := maxFieldNumber + r.Intn(100)
data = randFieldAppendEntriesResponses(data, r, fieldNumber, wire)
}
return data
}
func randFieldAppendEntriesResponses(data []byte, r randyAppendEntriesResponses, fieldNumber int, wire int) []byte {
key := uint32(fieldNumber)<<3 | uint32(wire)
switch wire {
case 0:
data = encodeVarintPopulateAppendEntriesResponses(data, uint64(key))
data = encodeVarintPopulateAppendEntriesResponses(data, uint64(r.Int63()))
case 1:
data = encodeVarintPopulateAppendEntriesResponses(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
case 2:
data = encodeVarintPopulateAppendEntriesResponses(data, uint64(key))
ll := r.Intn(100)
data = encodeVarintPopulateAppendEntriesResponses(data, uint64(ll))
for j := 0; j < ll; j++ {
data = append(data, byte(r.Intn(256)))
}
default:
data = encodeVarintPopulateAppendEntriesResponses(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
}
return data
}
func encodeVarintPopulateAppendEntriesResponses(data []byte, v uint64) []byte {
for v >= 1<<7 {
data = append(data, uint8(uint64(v)&0x7f|0x80))
v >>= 7
}
data = append(data, uint8(v))
return data
}
func (m *AppendEntriesResponse) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *AppendEntriesResponse) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.Term != nil {
data[i] = 0x8
i++
i = encodeVarintAppendEntriesResponses(data, i, uint64(*m.Term))
}
if m.Index != nil {
data[i] = 0x10
i++
i = encodeVarintAppendEntriesResponses(data, i, uint64(*m.Index))
}
if m.CommitIndex != nil {
data[i] = 0x18
i++
i = encodeVarintAppendEntriesResponses(data, i, uint64(*m.CommitIndex))
}
if m.Success != nil {
data[i] = 0x20
i++
if *m.Success {
data[i] = 1
} else {
data[i] = 0
}
i++
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func encodeFixed64AppendEntriesResponses(data []byte, offset int, v uint64) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
data[offset+4] = uint8(v >> 32)
data[offset+5] = uint8(v >> 40)
data[offset+6] = uint8(v >> 48)
data[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32AppendEntriesResponses(data []byte, offset int, v uint32) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintAppendEntriesResponses(data []byte, offset int, v uint64) int {
for v >= 1<<7 {
data[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
data[offset] = uint8(v)
return offset + 1
}
func (this *AppendEntriesResponse) GoString() string {
if this == nil {
return "nil"
}
s := strings5.Join([]string{`&protobuf.AppendEntriesResponse{` + `Term:` + valueToGoStringAppendEntriesResponses(this.Term, "uint64"), `Index:` + valueToGoStringAppendEntriesResponses(this.Index, "uint64"), `CommitIndex:` + valueToGoStringAppendEntriesResponses(this.CommitIndex, "uint64"), `Success:` + valueToGoStringAppendEntriesResponses(this.Success, "bool"), `XXX_unrecognized:` + fmt7.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func valueToGoStringAppendEntriesResponses(v interface{}, typ string) string {
rv := reflect5.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect5.Indirect(rv).Interface()
return fmt7.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
}
func extensionToGoStringAppendEntriesResponses(e map[int32]code_google_com_p_gogoprotobuf_proto5.Extension) string {
if e == nil {
return "nil"
}
s := "map[int32]proto.Extension{"
keys := make([]int, 0, len(e))
for k := range e {
keys = append(keys, int(k))
}
sort2.Ints(keys)
ss := []string{}
for _, k := range keys {
ss = append(ss, strconv2.Itoa(k)+": "+e[int32(k)].GoString())
}
s += strings5.Join(ss, ",") + "}"
return s
}
func (this *AppendEntriesResponse) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt8.Errorf("that == nil && this != nil")
}
that1, ok := that.(*AppendEntriesResponse)
if !ok {
return fmt8.Errorf("that is not of type *AppendEntriesResponse")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt8.Errorf("that is type *AppendEntriesResponse but is nil && this != nil")
} else if this == nil {
return fmt8.Errorf("that is type *AppendEntriesResponsebut is not nil && this == nil")
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return fmt8.Errorf("Term this(%v) Not Equal that(%v)", *this.Term, *that1.Term)
}
} else if this.Term != nil {
return fmt8.Errorf("this.Term == nil && that.Term != nil")
} else if that1.Term != nil {
return fmt8.Errorf("Term this(%v) Not Equal that(%v)", this.Term, that1.Term)
}
if this.Index != nil && that1.Index != nil {
if *this.Index != *that1.Index {
return fmt8.Errorf("Index this(%v) Not Equal that(%v)", *this.Index, *that1.Index)
}
} else if this.Index != nil {
return fmt8.Errorf("this.Index == nil && that.Index != nil")
} else if that1.Index != nil {
return fmt8.Errorf("Index this(%v) Not Equal that(%v)", this.Index, that1.Index)
}
if this.CommitIndex != nil && that1.CommitIndex != nil {
if *this.CommitIndex != *that1.CommitIndex {
return fmt8.Errorf("CommitIndex this(%v) Not Equal that(%v)", *this.CommitIndex, *that1.CommitIndex)
}
} else if this.CommitIndex != nil {
return fmt8.Errorf("this.CommitIndex == nil && that.CommitIndex != nil")
} else if that1.CommitIndex != nil {
return fmt8.Errorf("CommitIndex this(%v) Not Equal that(%v)", this.CommitIndex, that1.CommitIndex)
}
if this.Success != nil && that1.Success != nil {
if *this.Success != *that1.Success {
return fmt8.Errorf("Success this(%v) Not Equal that(%v)", *this.Success, *that1.Success)
}
} else if this.Success != nil {
return fmt8.Errorf("this.Success == nil && that.Success != nil")
} else if that1.Success != nil {
return fmt8.Errorf("Success this(%v) Not Equal that(%v)", this.Success, that1.Success)
}
if !bytes2.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt8.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *AppendEntriesResponse) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*AppendEntriesResponse)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return false
}
} else if this.Term != nil {
return false
} else if that1.Term != nil {
return false
}
if this.Index != nil && that1.Index != nil {
if *this.Index != *that1.Index {
return false
}
} else if this.Index != nil {
return false
} else if that1.Index != nil {
return false
}
if this.CommitIndex != nil && that1.CommitIndex != nil {
if *this.CommitIndex != *that1.CommitIndex {
return false
}
} else if this.CommitIndex != nil {
return false
} else if that1.CommitIndex != nil {
return false
}
if this.Success != nil && that1.Success != nil {
if *this.Success != *that1.Success {
return false
}
} else if this.Success != nil {
return false
} else if that1.Success != nil {
return false
}
if !bytes2.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}

View File

@ -1,6 +1,20 @@
package protobuf;
message ProtoAppendEntriesResponse {
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
option (gogoproto.gostring_all) = true;
option (gogoproto.equal_all) = true;
option (gogoproto.verbose_equal_all) = true;
option (gogoproto.goproto_stringer_all) = false;
option (gogoproto.stringer_all) = true;
option (gogoproto.populate_all) = true;
option (gogoproto.testgen_all) = true;
option (gogoproto.benchgen_all) = true;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
message AppendEntriesResponse {
required uint64 Term=1;
required uint64 Index=2;
required uint64 CommitIndex=3;

View File

@ -1,19 +1,38 @@
// Code generated by protoc-gen-go.
// Code generated by protoc-gen-gogo.
// source: log_entry.proto
// DO NOT EDIT!
package protobuf
import proto "code.google.com/p/goprotobuf/proto"
import proto "code.google.com/p/gogoprotobuf/proto"
import json "encoding/json"
import math "math"
// discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb"
import io "io"
import code_google_com_p_gogoprotobuf_proto "code.google.com/p/gogoprotobuf/proto"
import fmt "fmt"
import strings "strings"
import reflect "reflect"
import fmt1 "fmt"
import strings1 "strings"
import code_google_com_p_gogoprotobuf_proto1 "code.google.com/p/gogoprotobuf/proto"
import sort "sort"
import strconv "strconv"
import reflect1 "reflect"
import fmt2 "fmt"
import bytes "bytes"
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
var _ = proto.Marshal
var _ = &json.SyntaxError{}
var _ = math.Inf
type ProtoLogEntry struct {
type LogEntry struct {
Index *uint64 `protobuf:"varint,1,req" json:"Index,omitempty"`
Term *uint64 `protobuf:"varint,2,req" json:"Term,omitempty"`
CommandName *string `protobuf:"bytes,3,req" json:"CommandName,omitempty"`
@ -21,32 +40,31 @@ type ProtoLogEntry struct {
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoLogEntry) Reset() { *m = ProtoLogEntry{} }
func (m *ProtoLogEntry) String() string { return proto.CompactTextString(m) }
func (*ProtoLogEntry) ProtoMessage() {}
func (m *LogEntry) Reset() { *m = LogEntry{} }
func (*LogEntry) ProtoMessage() {}
func (m *ProtoLogEntry) GetIndex() uint64 {
func (m *LogEntry) GetIndex() uint64 {
if m != nil && m.Index != nil {
return *m.Index
}
return 0
}
func (m *ProtoLogEntry) GetTerm() uint64 {
func (m *LogEntry) GetTerm() uint64 {
if m != nil && m.Term != nil {
return *m.Term
}
return 0
}
func (m *ProtoLogEntry) GetCommandName() string {
func (m *LogEntry) GetCommandName() string {
if m != nil && m.CommandName != nil {
return *m.CommandName
}
return ""
}
func (m *ProtoLogEntry) GetCommand() []byte {
func (m *LogEntry) GetCommand() []byte {
if m != nil {
return m.Command
}
@ -55,3 +73,478 @@ func (m *ProtoLogEntry) GetCommand() []byte {
func init() {
}
func (m *LogEntry) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.Index = &v
case 2:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.Term = &v
case 3:
if wireType != 2 {
return proto.ErrWrongType
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io.ErrUnexpectedEOF
}
b := data[index]
index++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + int(stringLen)
if postIndex > l {
return io.ErrUnexpectedEOF
}
s := string(data[index:postIndex])
m.CommandName = &s
index = postIndex
case 4:
if wireType != 2 {
return proto.ErrWrongType
}
var byteLen int
for shift := uint(0); ; shift += 7 {
if index >= l {
return io.ErrUnexpectedEOF
}
b := data[index]
index++
byteLen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + byteLen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Command = append(m.Command, data[index:postIndex]...)
index = postIndex
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func (this *LogEntry) String() string {
if this == nil {
return "nil"
}
s := strings.Join([]string{`&LogEntry{`,
`Index:` + valueToStringLogEntry(this.Index) + `,`,
`Term:` + valueToStringLogEntry(this.Term) + `,`,
`CommandName:` + valueToStringLogEntry(this.CommandName) + `,`,
`Command:` + valueToStringLogEntry(this.Command) + `,`,
`XXX_unrecognized:` + fmt.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func valueToStringLogEntry(v interface{}) string {
rv := reflect.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect.Indirect(rv).Interface()
return fmt.Sprintf("*%v", pv)
}
func (m *LogEntry) Size() (n int) {
var l int
_ = l
if m.Index != nil {
n += 1 + sovLogEntry(uint64(*m.Index))
}
if m.Term != nil {
n += 1 + sovLogEntry(uint64(*m.Term))
}
if m.CommandName != nil {
l = len(*m.CommandName)
n += 1 + l + sovLogEntry(uint64(l))
}
if m.Command != nil {
l = len(m.Command)
n += 1 + l + sovLogEntry(uint64(l))
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func sovLogEntry(x uint64) (n int) {
for {
n++
x >>= 7
if x == 0 {
break
}
}
return n
}
func sozLogEntry(x uint64) (n int) {
return sovLogEntry(uint64((x << 1) ^ uint64((int64(x) >> 63))))
return sovLogEntry(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func NewPopulatedLogEntry(r randyLogEntry, easy bool) *LogEntry {
this := &LogEntry{}
v1 := uint64(r.Uint32())
this.Index = &v1
v2 := uint64(r.Uint32())
this.Term = &v2
v3 := randStringLogEntry(r)
this.CommandName = &v3
if r.Intn(10) != 0 {
v4 := r.Intn(100)
this.Command = make([]byte, v4)
for i := 0; i < v4; i++ {
this.Command[i] = byte(r.Intn(256))
}
}
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedLogEntry(r, 5)
}
return this
}
type randyLogEntry interface {
Float32() float32
Float64() float64
Int63() int64
Int31() int32
Uint32() uint32
Intn(n int) int
}
func randUTF8RuneLogEntry(r randyLogEntry) rune {
res := rune(r.Uint32() % 1112064)
if 55296 <= res {
res += 2047
}
return res
}
func randStringLogEntry(r randyLogEntry) string {
v5 := r.Intn(100)
tmps := make([]rune, v5)
for i := 0; i < v5; i++ {
tmps[i] = randUTF8RuneLogEntry(r)
}
return string(tmps)
}
func randUnrecognizedLogEntry(r randyLogEntry, maxFieldNumber int) (data []byte) {
l := r.Intn(5)
for i := 0; i < l; i++ {
wire := r.Intn(4)
if wire == 3 {
wire = 5
}
fieldNumber := maxFieldNumber + r.Intn(100)
data = randFieldLogEntry(data, r, fieldNumber, wire)
}
return data
}
func randFieldLogEntry(data []byte, r randyLogEntry, fieldNumber int, wire int) []byte {
key := uint32(fieldNumber)<<3 | uint32(wire)
switch wire {
case 0:
data = encodeVarintPopulateLogEntry(data, uint64(key))
data = encodeVarintPopulateLogEntry(data, uint64(r.Int63()))
case 1:
data = encodeVarintPopulateLogEntry(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
case 2:
data = encodeVarintPopulateLogEntry(data, uint64(key))
ll := r.Intn(100)
data = encodeVarintPopulateLogEntry(data, uint64(ll))
for j := 0; j < ll; j++ {
data = append(data, byte(r.Intn(256)))
}
default:
data = encodeVarintPopulateLogEntry(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
}
return data
}
func encodeVarintPopulateLogEntry(data []byte, v uint64) []byte {
for v >= 1<<7 {
data = append(data, uint8(uint64(v)&0x7f|0x80))
v >>= 7
}
data = append(data, uint8(v))
return data
}
func (m *LogEntry) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *LogEntry) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.Index != nil {
data[i] = 0x8
i++
i = encodeVarintLogEntry(data, i, uint64(*m.Index))
}
if m.Term != nil {
data[i] = 0x10
i++
i = encodeVarintLogEntry(data, i, uint64(*m.Term))
}
if m.CommandName != nil {
data[i] = 0x1a
i++
i = encodeVarintLogEntry(data, i, uint64(len(*m.CommandName)))
i += copy(data[i:], *m.CommandName)
}
if m.Command != nil {
data[i] = 0x22
i++
i = encodeVarintLogEntry(data, i, uint64(len(m.Command)))
i += copy(data[i:], m.Command)
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func encodeFixed64LogEntry(data []byte, offset int, v uint64) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
data[offset+4] = uint8(v >> 32)
data[offset+5] = uint8(v >> 40)
data[offset+6] = uint8(v >> 48)
data[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32LogEntry(data []byte, offset int, v uint32) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintLogEntry(data []byte, offset int, v uint64) int {
for v >= 1<<7 {
data[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
data[offset] = uint8(v)
return offset + 1
}
func (this *LogEntry) GoString() string {
if this == nil {
return "nil"
}
s := strings1.Join([]string{`&protobuf.LogEntry{` + `Index:` + valueToGoStringLogEntry(this.Index, "uint64"), `Term:` + valueToGoStringLogEntry(this.Term, "uint64"), `CommandName:` + valueToGoStringLogEntry(this.CommandName, "string"), `Command:` + valueToGoStringLogEntry(this.Command, "byte"), `XXX_unrecognized:` + fmt1.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func valueToGoStringLogEntry(v interface{}, typ string) string {
rv := reflect1.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect1.Indirect(rv).Interface()
return fmt1.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
}
func extensionToGoStringLogEntry(e map[int32]code_google_com_p_gogoprotobuf_proto1.Extension) string {
if e == nil {
return "nil"
}
s := "map[int32]proto.Extension{"
keys := make([]int, 0, len(e))
for k := range e {
keys = append(keys, int(k))
}
sort.Ints(keys)
ss := []string{}
for _, k := range keys {
ss = append(ss, strconv.Itoa(k)+": "+e[int32(k)].GoString())
}
s += strings1.Join(ss, ",") + "}"
return s
}
func (this *LogEntry) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt2.Errorf("that == nil && this != nil")
}
that1, ok := that.(*LogEntry)
if !ok {
return fmt2.Errorf("that is not of type *LogEntry")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt2.Errorf("that is type *LogEntry but is nil && this != nil")
} else if this == nil {
return fmt2.Errorf("that is type *LogEntrybut is not nil && this == nil")
}
if this.Index != nil && that1.Index != nil {
if *this.Index != *that1.Index {
return fmt2.Errorf("Index this(%v) Not Equal that(%v)", *this.Index, *that1.Index)
}
} else if this.Index != nil {
return fmt2.Errorf("this.Index == nil && that.Index != nil")
} else if that1.Index != nil {
return fmt2.Errorf("Index this(%v) Not Equal that(%v)", this.Index, that1.Index)
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return fmt2.Errorf("Term this(%v) Not Equal that(%v)", *this.Term, *that1.Term)
}
} else if this.Term != nil {
return fmt2.Errorf("this.Term == nil && that.Term != nil")
} else if that1.Term != nil {
return fmt2.Errorf("Term this(%v) Not Equal that(%v)", this.Term, that1.Term)
}
if this.CommandName != nil && that1.CommandName != nil {
if *this.CommandName != *that1.CommandName {
return fmt2.Errorf("CommandName this(%v) Not Equal that(%v)", *this.CommandName, *that1.CommandName)
}
} else if this.CommandName != nil {
return fmt2.Errorf("this.CommandName == nil && that.CommandName != nil")
} else if that1.CommandName != nil {
return fmt2.Errorf("CommandName this(%v) Not Equal that(%v)", this.CommandName, that1.CommandName)
}
if !bytes.Equal(this.Command, that1.Command) {
return fmt2.Errorf("Command this(%v) Not Equal that(%v)", this.Command, that1.Command)
}
if !bytes.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt2.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *LogEntry) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*LogEntry)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.Index != nil && that1.Index != nil {
if *this.Index != *that1.Index {
return false
}
} else if this.Index != nil {
return false
} else if that1.Index != nil {
return false
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return false
}
} else if this.Term != nil {
return false
} else if that1.Term != nil {
return false
}
if this.CommandName != nil && that1.CommandName != nil {
if *this.CommandName != *that1.CommandName {
return false
}
} else if this.CommandName != nil {
return false
} else if that1.CommandName != nil {
return false
}
if !bytes.Equal(this.Command, that1.Command) {
return false
}
if !bytes.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}

View File

@ -1,6 +1,20 @@
package protobuf;
message ProtoLogEntry {
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
option (gogoproto.gostring_all) = true;
option (gogoproto.equal_all) = true;
option (gogoproto.verbose_equal_all) = true;
option (gogoproto.goproto_stringer_all) = false;
option (gogoproto.stringer_all) = true;
option (gogoproto.populate_all) = true;
option (gogoproto.testgen_all) = true;
option (gogoproto.benchgen_all) = true;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
message LogEntry {
required uint64 Index=1;
required uint64 Term=2;
required string CommandName=3;

View File

@ -1,19 +1,38 @@
// Code generated by protoc-gen-go.
// Code generated by protoc-gen-gogo.
// source: request_vote_request.proto
// DO NOT EDIT!
package protobuf
import proto "code.google.com/p/goprotobuf/proto"
import proto "code.google.com/p/gogoprotobuf/proto"
import json "encoding/json"
import math "math"
// discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb"
import io3 "io"
import code_google_com_p_gogoprotobuf_proto6 "code.google.com/p/gogoprotobuf/proto"
import fmt9 "fmt"
import strings6 "strings"
import reflect6 "reflect"
import fmt10 "fmt"
import strings7 "strings"
import code_google_com_p_gogoprotobuf_proto7 "code.google.com/p/gogoprotobuf/proto"
import sort3 "sort"
import strconv3 "strconv"
import reflect7 "reflect"
import fmt11 "fmt"
import bytes3 "bytes"
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
var _ = proto.Marshal
var _ = &json.SyntaxError{}
var _ = math.Inf
type ProtoRequestVoteRequest struct {
type RequestVoteRequest struct {
Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"`
LastLogIndex *uint64 `protobuf:"varint,2,req" json:"LastLogIndex,omitempty"`
LastLogTerm *uint64 `protobuf:"varint,3,req" json:"LastLogTerm,omitempty"`
@ -21,32 +40,31 @@ type ProtoRequestVoteRequest struct {
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoRequestVoteRequest) Reset() { *m = ProtoRequestVoteRequest{} }
func (m *ProtoRequestVoteRequest) String() string { return proto.CompactTextString(m) }
func (*ProtoRequestVoteRequest) ProtoMessage() {}
func (m *RequestVoteRequest) Reset() { *m = RequestVoteRequest{} }
func (*RequestVoteRequest) ProtoMessage() {}
func (m *ProtoRequestVoteRequest) GetTerm() uint64 {
func (m *RequestVoteRequest) GetTerm() uint64 {
if m != nil && m.Term != nil {
return *m.Term
}
return 0
}
func (m *ProtoRequestVoteRequest) GetLastLogIndex() uint64 {
func (m *RequestVoteRequest) GetLastLogIndex() uint64 {
if m != nil && m.LastLogIndex != nil {
return *m.LastLogIndex
}
return 0
}
func (m *ProtoRequestVoteRequest) GetLastLogTerm() uint64 {
func (m *RequestVoteRequest) GetLastLogTerm() uint64 {
if m != nil && m.LastLogTerm != nil {
return *m.LastLogTerm
}
return 0
}
func (m *ProtoRequestVoteRequest) GetCandidateName() string {
func (m *RequestVoteRequest) GetCandidateName() string {
if m != nil && m.CandidateName != nil {
return *m.CandidateName
}
@ -55,3 +73,478 @@ func (m *ProtoRequestVoteRequest) GetCandidateName() string {
func init() {
}
func (m *RequestVoteRequest) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io3.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io3.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.Term = &v
case 2:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io3.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.LastLogIndex = &v
case 3:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io3.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.LastLogTerm = &v
case 4:
if wireType != 2 {
return proto.ErrWrongType
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io3.ErrUnexpectedEOF
}
b := data[index]
index++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + int(stringLen)
if postIndex > l {
return io3.ErrUnexpectedEOF
}
s := string(data[index:postIndex])
m.CandidateName = &s
index = postIndex
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto6.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func (this *RequestVoteRequest) String() string {
if this == nil {
return "nil"
}
s := strings6.Join([]string{`&RequestVoteRequest{`,
`Term:` + valueToStringRequestVoteRequest(this.Term) + `,`,
`LastLogIndex:` + valueToStringRequestVoteRequest(this.LastLogIndex) + `,`,
`LastLogTerm:` + valueToStringRequestVoteRequest(this.LastLogTerm) + `,`,
`CandidateName:` + valueToStringRequestVoteRequest(this.CandidateName) + `,`,
`XXX_unrecognized:` + fmt9.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func valueToStringRequestVoteRequest(v interface{}) string {
rv := reflect6.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect6.Indirect(rv).Interface()
return fmt9.Sprintf("*%v", pv)
}
func (m *RequestVoteRequest) Size() (n int) {
var l int
_ = l
if m.Term != nil {
n += 1 + sovRequestVoteRequest(uint64(*m.Term))
}
if m.LastLogIndex != nil {
n += 1 + sovRequestVoteRequest(uint64(*m.LastLogIndex))
}
if m.LastLogTerm != nil {
n += 1 + sovRequestVoteRequest(uint64(*m.LastLogTerm))
}
if m.CandidateName != nil {
l = len(*m.CandidateName)
n += 1 + l + sovRequestVoteRequest(uint64(l))
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func sovRequestVoteRequest(x uint64) (n int) {
for {
n++
x >>= 7
if x == 0 {
break
}
}
return n
}
func sozRequestVoteRequest(x uint64) (n int) {
return sovRequestVoteRequest(uint64((x << 1) ^ uint64((int64(x) >> 63))))
return sovRequestVoteRequest(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func NewPopulatedRequestVoteRequest(r randyRequestVoteRequest, easy bool) *RequestVoteRequest {
this := &RequestVoteRequest{}
v1 := uint64(r.Uint32())
this.Term = &v1
v2 := uint64(r.Uint32())
this.LastLogIndex = &v2
v3 := uint64(r.Uint32())
this.LastLogTerm = &v3
v4 := randStringRequestVoteRequest(r)
this.CandidateName = &v4
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedRequestVoteRequest(r, 5)
}
return this
}
type randyRequestVoteRequest interface {
Float32() float32
Float64() float64
Int63() int64
Int31() int32
Uint32() uint32
Intn(n int) int
}
func randUTF8RuneRequestVoteRequest(r randyRequestVoteRequest) rune {
res := rune(r.Uint32() % 1112064)
if 55296 <= res {
res += 2047
}
return res
}
func randStringRequestVoteRequest(r randyRequestVoteRequest) string {
v5 := r.Intn(100)
tmps := make([]rune, v5)
for i := 0; i < v5; i++ {
tmps[i] = randUTF8RuneRequestVoteRequest(r)
}
return string(tmps)
}
func randUnrecognizedRequestVoteRequest(r randyRequestVoteRequest, maxFieldNumber int) (data []byte) {
l := r.Intn(5)
for i := 0; i < l; i++ {
wire := r.Intn(4)
if wire == 3 {
wire = 5
}
fieldNumber := maxFieldNumber + r.Intn(100)
data = randFieldRequestVoteRequest(data, r, fieldNumber, wire)
}
return data
}
func randFieldRequestVoteRequest(data []byte, r randyRequestVoteRequest, fieldNumber int, wire int) []byte {
key := uint32(fieldNumber)<<3 | uint32(wire)
switch wire {
case 0:
data = encodeVarintPopulateRequestVoteRequest(data, uint64(key))
data = encodeVarintPopulateRequestVoteRequest(data, uint64(r.Int63()))
case 1:
data = encodeVarintPopulateRequestVoteRequest(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
case 2:
data = encodeVarintPopulateRequestVoteRequest(data, uint64(key))
ll := r.Intn(100)
data = encodeVarintPopulateRequestVoteRequest(data, uint64(ll))
for j := 0; j < ll; j++ {
data = append(data, byte(r.Intn(256)))
}
default:
data = encodeVarintPopulateRequestVoteRequest(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
}
return data
}
func encodeVarintPopulateRequestVoteRequest(data []byte, v uint64) []byte {
for v >= 1<<7 {
data = append(data, uint8(uint64(v)&0x7f|0x80))
v >>= 7
}
data = append(data, uint8(v))
return data
}
func (m *RequestVoteRequest) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *RequestVoteRequest) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.Term != nil {
data[i] = 0x8
i++
i = encodeVarintRequestVoteRequest(data, i, uint64(*m.Term))
}
if m.LastLogIndex != nil {
data[i] = 0x10
i++
i = encodeVarintRequestVoteRequest(data, i, uint64(*m.LastLogIndex))
}
if m.LastLogTerm != nil {
data[i] = 0x18
i++
i = encodeVarintRequestVoteRequest(data, i, uint64(*m.LastLogTerm))
}
if m.CandidateName != nil {
data[i] = 0x22
i++
i = encodeVarintRequestVoteRequest(data, i, uint64(len(*m.CandidateName)))
i += copy(data[i:], *m.CandidateName)
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func encodeFixed64RequestVoteRequest(data []byte, offset int, v uint64) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
data[offset+4] = uint8(v >> 32)
data[offset+5] = uint8(v >> 40)
data[offset+6] = uint8(v >> 48)
data[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32RequestVoteRequest(data []byte, offset int, v uint32) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintRequestVoteRequest(data []byte, offset int, v uint64) int {
for v >= 1<<7 {
data[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
data[offset] = uint8(v)
return offset + 1
}
func (this *RequestVoteRequest) GoString() string {
if this == nil {
return "nil"
}
s := strings7.Join([]string{`&protobuf.RequestVoteRequest{` + `Term:` + valueToGoStringRequestVoteRequest(this.Term, "uint64"), `LastLogIndex:` + valueToGoStringRequestVoteRequest(this.LastLogIndex, "uint64"), `LastLogTerm:` + valueToGoStringRequestVoteRequest(this.LastLogTerm, "uint64"), `CandidateName:` + valueToGoStringRequestVoteRequest(this.CandidateName, "string"), `XXX_unrecognized:` + fmt10.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func valueToGoStringRequestVoteRequest(v interface{}, typ string) string {
rv := reflect7.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect7.Indirect(rv).Interface()
return fmt10.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
}
func extensionToGoStringRequestVoteRequest(e map[int32]code_google_com_p_gogoprotobuf_proto7.Extension) string {
if e == nil {
return "nil"
}
s := "map[int32]proto.Extension{"
keys := make([]int, 0, len(e))
for k := range e {
keys = append(keys, int(k))
}
sort3.Ints(keys)
ss := []string{}
for _, k := range keys {
ss = append(ss, strconv3.Itoa(k)+": "+e[int32(k)].GoString())
}
s += strings7.Join(ss, ",") + "}"
return s
}
func (this *RequestVoteRequest) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt11.Errorf("that == nil && this != nil")
}
that1, ok := that.(*RequestVoteRequest)
if !ok {
return fmt11.Errorf("that is not of type *RequestVoteRequest")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt11.Errorf("that is type *RequestVoteRequest but is nil && this != nil")
} else if this == nil {
return fmt11.Errorf("that is type *RequestVoteRequestbut is not nil && this == nil")
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return fmt11.Errorf("Term this(%v) Not Equal that(%v)", *this.Term, *that1.Term)
}
} else if this.Term != nil {
return fmt11.Errorf("this.Term == nil && that.Term != nil")
} else if that1.Term != nil {
return fmt11.Errorf("Term this(%v) Not Equal that(%v)", this.Term, that1.Term)
}
if this.LastLogIndex != nil && that1.LastLogIndex != nil {
if *this.LastLogIndex != *that1.LastLogIndex {
return fmt11.Errorf("LastLogIndex this(%v) Not Equal that(%v)", *this.LastLogIndex, *that1.LastLogIndex)
}
} else if this.LastLogIndex != nil {
return fmt11.Errorf("this.LastLogIndex == nil && that.LastLogIndex != nil")
} else if that1.LastLogIndex != nil {
return fmt11.Errorf("LastLogIndex this(%v) Not Equal that(%v)", this.LastLogIndex, that1.LastLogIndex)
}
if this.LastLogTerm != nil && that1.LastLogTerm != nil {
if *this.LastLogTerm != *that1.LastLogTerm {
return fmt11.Errorf("LastLogTerm this(%v) Not Equal that(%v)", *this.LastLogTerm, *that1.LastLogTerm)
}
} else if this.LastLogTerm != nil {
return fmt11.Errorf("this.LastLogTerm == nil && that.LastLogTerm != nil")
} else if that1.LastLogTerm != nil {
return fmt11.Errorf("LastLogTerm this(%v) Not Equal that(%v)", this.LastLogTerm, that1.LastLogTerm)
}
if this.CandidateName != nil && that1.CandidateName != nil {
if *this.CandidateName != *that1.CandidateName {
return fmt11.Errorf("CandidateName this(%v) Not Equal that(%v)", *this.CandidateName, *that1.CandidateName)
}
} else if this.CandidateName != nil {
return fmt11.Errorf("this.CandidateName == nil && that.CandidateName != nil")
} else if that1.CandidateName != nil {
return fmt11.Errorf("CandidateName this(%v) Not Equal that(%v)", this.CandidateName, that1.CandidateName)
}
if !bytes3.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt11.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *RequestVoteRequest) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*RequestVoteRequest)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return false
}
} else if this.Term != nil {
return false
} else if that1.Term != nil {
return false
}
if this.LastLogIndex != nil && that1.LastLogIndex != nil {
if *this.LastLogIndex != *that1.LastLogIndex {
return false
}
} else if this.LastLogIndex != nil {
return false
} else if that1.LastLogIndex != nil {
return false
}
if this.LastLogTerm != nil && that1.LastLogTerm != nil {
if *this.LastLogTerm != *that1.LastLogTerm {
return false
}
} else if this.LastLogTerm != nil {
return false
} else if that1.LastLogTerm != nil {
return false
}
if this.CandidateName != nil && that1.CandidateName != nil {
if *this.CandidateName != *that1.CandidateName {
return false
}
} else if this.CandidateName != nil {
return false
} else if that1.CandidateName != nil {
return false
}
if !bytes3.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}

View File

@ -1,6 +1,20 @@
package protobuf;
message ProtoRequestVoteRequest {
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
option (gogoproto.gostring_all) = true;
option (gogoproto.equal_all) = true;
option (gogoproto.verbose_equal_all) = true;
option (gogoproto.goproto_stringer_all) = false;
option (gogoproto.stringer_all) = true;
option (gogoproto.populate_all) = true;
option (gogoproto.testgen_all) = true;
option (gogoproto.benchgen_all) = true;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
message RequestVoteRequest {
required uint64 Term=1;
required uint64 LastLogIndex=2;
required uint64 LastLogTerm=3;

View File

@ -1,36 +1,54 @@
// Code generated by protoc-gen-go.
// Code generated by protoc-gen-gogo.
// source: request_vote_responses.proto
// DO NOT EDIT!
package protobuf
import proto "code.google.com/p/goprotobuf/proto"
import proto "code.google.com/p/gogoprotobuf/proto"
import json "encoding/json"
import math "math"
// discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb"
import io4 "io"
import code_google_com_p_gogoprotobuf_proto8 "code.google.com/p/gogoprotobuf/proto"
import fmt12 "fmt"
import strings8 "strings"
import reflect8 "reflect"
import fmt13 "fmt"
import strings9 "strings"
import code_google_com_p_gogoprotobuf_proto9 "code.google.com/p/gogoprotobuf/proto"
import sort4 "sort"
import strconv4 "strconv"
import reflect9 "reflect"
import fmt14 "fmt"
import bytes4 "bytes"
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
var _ = proto.Marshal
var _ = &json.SyntaxError{}
var _ = math.Inf
type ProtoRequestVoteResponse struct {
type RequestVoteResponse struct {
Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"`
VoteGranted *bool `protobuf:"varint,2,req" json:"VoteGranted,omitempty"`
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoRequestVoteResponse) Reset() { *m = ProtoRequestVoteResponse{} }
func (m *ProtoRequestVoteResponse) String() string { return proto.CompactTextString(m) }
func (*ProtoRequestVoteResponse) ProtoMessage() {}
func (m *RequestVoteResponse) Reset() { *m = RequestVoteResponse{} }
func (*RequestVoteResponse) ProtoMessage() {}
func (m *ProtoRequestVoteResponse) GetTerm() uint64 {
func (m *RequestVoteResponse) GetTerm() uint64 {
if m != nil && m.Term != nil {
return *m.Term
}
return 0
}
func (m *ProtoRequestVoteResponse) GetVoteGranted() bool {
func (m *RequestVoteResponse) GetVoteGranted() bool {
if m != nil && m.VoteGranted != nil {
return *m.VoteGranted
}
@ -39,3 +57,384 @@ func (m *ProtoRequestVoteResponse) GetVoteGranted() bool {
func init() {
}
func (m *RequestVoteResponse) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io4.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io4.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.Term = &v
case 2:
if wireType != 0 {
return proto.ErrWrongType
}
var v int
for shift := uint(0); ; shift += 7 {
if index >= l {
return io4.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
b := bool(v != 0)
m.VoteGranted = &b
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto8.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func (this *RequestVoteResponse) String() string {
if this == nil {
return "nil"
}
s := strings8.Join([]string{`&RequestVoteResponse{`,
`Term:` + valueToStringRequestVoteResponses(this.Term) + `,`,
`VoteGranted:` + valueToStringRequestVoteResponses(this.VoteGranted) + `,`,
`XXX_unrecognized:` + fmt12.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func valueToStringRequestVoteResponses(v interface{}) string {
rv := reflect8.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect8.Indirect(rv).Interface()
return fmt12.Sprintf("*%v", pv)
}
func (m *RequestVoteResponse) Size() (n int) {
var l int
_ = l
if m.Term != nil {
n += 1 + sovRequestVoteResponses(uint64(*m.Term))
}
if m.VoteGranted != nil {
n += 2
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func sovRequestVoteResponses(x uint64) (n int) {
for {
n++
x >>= 7
if x == 0 {
break
}
}
return n
}
func sozRequestVoteResponses(x uint64) (n int) {
return sovRequestVoteResponses(uint64((x << 1) ^ uint64((int64(x) >> 63))))
return sovRequestVoteResponses(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func NewPopulatedRequestVoteResponse(r randyRequestVoteResponses, easy bool) *RequestVoteResponse {
this := &RequestVoteResponse{}
v1 := uint64(r.Uint32())
this.Term = &v1
v2 := bool(r.Intn(2) == 0)
this.VoteGranted = &v2
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedRequestVoteResponses(r, 3)
}
return this
}
type randyRequestVoteResponses interface {
Float32() float32
Float64() float64
Int63() int64
Int31() int32
Uint32() uint32
Intn(n int) int
}
func randUTF8RuneRequestVoteResponses(r randyRequestVoteResponses) rune {
res := rune(r.Uint32() % 1112064)
if 55296 <= res {
res += 2047
}
return res
}
func randStringRequestVoteResponses(r randyRequestVoteResponses) string {
v3 := r.Intn(100)
tmps := make([]rune, v3)
for i := 0; i < v3; i++ {
tmps[i] = randUTF8RuneRequestVoteResponses(r)
}
return string(tmps)
}
func randUnrecognizedRequestVoteResponses(r randyRequestVoteResponses, maxFieldNumber int) (data []byte) {
l := r.Intn(5)
for i := 0; i < l; i++ {
wire := r.Intn(4)
if wire == 3 {
wire = 5
}
fieldNumber := maxFieldNumber + r.Intn(100)
data = randFieldRequestVoteResponses(data, r, fieldNumber, wire)
}
return data
}
func randFieldRequestVoteResponses(data []byte, r randyRequestVoteResponses, fieldNumber int, wire int) []byte {
key := uint32(fieldNumber)<<3 | uint32(wire)
switch wire {
case 0:
data = encodeVarintPopulateRequestVoteResponses(data, uint64(key))
data = encodeVarintPopulateRequestVoteResponses(data, uint64(r.Int63()))
case 1:
data = encodeVarintPopulateRequestVoteResponses(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
case 2:
data = encodeVarintPopulateRequestVoteResponses(data, uint64(key))
ll := r.Intn(100)
data = encodeVarintPopulateRequestVoteResponses(data, uint64(ll))
for j := 0; j < ll; j++ {
data = append(data, byte(r.Intn(256)))
}
default:
data = encodeVarintPopulateRequestVoteResponses(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
}
return data
}
func encodeVarintPopulateRequestVoteResponses(data []byte, v uint64) []byte {
for v >= 1<<7 {
data = append(data, uint8(uint64(v)&0x7f|0x80))
v >>= 7
}
data = append(data, uint8(v))
return data
}
func (m *RequestVoteResponse) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *RequestVoteResponse) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.Term != nil {
data[i] = 0x8
i++
i = encodeVarintRequestVoteResponses(data, i, uint64(*m.Term))
}
if m.VoteGranted != nil {
data[i] = 0x10
i++
if *m.VoteGranted {
data[i] = 1
} else {
data[i] = 0
}
i++
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func encodeFixed64RequestVoteResponses(data []byte, offset int, v uint64) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
data[offset+4] = uint8(v >> 32)
data[offset+5] = uint8(v >> 40)
data[offset+6] = uint8(v >> 48)
data[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32RequestVoteResponses(data []byte, offset int, v uint32) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintRequestVoteResponses(data []byte, offset int, v uint64) int {
for v >= 1<<7 {
data[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
data[offset] = uint8(v)
return offset + 1
}
func (this *RequestVoteResponse) GoString() string {
if this == nil {
return "nil"
}
s := strings9.Join([]string{`&protobuf.RequestVoteResponse{` + `Term:` + valueToGoStringRequestVoteResponses(this.Term, "uint64"), `VoteGranted:` + valueToGoStringRequestVoteResponses(this.VoteGranted, "bool"), `XXX_unrecognized:` + fmt13.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func valueToGoStringRequestVoteResponses(v interface{}, typ string) string {
rv := reflect9.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect9.Indirect(rv).Interface()
return fmt13.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
}
func extensionToGoStringRequestVoteResponses(e map[int32]code_google_com_p_gogoprotobuf_proto9.Extension) string {
if e == nil {
return "nil"
}
s := "map[int32]proto.Extension{"
keys := make([]int, 0, len(e))
for k := range e {
keys = append(keys, int(k))
}
sort4.Ints(keys)
ss := []string{}
for _, k := range keys {
ss = append(ss, strconv4.Itoa(k)+": "+e[int32(k)].GoString())
}
s += strings9.Join(ss, ",") + "}"
return s
}
func (this *RequestVoteResponse) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt14.Errorf("that == nil && this != nil")
}
that1, ok := that.(*RequestVoteResponse)
if !ok {
return fmt14.Errorf("that is not of type *RequestVoteResponse")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt14.Errorf("that is type *RequestVoteResponse but is nil && this != nil")
} else if this == nil {
return fmt14.Errorf("that is type *RequestVoteResponsebut is not nil && this == nil")
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return fmt14.Errorf("Term this(%v) Not Equal that(%v)", *this.Term, *that1.Term)
}
} else if this.Term != nil {
return fmt14.Errorf("this.Term == nil && that.Term != nil")
} else if that1.Term != nil {
return fmt14.Errorf("Term this(%v) Not Equal that(%v)", this.Term, that1.Term)
}
if this.VoteGranted != nil && that1.VoteGranted != nil {
if *this.VoteGranted != *that1.VoteGranted {
return fmt14.Errorf("VoteGranted this(%v) Not Equal that(%v)", *this.VoteGranted, *that1.VoteGranted)
}
} else if this.VoteGranted != nil {
return fmt14.Errorf("this.VoteGranted == nil && that.VoteGranted != nil")
} else if that1.VoteGranted != nil {
return fmt14.Errorf("VoteGranted this(%v) Not Equal that(%v)", this.VoteGranted, that1.VoteGranted)
}
if !bytes4.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt14.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *RequestVoteResponse) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*RequestVoteResponse)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return false
}
} else if this.Term != nil {
return false
} else if that1.Term != nil {
return false
}
if this.VoteGranted != nil && that1.VoteGranted != nil {
if *this.VoteGranted != *that1.VoteGranted {
return false
}
} else if this.VoteGranted != nil {
return false
} else if that1.VoteGranted != nil {
return false
}
if !bytes4.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}

View File

@ -1,6 +1,20 @@
package protobuf;
message ProtoRequestVoteResponse {
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
option (gogoproto.gostring_all) = true;
option (gogoproto.equal_all) = true;
option (gogoproto.verbose_equal_all) = true;
option (gogoproto.goproto_stringer_all) = false;
option (gogoproto.stringer_all) = true;
option (gogoproto.populate_all) = true;
option (gogoproto.testgen_all) = true;
option (gogoproto.benchgen_all) = true;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
message RequestVoteResponse {
required uint64 Term=1;
required bool VoteGranted=2;
}

View File

@ -1,86 +1,101 @@
// Code generated by protoc-gen-go.
// Code generated by protoc-gen-gogo.
// source: snapshot_recovery_request.proto
// DO NOT EDIT!
package protobuf
import proto "code.google.com/p/goprotobuf/proto"
import proto "code.google.com/p/gogoprotobuf/proto"
import json "encoding/json"
import math "math"
// discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb"
import io5 "io"
import code_google_com_p_gogoprotobuf_proto10 "code.google.com/p/gogoprotobuf/proto"
import fmt15 "fmt"
import strings10 "strings"
import reflect10 "reflect"
import fmt16 "fmt"
import strings11 "strings"
import code_google_com_p_gogoprotobuf_proto11 "code.google.com/p/gogoprotobuf/proto"
import sort5 "sort"
import strconv5 "strconv"
import reflect11 "reflect"
import fmt17 "fmt"
import bytes5 "bytes"
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
var _ = proto.Marshal
var _ = &json.SyntaxError{}
var _ = math.Inf
type ProtoSnapshotRecoveryRequest struct {
LeaderName *string `protobuf:"bytes,1,req" json:"LeaderName,omitempty"`
LastIndex *uint64 `protobuf:"varint,2,req" json:"LastIndex,omitempty"`
LastTerm *uint64 `protobuf:"varint,3,req" json:"LastTerm,omitempty"`
Peers []*ProtoSnapshotRecoveryRequest_ProtoPeer `protobuf:"bytes,4,rep" json:"Peers,omitempty"`
State []byte `protobuf:"bytes,5,req" json:"State,omitempty"`
XXX_unrecognized []byte `json:"-"`
type SnapshotRecoveryRequest struct {
LeaderName *string `protobuf:"bytes,1,req" json:"LeaderName,omitempty"`
LastIndex *uint64 `protobuf:"varint,2,req" json:"LastIndex,omitempty"`
LastTerm *uint64 `protobuf:"varint,3,req" json:"LastTerm,omitempty"`
Peers []*SnapshotRecoveryRequest_Peer `protobuf:"bytes,4,rep" json:"Peers,omitempty"`
State []byte `protobuf:"bytes,5,req" json:"State,omitempty"`
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoSnapshotRecoveryRequest) Reset() { *m = ProtoSnapshotRecoveryRequest{} }
func (m *ProtoSnapshotRecoveryRequest) String() string { return proto.CompactTextString(m) }
func (*ProtoSnapshotRecoveryRequest) ProtoMessage() {}
func (m *SnapshotRecoveryRequest) Reset() { *m = SnapshotRecoveryRequest{} }
func (*SnapshotRecoveryRequest) ProtoMessage() {}
func (m *ProtoSnapshotRecoveryRequest) GetLeaderName() string {
func (m *SnapshotRecoveryRequest) GetLeaderName() string {
if m != nil && m.LeaderName != nil {
return *m.LeaderName
}
return ""
}
func (m *ProtoSnapshotRecoveryRequest) GetLastIndex() uint64 {
func (m *SnapshotRecoveryRequest) GetLastIndex() uint64 {
if m != nil && m.LastIndex != nil {
return *m.LastIndex
}
return 0
}
func (m *ProtoSnapshotRecoveryRequest) GetLastTerm() uint64 {
func (m *SnapshotRecoveryRequest) GetLastTerm() uint64 {
if m != nil && m.LastTerm != nil {
return *m.LastTerm
}
return 0
}
func (m *ProtoSnapshotRecoveryRequest) GetPeers() []*ProtoSnapshotRecoveryRequest_ProtoPeer {
func (m *SnapshotRecoveryRequest) GetPeers() []*SnapshotRecoveryRequest_Peer {
if m != nil {
return m.Peers
}
return nil
}
func (m *ProtoSnapshotRecoveryRequest) GetState() []byte {
func (m *SnapshotRecoveryRequest) GetState() []byte {
if m != nil {
return m.State
}
return nil
}
type ProtoSnapshotRecoveryRequest_ProtoPeer struct {
type SnapshotRecoveryRequest_Peer struct {
Name *string `protobuf:"bytes,1,req" json:"Name,omitempty"`
ConnectionString *string `protobuf:"bytes,2,req" json:"ConnectionString,omitempty"`
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoSnapshotRecoveryRequest_ProtoPeer) Reset() {
*m = ProtoSnapshotRecoveryRequest_ProtoPeer{}
}
func (m *ProtoSnapshotRecoveryRequest_ProtoPeer) String() string { return proto.CompactTextString(m) }
func (*ProtoSnapshotRecoveryRequest_ProtoPeer) ProtoMessage() {}
func (m *SnapshotRecoveryRequest_Peer) Reset() { *m = SnapshotRecoveryRequest_Peer{} }
func (*SnapshotRecoveryRequest_Peer) ProtoMessage() {}
func (m *ProtoSnapshotRecoveryRequest_ProtoPeer) GetName() string {
func (m *SnapshotRecoveryRequest_Peer) GetName() string {
if m != nil && m.Name != nil {
return *m.Name
}
return ""
}
func (m *ProtoSnapshotRecoveryRequest_ProtoPeer) GetConnectionString() string {
func (m *SnapshotRecoveryRequest_Peer) GetConnectionString() string {
if m != nil && m.ConnectionString != nil {
return *m.ConnectionString
}
@ -89,3 +104,791 @@ func (m *ProtoSnapshotRecoveryRequest_ProtoPeer) GetConnectionString() string {
func init() {
}
func (m *SnapshotRecoveryRequest) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io5.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 2 {
return proto.ErrWrongType
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io5.ErrUnexpectedEOF
}
b := data[index]
index++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + int(stringLen)
if postIndex > l {
return io5.ErrUnexpectedEOF
}
s := string(data[index:postIndex])
m.LeaderName = &s
index = postIndex
case 2:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io5.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.LastIndex = &v
case 3:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io5.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.LastTerm = &v
case 4:
if wireType != 2 {
return proto.ErrWrongType
}
var msglen int
for shift := uint(0); ; shift += 7 {
if index >= l {
return io5.ErrUnexpectedEOF
}
b := data[index]
index++
msglen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + msglen
if postIndex > l {
return io5.ErrUnexpectedEOF
}
m.Peers = append(m.Peers, &SnapshotRecoveryRequest_Peer{})
m.Peers[len(m.Peers)-1].Unmarshal(data[index:postIndex])
index = postIndex
case 5:
if wireType != 2 {
return proto.ErrWrongType
}
var byteLen int
for shift := uint(0); ; shift += 7 {
if index >= l {
return io5.ErrUnexpectedEOF
}
b := data[index]
index++
byteLen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + byteLen
if postIndex > l {
return io5.ErrUnexpectedEOF
}
m.State = append(m.State, data[index:postIndex]...)
index = postIndex
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto10.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func (m *SnapshotRecoveryRequest_Peer) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io5.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 2 {
return proto.ErrWrongType
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io5.ErrUnexpectedEOF
}
b := data[index]
index++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + int(stringLen)
if postIndex > l {
return io5.ErrUnexpectedEOF
}
s := string(data[index:postIndex])
m.Name = &s
index = postIndex
case 2:
if wireType != 2 {
return proto.ErrWrongType
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io5.ErrUnexpectedEOF
}
b := data[index]
index++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + int(stringLen)
if postIndex > l {
return io5.ErrUnexpectedEOF
}
s := string(data[index:postIndex])
m.ConnectionString = &s
index = postIndex
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto10.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func (this *SnapshotRecoveryRequest) String() string {
if this == nil {
return "nil"
}
s := strings10.Join([]string{`&SnapshotRecoveryRequest{`,
`LeaderName:` + valueToStringSnapshotRecoveryRequest(this.LeaderName) + `,`,
`LastIndex:` + valueToStringSnapshotRecoveryRequest(this.LastIndex) + `,`,
`LastTerm:` + valueToStringSnapshotRecoveryRequest(this.LastTerm) + `,`,
`Peers:` + strings10.Replace(fmt15.Sprintf("%v", this.Peers), "SnapshotRecoveryRequest_Peer", "SnapshotRecoveryRequest_Peer", 1) + `,`,
`State:` + valueToStringSnapshotRecoveryRequest(this.State) + `,`,
`XXX_unrecognized:` + fmt15.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func (this *SnapshotRecoveryRequest_Peer) String() string {
if this == nil {
return "nil"
}
s := strings10.Join([]string{`&SnapshotRecoveryRequest_Peer{`,
`Name:` + valueToStringSnapshotRecoveryRequest(this.Name) + `,`,
`ConnectionString:` + valueToStringSnapshotRecoveryRequest(this.ConnectionString) + `,`,
`XXX_unrecognized:` + fmt15.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func valueToStringSnapshotRecoveryRequest(v interface{}) string {
rv := reflect10.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect10.Indirect(rv).Interface()
return fmt15.Sprintf("*%v", pv)
}
func (m *SnapshotRecoveryRequest) Size() (n int) {
var l int
_ = l
if m.LeaderName != nil {
l = len(*m.LeaderName)
n += 1 + l + sovSnapshotRecoveryRequest(uint64(l))
}
if m.LastIndex != nil {
n += 1 + sovSnapshotRecoveryRequest(uint64(*m.LastIndex))
}
if m.LastTerm != nil {
n += 1 + sovSnapshotRecoveryRequest(uint64(*m.LastTerm))
}
if len(m.Peers) > 0 {
for _, e := range m.Peers {
l = e.Size()
n += 1 + l + sovSnapshotRecoveryRequest(uint64(l))
}
}
if m.State != nil {
l = len(m.State)
n += 1 + l + sovSnapshotRecoveryRequest(uint64(l))
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func (m *SnapshotRecoveryRequest_Peer) Size() (n int) {
var l int
_ = l
if m.Name != nil {
l = len(*m.Name)
n += 1 + l + sovSnapshotRecoveryRequest(uint64(l))
}
if m.ConnectionString != nil {
l = len(*m.ConnectionString)
n += 1 + l + sovSnapshotRecoveryRequest(uint64(l))
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func sovSnapshotRecoveryRequest(x uint64) (n int) {
for {
n++
x >>= 7
if x == 0 {
break
}
}
return n
}
func sozSnapshotRecoveryRequest(x uint64) (n int) {
return sovSnapshotRecoveryRequest(uint64((x << 1) ^ uint64((int64(x) >> 63))))
return sovSnapshotRecoveryRequest(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func NewPopulatedSnapshotRecoveryRequest(r randySnapshotRecoveryRequest, easy bool) *SnapshotRecoveryRequest {
this := &SnapshotRecoveryRequest{}
v1 := randStringSnapshotRecoveryRequest(r)
this.LeaderName = &v1
v2 := uint64(r.Uint32())
this.LastIndex = &v2
v3 := uint64(r.Uint32())
this.LastTerm = &v3
if r.Intn(10) != 0 {
v4 := r.Intn(10)
this.Peers = make([]*SnapshotRecoveryRequest_Peer, v4)
for i := 0; i < v4; i++ {
this.Peers[i] = NewPopulatedSnapshotRecoveryRequest_Peer(r, easy)
}
}
v5 := r.Intn(100)
this.State = make([]byte, v5)
for i := 0; i < v5; i++ {
this.State[i] = byte(r.Intn(256))
}
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedSnapshotRecoveryRequest(r, 6)
}
return this
}
func NewPopulatedSnapshotRecoveryRequest_Peer(r randySnapshotRecoveryRequest, easy bool) *SnapshotRecoveryRequest_Peer {
this := &SnapshotRecoveryRequest_Peer{}
v6 := randStringSnapshotRecoveryRequest(r)
this.Name = &v6
v7 := randStringSnapshotRecoveryRequest(r)
this.ConnectionString = &v7
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedSnapshotRecoveryRequest(r, 3)
}
return this
}
type randySnapshotRecoveryRequest interface {
Float32() float32
Float64() float64
Int63() int64
Int31() int32
Uint32() uint32
Intn(n int) int
}
func randUTF8RuneSnapshotRecoveryRequest(r randySnapshotRecoveryRequest) rune {
res := rune(r.Uint32() % 1112064)
if 55296 <= res {
res += 2047
}
return res
}
func randStringSnapshotRecoveryRequest(r randySnapshotRecoveryRequest) string {
v8 := r.Intn(100)
tmps := make([]rune, v8)
for i := 0; i < v8; i++ {
tmps[i] = randUTF8RuneSnapshotRecoveryRequest(r)
}
return string(tmps)
}
func randUnrecognizedSnapshotRecoveryRequest(r randySnapshotRecoveryRequest, maxFieldNumber int) (data []byte) {
l := r.Intn(5)
for i := 0; i < l; i++ {
wire := r.Intn(4)
if wire == 3 {
wire = 5
}
fieldNumber := maxFieldNumber + r.Intn(100)
data = randFieldSnapshotRecoveryRequest(data, r, fieldNumber, wire)
}
return data
}
func randFieldSnapshotRecoveryRequest(data []byte, r randySnapshotRecoveryRequest, fieldNumber int, wire int) []byte {
key := uint32(fieldNumber)<<3 | uint32(wire)
switch wire {
case 0:
data = encodeVarintPopulateSnapshotRecoveryRequest(data, uint64(key))
data = encodeVarintPopulateSnapshotRecoveryRequest(data, uint64(r.Int63()))
case 1:
data = encodeVarintPopulateSnapshotRecoveryRequest(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
case 2:
data = encodeVarintPopulateSnapshotRecoveryRequest(data, uint64(key))
ll := r.Intn(100)
data = encodeVarintPopulateSnapshotRecoveryRequest(data, uint64(ll))
for j := 0; j < ll; j++ {
data = append(data, byte(r.Intn(256)))
}
default:
data = encodeVarintPopulateSnapshotRecoveryRequest(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
}
return data
}
func encodeVarintPopulateSnapshotRecoveryRequest(data []byte, v uint64) []byte {
for v >= 1<<7 {
data = append(data, uint8(uint64(v)&0x7f|0x80))
v >>= 7
}
data = append(data, uint8(v))
return data
}
func (m *SnapshotRecoveryRequest) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *SnapshotRecoveryRequest) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.LeaderName != nil {
data[i] = 0xa
i++
i = encodeVarintSnapshotRecoveryRequest(data, i, uint64(len(*m.LeaderName)))
i += copy(data[i:], *m.LeaderName)
}
if m.LastIndex != nil {
data[i] = 0x10
i++
i = encodeVarintSnapshotRecoveryRequest(data, i, uint64(*m.LastIndex))
}
if m.LastTerm != nil {
data[i] = 0x18
i++
i = encodeVarintSnapshotRecoveryRequest(data, i, uint64(*m.LastTerm))
}
if len(m.Peers) > 0 {
for _, msg := range m.Peers {
data[i] = 0x22
i++
i = encodeVarintSnapshotRecoveryRequest(data, i, uint64(msg.Size()))
n, err := msg.MarshalTo(data[i:])
if err != nil {
return 0, err
}
i += n
}
}
if m.State != nil {
data[i] = 0x2a
i++
i = encodeVarintSnapshotRecoveryRequest(data, i, uint64(len(m.State)))
i += copy(data[i:], m.State)
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func (m *SnapshotRecoveryRequest_Peer) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *SnapshotRecoveryRequest_Peer) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.Name != nil {
data[i] = 0xa
i++
i = encodeVarintSnapshotRecoveryRequest(data, i, uint64(len(*m.Name)))
i += copy(data[i:], *m.Name)
}
if m.ConnectionString != nil {
data[i] = 0x12
i++
i = encodeVarintSnapshotRecoveryRequest(data, i, uint64(len(*m.ConnectionString)))
i += copy(data[i:], *m.ConnectionString)
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func encodeFixed64SnapshotRecoveryRequest(data []byte, offset int, v uint64) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
data[offset+4] = uint8(v >> 32)
data[offset+5] = uint8(v >> 40)
data[offset+6] = uint8(v >> 48)
data[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32SnapshotRecoveryRequest(data []byte, offset int, v uint32) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintSnapshotRecoveryRequest(data []byte, offset int, v uint64) int {
for v >= 1<<7 {
data[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
data[offset] = uint8(v)
return offset + 1
}
func (this *SnapshotRecoveryRequest) GoString() string {
if this == nil {
return "nil"
}
s := strings11.Join([]string{`&protobuf.SnapshotRecoveryRequest{` + `LeaderName:` + valueToGoStringSnapshotRecoveryRequest(this.LeaderName, "string"), `LastIndex:` + valueToGoStringSnapshotRecoveryRequest(this.LastIndex, "uint64"), `LastTerm:` + valueToGoStringSnapshotRecoveryRequest(this.LastTerm, "uint64"), `Peers:` + fmt16.Sprintf("%#v", this.Peers), `State:` + valueToGoStringSnapshotRecoveryRequest(this.State, "byte"), `XXX_unrecognized:` + fmt16.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func (this *SnapshotRecoveryRequest_Peer) GoString() string {
if this == nil {
return "nil"
}
s := strings11.Join([]string{`&protobuf.SnapshotRecoveryRequest_Peer{` + `Name:` + valueToGoStringSnapshotRecoveryRequest(this.Name, "string"), `ConnectionString:` + valueToGoStringSnapshotRecoveryRequest(this.ConnectionString, "string"), `XXX_unrecognized:` + fmt16.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func valueToGoStringSnapshotRecoveryRequest(v interface{}, typ string) string {
rv := reflect11.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect11.Indirect(rv).Interface()
return fmt16.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
}
func extensionToGoStringSnapshotRecoveryRequest(e map[int32]code_google_com_p_gogoprotobuf_proto11.Extension) string {
if e == nil {
return "nil"
}
s := "map[int32]proto.Extension{"
keys := make([]int, 0, len(e))
for k := range e {
keys = append(keys, int(k))
}
sort5.Ints(keys)
ss := []string{}
for _, k := range keys {
ss = append(ss, strconv5.Itoa(k)+": "+e[int32(k)].GoString())
}
s += strings11.Join(ss, ",") + "}"
return s
}
func (this *SnapshotRecoveryRequest) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt17.Errorf("that == nil && this != nil")
}
that1, ok := that.(*SnapshotRecoveryRequest)
if !ok {
return fmt17.Errorf("that is not of type *SnapshotRecoveryRequest")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt17.Errorf("that is type *SnapshotRecoveryRequest but is nil && this != nil")
} else if this == nil {
return fmt17.Errorf("that is type *SnapshotRecoveryRequestbut is not nil && this == nil")
}
if this.LeaderName != nil && that1.LeaderName != nil {
if *this.LeaderName != *that1.LeaderName {
return fmt17.Errorf("LeaderName this(%v) Not Equal that(%v)", *this.LeaderName, *that1.LeaderName)
}
} else if this.LeaderName != nil {
return fmt17.Errorf("this.LeaderName == nil && that.LeaderName != nil")
} else if that1.LeaderName != nil {
return fmt17.Errorf("LeaderName this(%v) Not Equal that(%v)", this.LeaderName, that1.LeaderName)
}
if this.LastIndex != nil && that1.LastIndex != nil {
if *this.LastIndex != *that1.LastIndex {
return fmt17.Errorf("LastIndex this(%v) Not Equal that(%v)", *this.LastIndex, *that1.LastIndex)
}
} else if this.LastIndex != nil {
return fmt17.Errorf("this.LastIndex == nil && that.LastIndex != nil")
} else if that1.LastIndex != nil {
return fmt17.Errorf("LastIndex this(%v) Not Equal that(%v)", this.LastIndex, that1.LastIndex)
}
if this.LastTerm != nil && that1.LastTerm != nil {
if *this.LastTerm != *that1.LastTerm {
return fmt17.Errorf("LastTerm this(%v) Not Equal that(%v)", *this.LastTerm, *that1.LastTerm)
}
} else if this.LastTerm != nil {
return fmt17.Errorf("this.LastTerm == nil && that.LastTerm != nil")
} else if that1.LastTerm != nil {
return fmt17.Errorf("LastTerm this(%v) Not Equal that(%v)", this.LastTerm, that1.LastTerm)
}
if len(this.Peers) != len(that1.Peers) {
return fmt17.Errorf("Peers this(%v) Not Equal that(%v)", len(this.Peers), len(that1.Peers))
}
for i := range this.Peers {
if !this.Peers[i].Equal(that1.Peers[i]) {
return fmt17.Errorf("Peers this[%v](%v) Not Equal that[%v](%v)", i, this.Peers[i], i, that1.Peers[i])
}
}
if !bytes5.Equal(this.State, that1.State) {
return fmt17.Errorf("State this(%v) Not Equal that(%v)", this.State, that1.State)
}
if !bytes5.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt17.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *SnapshotRecoveryRequest) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*SnapshotRecoveryRequest)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.LeaderName != nil && that1.LeaderName != nil {
if *this.LeaderName != *that1.LeaderName {
return false
}
} else if this.LeaderName != nil {
return false
} else if that1.LeaderName != nil {
return false
}
if this.LastIndex != nil && that1.LastIndex != nil {
if *this.LastIndex != *that1.LastIndex {
return false
}
} else if this.LastIndex != nil {
return false
} else if that1.LastIndex != nil {
return false
}
if this.LastTerm != nil && that1.LastTerm != nil {
if *this.LastTerm != *that1.LastTerm {
return false
}
} else if this.LastTerm != nil {
return false
} else if that1.LastTerm != nil {
return false
}
if len(this.Peers) != len(that1.Peers) {
return false
}
for i := range this.Peers {
if !this.Peers[i].Equal(that1.Peers[i]) {
return false
}
}
if !bytes5.Equal(this.State, that1.State) {
return false
}
if !bytes5.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}
func (this *SnapshotRecoveryRequest_Peer) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt17.Errorf("that == nil && this != nil")
}
that1, ok := that.(*SnapshotRecoveryRequest_Peer)
if !ok {
return fmt17.Errorf("that is not of type *SnapshotRecoveryRequest_Peer")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt17.Errorf("that is type *SnapshotRecoveryRequest_Peer but is nil && this != nil")
} else if this == nil {
return fmt17.Errorf("that is type *SnapshotRecoveryRequest_Peerbut is not nil && this == nil")
}
if this.Name != nil && that1.Name != nil {
if *this.Name != *that1.Name {
return fmt17.Errorf("Name this(%v) Not Equal that(%v)", *this.Name, *that1.Name)
}
} else if this.Name != nil {
return fmt17.Errorf("this.Name == nil && that.Name != nil")
} else if that1.Name != nil {
return fmt17.Errorf("Name this(%v) Not Equal that(%v)", this.Name, that1.Name)
}
if this.ConnectionString != nil && that1.ConnectionString != nil {
if *this.ConnectionString != *that1.ConnectionString {
return fmt17.Errorf("ConnectionString this(%v) Not Equal that(%v)", *this.ConnectionString, *that1.ConnectionString)
}
} else if this.ConnectionString != nil {
return fmt17.Errorf("this.ConnectionString == nil && that.ConnectionString != nil")
} else if that1.ConnectionString != nil {
return fmt17.Errorf("ConnectionString this(%v) Not Equal that(%v)", this.ConnectionString, that1.ConnectionString)
}
if !bytes5.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt17.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *SnapshotRecoveryRequest_Peer) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*SnapshotRecoveryRequest_Peer)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.Name != nil && that1.Name != nil {
if *this.Name != *that1.Name {
return false
}
} else if this.Name != nil {
return false
} else if that1.Name != nil {
return false
}
if this.ConnectionString != nil && that1.ConnectionString != nil {
if *this.ConnectionString != *that1.ConnectionString {
return false
}
} else if this.ConnectionString != nil {
return false
} else if that1.ConnectionString != nil {
return false
}
if !bytes5.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}

View File

@ -1,15 +1,29 @@
package protobuf;
message ProtoSnapshotRecoveryRequest {
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
option (gogoproto.gostring_all) = true;
option (gogoproto.equal_all) = true;
option (gogoproto.verbose_equal_all) = true;
option (gogoproto.goproto_stringer_all) = false;
option (gogoproto.stringer_all) = true;
option (gogoproto.populate_all) = true;
option (gogoproto.testgen_all) = true;
option (gogoproto.benchgen_all) = true;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
message SnapshotRecoveryRequest {
required string LeaderName=1;
required uint64 LastIndex=2;
required uint64 LastTerm=3;
message ProtoPeer {
message Peer {
required string Name=1;
required string ConnectionString=2;
}
repeated ProtoPeer Peers=4;
repeated Peer Peers=4;
required bytes State=5;
}

View File

@ -1,44 +1,62 @@
// Code generated by protoc-gen-go.
// Code generated by protoc-gen-gogo.
// source: snapshot_recovery_response.proto
// DO NOT EDIT!
package protobuf
import proto "code.google.com/p/goprotobuf/proto"
import proto "code.google.com/p/gogoprotobuf/proto"
import json "encoding/json"
import math "math"
// discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb"
import io6 "io"
import code_google_com_p_gogoprotobuf_proto12 "code.google.com/p/gogoprotobuf/proto"
import fmt18 "fmt"
import strings12 "strings"
import reflect12 "reflect"
import fmt19 "fmt"
import strings13 "strings"
import code_google_com_p_gogoprotobuf_proto13 "code.google.com/p/gogoprotobuf/proto"
import sort6 "sort"
import strconv6 "strconv"
import reflect13 "reflect"
import fmt20 "fmt"
import bytes6 "bytes"
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
var _ = proto.Marshal
var _ = &json.SyntaxError{}
var _ = math.Inf
type ProtoSnapshotRecoveryResponse struct {
type SnapshotRecoveryResponse struct {
Term *uint64 `protobuf:"varint,1,req" json:"Term,omitempty"`
Success *bool `protobuf:"varint,2,req" json:"Success,omitempty"`
CommitIndex *uint64 `protobuf:"varint,3,req" json:"CommitIndex,omitempty"`
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoSnapshotRecoveryResponse) Reset() { *m = ProtoSnapshotRecoveryResponse{} }
func (m *ProtoSnapshotRecoveryResponse) String() string { return proto.CompactTextString(m) }
func (*ProtoSnapshotRecoveryResponse) ProtoMessage() {}
func (m *SnapshotRecoveryResponse) Reset() { *m = SnapshotRecoveryResponse{} }
func (*SnapshotRecoveryResponse) ProtoMessage() {}
func (m *ProtoSnapshotRecoveryResponse) GetTerm() uint64 {
func (m *SnapshotRecoveryResponse) GetTerm() uint64 {
if m != nil && m.Term != nil {
return *m.Term
}
return 0
}
func (m *ProtoSnapshotRecoveryResponse) GetSuccess() bool {
func (m *SnapshotRecoveryResponse) GetSuccess() bool {
if m != nil && m.Success != nil {
return *m.Success
}
return false
}
func (m *ProtoSnapshotRecoveryResponse) GetCommitIndex() uint64 {
func (m *SnapshotRecoveryResponse) GetCommitIndex() uint64 {
if m != nil && m.CommitIndex != nil {
return *m.CommitIndex
}
@ -47,3 +65,430 @@ func (m *ProtoSnapshotRecoveryResponse) GetCommitIndex() uint64 {
func init() {
}
func (m *SnapshotRecoveryResponse) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io6.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io6.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.Term = &v
case 2:
if wireType != 0 {
return proto.ErrWrongType
}
var v int
for shift := uint(0); ; shift += 7 {
if index >= l {
return io6.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
b := bool(v != 0)
m.Success = &b
case 3:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io6.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.CommitIndex = &v
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto12.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func (this *SnapshotRecoveryResponse) String() string {
if this == nil {
return "nil"
}
s := strings12.Join([]string{`&SnapshotRecoveryResponse{`,
`Term:` + valueToStringSnapshotRecoveryResponse(this.Term) + `,`,
`Success:` + valueToStringSnapshotRecoveryResponse(this.Success) + `,`,
`CommitIndex:` + valueToStringSnapshotRecoveryResponse(this.CommitIndex) + `,`,
`XXX_unrecognized:` + fmt18.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func valueToStringSnapshotRecoveryResponse(v interface{}) string {
rv := reflect12.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect12.Indirect(rv).Interface()
return fmt18.Sprintf("*%v", pv)
}
func (m *SnapshotRecoveryResponse) Size() (n int) {
var l int
_ = l
if m.Term != nil {
n += 1 + sovSnapshotRecoveryResponse(uint64(*m.Term))
}
if m.Success != nil {
n += 2
}
if m.CommitIndex != nil {
n += 1 + sovSnapshotRecoveryResponse(uint64(*m.CommitIndex))
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func sovSnapshotRecoveryResponse(x uint64) (n int) {
for {
n++
x >>= 7
if x == 0 {
break
}
}
return n
}
func sozSnapshotRecoveryResponse(x uint64) (n int) {
return sovSnapshotRecoveryResponse(uint64((x << 1) ^ uint64((int64(x) >> 63))))
return sovSnapshotRecoveryResponse(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func NewPopulatedSnapshotRecoveryResponse(r randySnapshotRecoveryResponse, easy bool) *SnapshotRecoveryResponse {
this := &SnapshotRecoveryResponse{}
v1 := uint64(r.Uint32())
this.Term = &v1
v2 := bool(r.Intn(2) == 0)
this.Success = &v2
v3 := uint64(r.Uint32())
this.CommitIndex = &v3
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedSnapshotRecoveryResponse(r, 4)
}
return this
}
type randySnapshotRecoveryResponse interface {
Float32() float32
Float64() float64
Int63() int64
Int31() int32
Uint32() uint32
Intn(n int) int
}
func randUTF8RuneSnapshotRecoveryResponse(r randySnapshotRecoveryResponse) rune {
res := rune(r.Uint32() % 1112064)
if 55296 <= res {
res += 2047
}
return res
}
func randStringSnapshotRecoveryResponse(r randySnapshotRecoveryResponse) string {
v4 := r.Intn(100)
tmps := make([]rune, v4)
for i := 0; i < v4; i++ {
tmps[i] = randUTF8RuneSnapshotRecoveryResponse(r)
}
return string(tmps)
}
func randUnrecognizedSnapshotRecoveryResponse(r randySnapshotRecoveryResponse, maxFieldNumber int) (data []byte) {
l := r.Intn(5)
for i := 0; i < l; i++ {
wire := r.Intn(4)
if wire == 3 {
wire = 5
}
fieldNumber := maxFieldNumber + r.Intn(100)
data = randFieldSnapshotRecoveryResponse(data, r, fieldNumber, wire)
}
return data
}
func randFieldSnapshotRecoveryResponse(data []byte, r randySnapshotRecoveryResponse, fieldNumber int, wire int) []byte {
key := uint32(fieldNumber)<<3 | uint32(wire)
switch wire {
case 0:
data = encodeVarintPopulateSnapshotRecoveryResponse(data, uint64(key))
data = encodeVarintPopulateSnapshotRecoveryResponse(data, uint64(r.Int63()))
case 1:
data = encodeVarintPopulateSnapshotRecoveryResponse(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
case 2:
data = encodeVarintPopulateSnapshotRecoveryResponse(data, uint64(key))
ll := r.Intn(100)
data = encodeVarintPopulateSnapshotRecoveryResponse(data, uint64(ll))
for j := 0; j < ll; j++ {
data = append(data, byte(r.Intn(256)))
}
default:
data = encodeVarintPopulateSnapshotRecoveryResponse(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
}
return data
}
func encodeVarintPopulateSnapshotRecoveryResponse(data []byte, v uint64) []byte {
for v >= 1<<7 {
data = append(data, uint8(uint64(v)&0x7f|0x80))
v >>= 7
}
data = append(data, uint8(v))
return data
}
func (m *SnapshotRecoveryResponse) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *SnapshotRecoveryResponse) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.Term != nil {
data[i] = 0x8
i++
i = encodeVarintSnapshotRecoveryResponse(data, i, uint64(*m.Term))
}
if m.Success != nil {
data[i] = 0x10
i++
if *m.Success {
data[i] = 1
} else {
data[i] = 0
}
i++
}
if m.CommitIndex != nil {
data[i] = 0x18
i++
i = encodeVarintSnapshotRecoveryResponse(data, i, uint64(*m.CommitIndex))
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func encodeFixed64SnapshotRecoveryResponse(data []byte, offset int, v uint64) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
data[offset+4] = uint8(v >> 32)
data[offset+5] = uint8(v >> 40)
data[offset+6] = uint8(v >> 48)
data[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32SnapshotRecoveryResponse(data []byte, offset int, v uint32) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintSnapshotRecoveryResponse(data []byte, offset int, v uint64) int {
for v >= 1<<7 {
data[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
data[offset] = uint8(v)
return offset + 1
}
func (this *SnapshotRecoveryResponse) GoString() string {
if this == nil {
return "nil"
}
s := strings13.Join([]string{`&protobuf.SnapshotRecoveryResponse{` + `Term:` + valueToGoStringSnapshotRecoveryResponse(this.Term, "uint64"), `Success:` + valueToGoStringSnapshotRecoveryResponse(this.Success, "bool"), `CommitIndex:` + valueToGoStringSnapshotRecoveryResponse(this.CommitIndex, "uint64"), `XXX_unrecognized:` + fmt19.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func valueToGoStringSnapshotRecoveryResponse(v interface{}, typ string) string {
rv := reflect13.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect13.Indirect(rv).Interface()
return fmt19.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
}
func extensionToGoStringSnapshotRecoveryResponse(e map[int32]code_google_com_p_gogoprotobuf_proto13.Extension) string {
if e == nil {
return "nil"
}
s := "map[int32]proto.Extension{"
keys := make([]int, 0, len(e))
for k := range e {
keys = append(keys, int(k))
}
sort6.Ints(keys)
ss := []string{}
for _, k := range keys {
ss = append(ss, strconv6.Itoa(k)+": "+e[int32(k)].GoString())
}
s += strings13.Join(ss, ",") + "}"
return s
}
func (this *SnapshotRecoveryResponse) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt20.Errorf("that == nil && this != nil")
}
that1, ok := that.(*SnapshotRecoveryResponse)
if !ok {
return fmt20.Errorf("that is not of type *SnapshotRecoveryResponse")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt20.Errorf("that is type *SnapshotRecoveryResponse but is nil && this != nil")
} else if this == nil {
return fmt20.Errorf("that is type *SnapshotRecoveryResponsebut is not nil && this == nil")
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return fmt20.Errorf("Term this(%v) Not Equal that(%v)", *this.Term, *that1.Term)
}
} else if this.Term != nil {
return fmt20.Errorf("this.Term == nil && that.Term != nil")
} else if that1.Term != nil {
return fmt20.Errorf("Term this(%v) Not Equal that(%v)", this.Term, that1.Term)
}
if this.Success != nil && that1.Success != nil {
if *this.Success != *that1.Success {
return fmt20.Errorf("Success this(%v) Not Equal that(%v)", *this.Success, *that1.Success)
}
} else if this.Success != nil {
return fmt20.Errorf("this.Success == nil && that.Success != nil")
} else if that1.Success != nil {
return fmt20.Errorf("Success this(%v) Not Equal that(%v)", this.Success, that1.Success)
}
if this.CommitIndex != nil && that1.CommitIndex != nil {
if *this.CommitIndex != *that1.CommitIndex {
return fmt20.Errorf("CommitIndex this(%v) Not Equal that(%v)", *this.CommitIndex, *that1.CommitIndex)
}
} else if this.CommitIndex != nil {
return fmt20.Errorf("this.CommitIndex == nil && that.CommitIndex != nil")
} else if that1.CommitIndex != nil {
return fmt20.Errorf("CommitIndex this(%v) Not Equal that(%v)", this.CommitIndex, that1.CommitIndex)
}
if !bytes6.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt20.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *SnapshotRecoveryResponse) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*SnapshotRecoveryResponse)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.Term != nil && that1.Term != nil {
if *this.Term != *that1.Term {
return false
}
} else if this.Term != nil {
return false
} else if that1.Term != nil {
return false
}
if this.Success != nil && that1.Success != nil {
if *this.Success != *that1.Success {
return false
}
} else if this.Success != nil {
return false
} else if that1.Success != nil {
return false
}
if this.CommitIndex != nil && that1.CommitIndex != nil {
if *this.CommitIndex != *that1.CommitIndex {
return false
}
} else if this.CommitIndex != nil {
return false
} else if that1.CommitIndex != nil {
return false
}
if !bytes6.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}

View File

@ -1,6 +1,20 @@
package protobuf;
message ProtoSnapshotRecoveryResponse {
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
option (gogoproto.gostring_all) = true;
option (gogoproto.equal_all) = true;
option (gogoproto.verbose_equal_all) = true;
option (gogoproto.goproto_stringer_all) = false;
option (gogoproto.stringer_all) = true;
option (gogoproto.populate_all) = true;
option (gogoproto.testgen_all) = true;
option (gogoproto.benchgen_all) = true;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
message SnapshotRecoveryResponse {
required uint64 Term=1;
required bool Success=2;
required uint64 CommitIndex=3;

View File

@ -1,44 +1,62 @@
// Code generated by protoc-gen-go.
// Code generated by protoc-gen-gogo.
// source: snapshot_request.proto
// DO NOT EDIT!
package protobuf
import proto "code.google.com/p/goprotobuf/proto"
import proto "code.google.com/p/gogoprotobuf/proto"
import json "encoding/json"
import math "math"
// discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb"
import io7 "io"
import code_google_com_p_gogoprotobuf_proto14 "code.google.com/p/gogoprotobuf/proto"
import fmt21 "fmt"
import strings14 "strings"
import reflect14 "reflect"
import fmt22 "fmt"
import strings15 "strings"
import code_google_com_p_gogoprotobuf_proto15 "code.google.com/p/gogoprotobuf/proto"
import sort7 "sort"
import strconv7 "strconv"
import reflect15 "reflect"
import fmt23 "fmt"
import bytes7 "bytes"
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
var _ = proto.Marshal
var _ = &json.SyntaxError{}
var _ = math.Inf
type ProtoSnapshotRequest struct {
type SnapshotRequest struct {
LeaderName *string `protobuf:"bytes,1,req" json:"LeaderName,omitempty"`
LastIndex *uint64 `protobuf:"varint,2,req" json:"LastIndex,omitempty"`
LastTerm *uint64 `protobuf:"varint,3,req" json:"LastTerm,omitempty"`
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoSnapshotRequest) Reset() { *m = ProtoSnapshotRequest{} }
func (m *ProtoSnapshotRequest) String() string { return proto.CompactTextString(m) }
func (*ProtoSnapshotRequest) ProtoMessage() {}
func (m *SnapshotRequest) Reset() { *m = SnapshotRequest{} }
func (*SnapshotRequest) ProtoMessage() {}
func (m *ProtoSnapshotRequest) GetLeaderName() string {
func (m *SnapshotRequest) GetLeaderName() string {
if m != nil && m.LeaderName != nil {
return *m.LeaderName
}
return ""
}
func (m *ProtoSnapshotRequest) GetLastIndex() uint64 {
func (m *SnapshotRequest) GetLastIndex() uint64 {
if m != nil && m.LastIndex != nil {
return *m.LastIndex
}
return 0
}
func (m *ProtoSnapshotRequest) GetLastTerm() uint64 {
func (m *SnapshotRequest) GetLastTerm() uint64 {
if m != nil && m.LastTerm != nil {
return *m.LastTerm
}
@ -47,3 +65,432 @@ func (m *ProtoSnapshotRequest) GetLastTerm() uint64 {
func init() {
}
func (m *SnapshotRequest) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io7.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 2 {
return proto.ErrWrongType
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io7.ErrUnexpectedEOF
}
b := data[index]
index++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
postIndex := index + int(stringLen)
if postIndex > l {
return io7.ErrUnexpectedEOF
}
s := string(data[index:postIndex])
m.LeaderName = &s
index = postIndex
case 2:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io7.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.LastIndex = &v
case 3:
if wireType != 0 {
return proto.ErrWrongType
}
var v uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io7.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
m.LastTerm = &v
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto14.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func (this *SnapshotRequest) String() string {
if this == nil {
return "nil"
}
s := strings14.Join([]string{`&SnapshotRequest{`,
`LeaderName:` + valueToStringSnapshotRequest(this.LeaderName) + `,`,
`LastIndex:` + valueToStringSnapshotRequest(this.LastIndex) + `,`,
`LastTerm:` + valueToStringSnapshotRequest(this.LastTerm) + `,`,
`XXX_unrecognized:` + fmt21.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func valueToStringSnapshotRequest(v interface{}) string {
rv := reflect14.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect14.Indirect(rv).Interface()
return fmt21.Sprintf("*%v", pv)
}
func (m *SnapshotRequest) Size() (n int) {
var l int
_ = l
if m.LeaderName != nil {
l = len(*m.LeaderName)
n += 1 + l + sovSnapshotRequest(uint64(l))
}
if m.LastIndex != nil {
n += 1 + sovSnapshotRequest(uint64(*m.LastIndex))
}
if m.LastTerm != nil {
n += 1 + sovSnapshotRequest(uint64(*m.LastTerm))
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func sovSnapshotRequest(x uint64) (n int) {
for {
n++
x >>= 7
if x == 0 {
break
}
}
return n
}
func sozSnapshotRequest(x uint64) (n int) {
return sovSnapshotRequest(uint64((x << 1) ^ uint64((int64(x) >> 63))))
return sovSnapshotRequest(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func NewPopulatedSnapshotRequest(r randySnapshotRequest, easy bool) *SnapshotRequest {
this := &SnapshotRequest{}
v1 := randStringSnapshotRequest(r)
this.LeaderName = &v1
v2 := uint64(r.Uint32())
this.LastIndex = &v2
v3 := uint64(r.Uint32())
this.LastTerm = &v3
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedSnapshotRequest(r, 4)
}
return this
}
type randySnapshotRequest interface {
Float32() float32
Float64() float64
Int63() int64
Int31() int32
Uint32() uint32
Intn(n int) int
}
func randUTF8RuneSnapshotRequest(r randySnapshotRequest) rune {
res := rune(r.Uint32() % 1112064)
if 55296 <= res {
res += 2047
}
return res
}
func randStringSnapshotRequest(r randySnapshotRequest) string {
v4 := r.Intn(100)
tmps := make([]rune, v4)
for i := 0; i < v4; i++ {
tmps[i] = randUTF8RuneSnapshotRequest(r)
}
return string(tmps)
}
func randUnrecognizedSnapshotRequest(r randySnapshotRequest, maxFieldNumber int) (data []byte) {
l := r.Intn(5)
for i := 0; i < l; i++ {
wire := r.Intn(4)
if wire == 3 {
wire = 5
}
fieldNumber := maxFieldNumber + r.Intn(100)
data = randFieldSnapshotRequest(data, r, fieldNumber, wire)
}
return data
}
func randFieldSnapshotRequest(data []byte, r randySnapshotRequest, fieldNumber int, wire int) []byte {
key := uint32(fieldNumber)<<3 | uint32(wire)
switch wire {
case 0:
data = encodeVarintPopulateSnapshotRequest(data, uint64(key))
data = encodeVarintPopulateSnapshotRequest(data, uint64(r.Int63()))
case 1:
data = encodeVarintPopulateSnapshotRequest(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
case 2:
data = encodeVarintPopulateSnapshotRequest(data, uint64(key))
ll := r.Intn(100)
data = encodeVarintPopulateSnapshotRequest(data, uint64(ll))
for j := 0; j < ll; j++ {
data = append(data, byte(r.Intn(256)))
}
default:
data = encodeVarintPopulateSnapshotRequest(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
}
return data
}
func encodeVarintPopulateSnapshotRequest(data []byte, v uint64) []byte {
for v >= 1<<7 {
data = append(data, uint8(uint64(v)&0x7f|0x80))
v >>= 7
}
data = append(data, uint8(v))
return data
}
func (m *SnapshotRequest) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *SnapshotRequest) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.LeaderName != nil {
data[i] = 0xa
i++
i = encodeVarintSnapshotRequest(data, i, uint64(len(*m.LeaderName)))
i += copy(data[i:], *m.LeaderName)
}
if m.LastIndex != nil {
data[i] = 0x10
i++
i = encodeVarintSnapshotRequest(data, i, uint64(*m.LastIndex))
}
if m.LastTerm != nil {
data[i] = 0x18
i++
i = encodeVarintSnapshotRequest(data, i, uint64(*m.LastTerm))
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func encodeFixed64SnapshotRequest(data []byte, offset int, v uint64) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
data[offset+4] = uint8(v >> 32)
data[offset+5] = uint8(v >> 40)
data[offset+6] = uint8(v >> 48)
data[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32SnapshotRequest(data []byte, offset int, v uint32) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintSnapshotRequest(data []byte, offset int, v uint64) int {
for v >= 1<<7 {
data[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
data[offset] = uint8(v)
return offset + 1
}
func (this *SnapshotRequest) GoString() string {
if this == nil {
return "nil"
}
s := strings15.Join([]string{`&protobuf.SnapshotRequest{` + `LeaderName:` + valueToGoStringSnapshotRequest(this.LeaderName, "string"), `LastIndex:` + valueToGoStringSnapshotRequest(this.LastIndex, "uint64"), `LastTerm:` + valueToGoStringSnapshotRequest(this.LastTerm, "uint64"), `XXX_unrecognized:` + fmt22.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func valueToGoStringSnapshotRequest(v interface{}, typ string) string {
rv := reflect15.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect15.Indirect(rv).Interface()
return fmt22.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
}
func extensionToGoStringSnapshotRequest(e map[int32]code_google_com_p_gogoprotobuf_proto15.Extension) string {
if e == nil {
return "nil"
}
s := "map[int32]proto.Extension{"
keys := make([]int, 0, len(e))
for k := range e {
keys = append(keys, int(k))
}
sort7.Ints(keys)
ss := []string{}
for _, k := range keys {
ss = append(ss, strconv7.Itoa(k)+": "+e[int32(k)].GoString())
}
s += strings15.Join(ss, ",") + "}"
return s
}
func (this *SnapshotRequest) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt23.Errorf("that == nil && this != nil")
}
that1, ok := that.(*SnapshotRequest)
if !ok {
return fmt23.Errorf("that is not of type *SnapshotRequest")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt23.Errorf("that is type *SnapshotRequest but is nil && this != nil")
} else if this == nil {
return fmt23.Errorf("that is type *SnapshotRequestbut is not nil && this == nil")
}
if this.LeaderName != nil && that1.LeaderName != nil {
if *this.LeaderName != *that1.LeaderName {
return fmt23.Errorf("LeaderName this(%v) Not Equal that(%v)", *this.LeaderName, *that1.LeaderName)
}
} else if this.LeaderName != nil {
return fmt23.Errorf("this.LeaderName == nil && that.LeaderName != nil")
} else if that1.LeaderName != nil {
return fmt23.Errorf("LeaderName this(%v) Not Equal that(%v)", this.LeaderName, that1.LeaderName)
}
if this.LastIndex != nil && that1.LastIndex != nil {
if *this.LastIndex != *that1.LastIndex {
return fmt23.Errorf("LastIndex this(%v) Not Equal that(%v)", *this.LastIndex, *that1.LastIndex)
}
} else if this.LastIndex != nil {
return fmt23.Errorf("this.LastIndex == nil && that.LastIndex != nil")
} else if that1.LastIndex != nil {
return fmt23.Errorf("LastIndex this(%v) Not Equal that(%v)", this.LastIndex, that1.LastIndex)
}
if this.LastTerm != nil && that1.LastTerm != nil {
if *this.LastTerm != *that1.LastTerm {
return fmt23.Errorf("LastTerm this(%v) Not Equal that(%v)", *this.LastTerm, *that1.LastTerm)
}
} else if this.LastTerm != nil {
return fmt23.Errorf("this.LastTerm == nil && that.LastTerm != nil")
} else if that1.LastTerm != nil {
return fmt23.Errorf("LastTerm this(%v) Not Equal that(%v)", this.LastTerm, that1.LastTerm)
}
if !bytes7.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt23.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *SnapshotRequest) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*SnapshotRequest)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.LeaderName != nil && that1.LeaderName != nil {
if *this.LeaderName != *that1.LeaderName {
return false
}
} else if this.LeaderName != nil {
return false
} else if that1.LeaderName != nil {
return false
}
if this.LastIndex != nil && that1.LastIndex != nil {
if *this.LastIndex != *that1.LastIndex {
return false
}
} else if this.LastIndex != nil {
return false
} else if that1.LastIndex != nil {
return false
}
if this.LastTerm != nil && that1.LastTerm != nil {
if *this.LastTerm != *that1.LastTerm {
return false
}
} else if this.LastTerm != nil {
return false
} else if that1.LastTerm != nil {
return false
}
if !bytes7.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}

View File

@ -1,6 +1,20 @@
package protobuf;
message ProtoSnapshotRequest {
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
option (gogoproto.gostring_all) = true;
option (gogoproto.equal_all) = true;
option (gogoproto.verbose_equal_all) = true;
option (gogoproto.goproto_stringer_all) = false;
option (gogoproto.stringer_all) = true;
option (gogoproto.populate_all) = true;
option (gogoproto.testgen_all) = true;
option (gogoproto.benchgen_all) = true;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
message SnapshotRequest {
required string LeaderName=1;
required uint64 LastIndex=2;
required uint64 LastTerm=3;

View File

@ -1,28 +1,46 @@
// Code generated by protoc-gen-go.
// Code generated by protoc-gen-gogo.
// source: snapshot_response.proto
// DO NOT EDIT!
package protobuf
import proto "code.google.com/p/goprotobuf/proto"
import proto "code.google.com/p/gogoprotobuf/proto"
import json "encoding/json"
import math "math"
// discarding unused import gogoproto "code.google.com/p/gogoprotobuf/gogoproto/gogo.pb"
import io8 "io"
import code_google_com_p_gogoprotobuf_proto16 "code.google.com/p/gogoprotobuf/proto"
import fmt24 "fmt"
import strings16 "strings"
import reflect16 "reflect"
import fmt25 "fmt"
import strings17 "strings"
import code_google_com_p_gogoprotobuf_proto17 "code.google.com/p/gogoprotobuf/proto"
import sort8 "sort"
import strconv8 "strconv"
import reflect17 "reflect"
import fmt26 "fmt"
import bytes8 "bytes"
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
var _ = proto.Marshal
var _ = &json.SyntaxError{}
var _ = math.Inf
type ProtoSnapshotResponse struct {
type SnapshotResponse struct {
Success *bool `protobuf:"varint,1,req" json:"Success,omitempty"`
XXX_unrecognized []byte `json:"-"`
}
func (m *ProtoSnapshotResponse) Reset() { *m = ProtoSnapshotResponse{} }
func (m *ProtoSnapshotResponse) String() string { return proto.CompactTextString(m) }
func (*ProtoSnapshotResponse) ProtoMessage() {}
func (m *SnapshotResponse) Reset() { *m = SnapshotResponse{} }
func (*SnapshotResponse) ProtoMessage() {}
func (m *ProtoSnapshotResponse) GetSuccess() bool {
func (m *SnapshotResponse) GetSuccess() bool {
if m != nil && m.Success != nil {
return *m.Success
}
@ -31,3 +49,338 @@ func (m *ProtoSnapshotResponse) GetSuccess() bool {
func init() {
}
func (m *SnapshotResponse) Unmarshal(data []byte) error {
l := len(data)
index := 0
for index < l {
var wire uint64
for shift := uint(0); ; shift += 7 {
if index >= l {
return io8.ErrUnexpectedEOF
}
b := data[index]
index++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
switch fieldNum {
case 1:
if wireType != 0 {
return proto.ErrWrongType
}
var v int
for shift := uint(0); ; shift += 7 {
if index >= l {
return io8.ErrUnexpectedEOF
}
b := data[index]
index++
v |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
b := bool(v != 0)
m.Success = &b
default:
var sizeOfWire int
for {
sizeOfWire++
wire >>= 7
if wire == 0 {
break
}
}
index -= sizeOfWire
skippy, err := code_google_com_p_gogoprotobuf_proto16.Skip(data[index:])
if err != nil {
return err
}
m.XXX_unrecognized = append(m.XXX_unrecognized, data[index:index+skippy]...)
index += skippy
}
}
return nil
}
func (this *SnapshotResponse) String() string {
if this == nil {
return "nil"
}
s := strings16.Join([]string{`&SnapshotResponse{`,
`Success:` + valueToStringSnapshotResponse(this.Success) + `,`,
`XXX_unrecognized:` + fmt24.Sprintf("%v", this.XXX_unrecognized) + `,`,
`}`,
}, "")
return s
}
func valueToStringSnapshotResponse(v interface{}) string {
rv := reflect16.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect16.Indirect(rv).Interface()
return fmt24.Sprintf("*%v", pv)
}
func (m *SnapshotResponse) Size() (n int) {
var l int
_ = l
if m.Success != nil {
n += 2
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
return n
}
func sovSnapshotResponse(x uint64) (n int) {
for {
n++
x >>= 7
if x == 0 {
break
}
}
return n
}
func sozSnapshotResponse(x uint64) (n int) {
return sovSnapshotResponse(uint64((x << 1) ^ uint64((int64(x) >> 63))))
return sovSnapshotResponse(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func NewPopulatedSnapshotResponse(r randySnapshotResponse, easy bool) *SnapshotResponse {
this := &SnapshotResponse{}
v1 := bool(r.Intn(2) == 0)
this.Success = &v1
if !easy && r.Intn(10) != 0 {
this.XXX_unrecognized = randUnrecognizedSnapshotResponse(r, 2)
}
return this
}
type randySnapshotResponse interface {
Float32() float32
Float64() float64
Int63() int64
Int31() int32
Uint32() uint32
Intn(n int) int
}
func randUTF8RuneSnapshotResponse(r randySnapshotResponse) rune {
res := rune(r.Uint32() % 1112064)
if 55296 <= res {
res += 2047
}
return res
}
func randStringSnapshotResponse(r randySnapshotResponse) string {
v2 := r.Intn(100)
tmps := make([]rune, v2)
for i := 0; i < v2; i++ {
tmps[i] = randUTF8RuneSnapshotResponse(r)
}
return string(tmps)
}
func randUnrecognizedSnapshotResponse(r randySnapshotResponse, maxFieldNumber int) (data []byte) {
l := r.Intn(5)
for i := 0; i < l; i++ {
wire := r.Intn(4)
if wire == 3 {
wire = 5
}
fieldNumber := maxFieldNumber + r.Intn(100)
data = randFieldSnapshotResponse(data, r, fieldNumber, wire)
}
return data
}
func randFieldSnapshotResponse(data []byte, r randySnapshotResponse, fieldNumber int, wire int) []byte {
key := uint32(fieldNumber)<<3 | uint32(wire)
switch wire {
case 0:
data = encodeVarintPopulateSnapshotResponse(data, uint64(key))
data = encodeVarintPopulateSnapshotResponse(data, uint64(r.Int63()))
case 1:
data = encodeVarintPopulateSnapshotResponse(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
case 2:
data = encodeVarintPopulateSnapshotResponse(data, uint64(key))
ll := r.Intn(100)
data = encodeVarintPopulateSnapshotResponse(data, uint64(ll))
for j := 0; j < ll; j++ {
data = append(data, byte(r.Intn(256)))
}
default:
data = encodeVarintPopulateSnapshotResponse(data, uint64(key))
data = append(data, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)))
}
return data
}
func encodeVarintPopulateSnapshotResponse(data []byte, v uint64) []byte {
for v >= 1<<7 {
data = append(data, uint8(uint64(v)&0x7f|0x80))
v >>= 7
}
data = append(data, uint8(v))
return data
}
func (m *SnapshotResponse) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
if err != nil {
return nil, err
}
return data[:n], nil
}
func (m *SnapshotResponse) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
_ = l
if m.Success != nil {
data[i] = 0x8
i++
if *m.Success {
data[i] = 1
} else {
data[i] = 0
}
i++
}
if m.XXX_unrecognized != nil {
i += copy(data[i:], m.XXX_unrecognized)
}
return i, nil
}
func encodeFixed64SnapshotResponse(data []byte, offset int, v uint64) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
data[offset+4] = uint8(v >> 32)
data[offset+5] = uint8(v >> 40)
data[offset+6] = uint8(v >> 48)
data[offset+7] = uint8(v >> 56)
return offset + 8
}
func encodeFixed32SnapshotResponse(data []byte, offset int, v uint32) int {
data[offset] = uint8(v)
data[offset+1] = uint8(v >> 8)
data[offset+2] = uint8(v >> 16)
data[offset+3] = uint8(v >> 24)
return offset + 4
}
func encodeVarintSnapshotResponse(data []byte, offset int, v uint64) int {
for v >= 1<<7 {
data[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
data[offset] = uint8(v)
return offset + 1
}
func (this *SnapshotResponse) GoString() string {
if this == nil {
return "nil"
}
s := strings17.Join([]string{`&protobuf.SnapshotResponse{` + `Success:` + valueToGoStringSnapshotResponse(this.Success, "bool"), `XXX_unrecognized:` + fmt25.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
return s
}
func valueToGoStringSnapshotResponse(v interface{}, typ string) string {
rv := reflect17.ValueOf(v)
if rv.IsNil() {
return "nil"
}
pv := reflect17.Indirect(rv).Interface()
return fmt25.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
}
func extensionToGoStringSnapshotResponse(e map[int32]code_google_com_p_gogoprotobuf_proto17.Extension) string {
if e == nil {
return "nil"
}
s := "map[int32]proto.Extension{"
keys := make([]int, 0, len(e))
for k := range e {
keys = append(keys, int(k))
}
sort8.Ints(keys)
ss := []string{}
for _, k := range keys {
ss = append(ss, strconv8.Itoa(k)+": "+e[int32(k)].GoString())
}
s += strings17.Join(ss, ",") + "}"
return s
}
func (this *SnapshotResponse) VerboseEqual(that interface{}) error {
if that == nil {
if this == nil {
return nil
}
return fmt26.Errorf("that == nil && this != nil")
}
that1, ok := that.(*SnapshotResponse)
if !ok {
return fmt26.Errorf("that is not of type *SnapshotResponse")
}
if that1 == nil {
if this == nil {
return nil
}
return fmt26.Errorf("that is type *SnapshotResponse but is nil && this != nil")
} else if this == nil {
return fmt26.Errorf("that is type *SnapshotResponsebut is not nil && this == nil")
}
if this.Success != nil && that1.Success != nil {
if *this.Success != *that1.Success {
return fmt26.Errorf("Success this(%v) Not Equal that(%v)", *this.Success, *that1.Success)
}
} else if this.Success != nil {
return fmt26.Errorf("this.Success == nil && that.Success != nil")
} else if that1.Success != nil {
return fmt26.Errorf("Success this(%v) Not Equal that(%v)", this.Success, that1.Success)
}
if !bytes8.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return fmt26.Errorf("XXX_unrecognized this(%v) Not Equal that(%v)", this.XXX_unrecognized, that1.XXX_unrecognized)
}
return nil
}
func (this *SnapshotResponse) Equal(that interface{}) bool {
if that == nil {
if this == nil {
return true
}
return false
}
that1, ok := that.(*SnapshotResponse)
if !ok {
return false
}
if that1 == nil {
if this == nil {
return true
}
return false
} else if this == nil {
return false
}
if this.Success != nil && that1.Success != nil {
if *this.Success != *that1.Success {
return false
}
} else if this.Success != nil {
return false
} else if that1.Success != nil {
return false
}
if !bytes8.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
return false
}
return true
}

View File

@ -1,5 +1,19 @@
package protobuf;
message ProtoSnapshotResponse {
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
option (gogoproto.gostring_all) = true;
option (gogoproto.equal_all) = true;
option (gogoproto.verbose_equal_all) = true;
option (gogoproto.goproto_stringer_all) = false;
option (gogoproto.stringer_all) = true;
option (gogoproto.populate_all) = true;
option (gogoproto.testgen_all) = true;
option (gogoproto.benchgen_all) = true;
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
message SnapshotResponse {
required bool Success=1;
}

View File

@ -1,10 +1,11 @@
package raft
import (
"code.google.com/p/goprotobuf/proto"
"github.com/goraft/raft/protobuf"
"io"
"io/ioutil"
"code.google.com/p/gogoprotobuf/proto"
"github.com/goraft/raft/protobuf"
)
// The request sent to a server to vote for a candidate to become a leader.
@ -29,7 +30,7 @@ func newRequestVoteRequest(term uint64, candidateName string, lastLogIndex uint6
// Encodes the RequestVoteRequest to a buffer. Returns the number of bytes
// written and any error that may have occurred.
func (req *RequestVoteRequest) Encode(w io.Writer) (int, error) {
pb := &protobuf.ProtoRequestVoteRequest{
pb := &protobuf.RequestVoteRequest{
Term: proto.Uint64(req.Term),
LastLogIndex: proto.Uint64(req.LastLogIndex),
LastLogTerm: proto.Uint64(req.LastLogTerm),
@ -54,7 +55,7 @@ func (req *RequestVoteRequest) Decode(r io.Reader) (int, error) {
totalBytes := len(data)
pb := &protobuf.ProtoRequestVoteRequest{}
pb := &protobuf.RequestVoteRequest{}
if err = proto.Unmarshal(data, pb); err != nil {
return -1, err
}

View File

@ -1,10 +1,11 @@
package raft
import (
"code.google.com/p/goprotobuf/proto"
"github.com/goraft/raft/protobuf"
"io"
"io/ioutil"
"code.google.com/p/gogoprotobuf/proto"
"github.com/goraft/raft/protobuf"
)
// The response returned from a server after a vote for a candidate to become a leader.
@ -25,7 +26,7 @@ func newRequestVoteResponse(term uint64, voteGranted bool) *RequestVoteResponse
// Encodes the RequestVoteResponse to a buffer. Returns the number of bytes
// written and any error that may have occurred.
func (resp *RequestVoteResponse) Encode(w io.Writer) (int, error) {
pb := &protobuf.ProtoRequestVoteResponse{
pb := &protobuf.RequestVoteResponse{
Term: proto.Uint64(resp.Term),
VoteGranted: proto.Bool(resp.VoteGranted),
}
@ -49,7 +50,7 @@ func (resp *RequestVoteResponse) Decode(r io.Reader) (int, error) {
totalBytes := len(data)
pb := &protobuf.ProtoRequestVoteResponse{}
pb := &protobuf.RequestVoteResponse{}
if err = proto.Unmarshal(data, pb); err != nil {
return -1, err
}

View File

@ -33,7 +33,7 @@ const (
)
const (
DefaultHeartbeatTimeout = 50 * time.Millisecond
DefaultHeartbeatTimeout = 1 * time.Millisecond
DefaultElectionTimeout = 150 * time.Millisecond
)
@ -926,13 +926,13 @@ func (s *server) processAppendEntriesRequest(req *AppendEntriesRequest) (*Append
func (s *server) processAppendEntriesResponse(resp *AppendEntriesResponse) {
// If we find a higher term then change to a follower and exit.
if resp.Term > s.Term() {
s.setCurrentTerm(resp.Term, "", false)
if resp.Term() > s.Term() {
s.setCurrentTerm(resp.Term(), "", false)
return
}
// panic response if it's not successful.
if !resp.Success {
if !resp.Success() {
return
}
@ -1118,9 +1118,9 @@ func (s *server) TakeSnapshot() error {
// We keep some log entries after the snapshot.
// We do not want to send the whole snapshot to the slightly slow machines
if lastIndex - s.log.startIndex > NumberOfLogEntriesAfterSnapshot {
if lastIndex-s.log.startIndex > NumberOfLogEntriesAfterSnapshot {
compactIndex := lastIndex - NumberOfLogEntriesAfterSnapshot
compactTerm := s.log.getEntry(compactIndex).Term
compactTerm := s.log.getEntry(compactIndex).Term()
s.log.compact(compactIndex, compactTerm)
}
@ -1167,7 +1167,8 @@ func (s *server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse
// that matches the snapshots last term, then the follower already has all the
// information found in the snapshot and can reply false.
entry := s.log.getEntry(req.LastIndex)
if entry != nil && entry.Term == req.LastTerm {
if entry != nil && entry.Term() == req.LastTerm {
return newSnapshotResponse(false)
}

View File

@ -1,9 +1,9 @@
package raft
import (
"bytes"
"encoding/json"
"fmt"
"reflect"
"strconv"
"sync"
"testing"
@ -201,7 +201,7 @@ func TestServerAppendEntries(t *testing.T) {
e, _ := newLogEntry(nil, nil, 1, 1, &testCommand1{Val: "foo", I: 10})
entries := []*LogEntry{e}
resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries))
if resp.Term != 1 || !resp.Success {
if resp.Term() != 1 || !resp.Success() {
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
}
if index, term := s.(*server).log.commitInfo(); index != 0 || term != 0 {
@ -213,7 +213,7 @@ func TestServerAppendEntries(t *testing.T) {
e2, _ := newLogEntry(nil, nil, 3, 1, &testCommand1{Val: "baz", I: 30})
entries = []*LogEntry{e1, e2}
resp = s.AppendEntries(newAppendEntriesRequest(1, 1, 1, 1, "ldr", entries))
if resp.Term != 1 || !resp.Success {
if resp.Term() != 1 || !resp.Success() {
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
}
if index, term := s.(*server).log.commitInfo(); index != 1 || term != 1 {
@ -222,7 +222,7 @@ func TestServerAppendEntries(t *testing.T) {
// Send zero entries and commit everything.
resp = s.AppendEntries(newAppendEntriesRequest(2, 3, 1, 3, "ldr", []*LogEntry{}))
if resp.Term != 2 || !resp.Success {
if resp.Term() != 2 || !resp.Success() {
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
}
if index, term := s.(*server).log.commitInfo(); index != 3 || term != 1 {
@ -245,7 +245,7 @@ func TestServerAppendEntriesWithStaleTermsAreRejected(t *testing.T) {
e, _ := newLogEntry(nil, nil, 1, 1, &testCommand1{Val: "foo", I: 10})
entries := []*LogEntry{e}
resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries))
if resp.Term != 2 || resp.Success {
if resp.Term() != 2 || resp.Success() {
t.Fatalf("AppendEntries should have failed: %v/%v", resp.Term, resp.Success)
}
if index, term := s.(*server).log.commitInfo(); index != 0 || term != 0 {
@ -264,7 +264,7 @@ func TestServerAppendEntriesRejectedIfAlreadyCommitted(t *testing.T) {
e2, _ := newLogEntry(nil, nil, 2, 1, &testCommand1{Val: "foo", I: 15})
entries := []*LogEntry{e1, e2}
resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 2, "ldr", entries))
if resp.Term != 1 || !resp.Success {
if resp.Term() != 1 || !resp.Success() {
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
}
@ -272,7 +272,7 @@ func TestServerAppendEntriesRejectedIfAlreadyCommitted(t *testing.T) {
e, _ := newLogEntry(nil, nil, 2, 1, &testCommand1{Val: "bar", I: 20})
entries = []*LogEntry{e}
resp = s.AppendEntries(newAppendEntriesRequest(1, 2, 1, 1, "ldr", entries))
if resp.Term != 1 || resp.Success {
if resp.Term() != 1 || resp.Success() {
t.Fatalf("AppendEntries should have failed: %v/%v", resp.Term, resp.Success)
}
}
@ -283,23 +283,36 @@ func TestServerAppendEntriesOverwritesUncommittedEntries(t *testing.T) {
s.Start()
defer s.Stop()
entry1, _ := newLogEntry(nil, nil, 1, 1, &testCommand1{Val: "foo", I: 10})
entry2, _ := newLogEntry(nil, nil, 2, 1, &testCommand1{Val: "foo", I: 15})
entry3, _ := newLogEntry(nil, nil, 2, 2, &testCommand1{Val: "bar", I: 20})
entry1, _ := newLogEntry(s.(*server).log, nil, 1, 1, &testCommand1{Val: "foo", I: 10})
entry2, _ := newLogEntry(s.(*server).log, nil, 2, 1, &testCommand1{Val: "foo", I: 15})
entry3, _ := newLogEntry(s.(*server).log, nil, 2, 2, &testCommand1{Val: "bar", I: 20})
// Append single entry + commit.
entries := []*LogEntry{entry1, entry2}
resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 1, "ldr", entries))
if resp.Term != 1 || !resp.Success || s.(*server).log.commitIndex != 1 || !reflect.DeepEqual(s.(*server).log.entries, []*LogEntry{entry1, entry2}) {
if resp.Term() != 1 || !resp.Success() || s.(*server).log.commitIndex != 1 {
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
}
for i, entry := range s.(*server).log.entries {
if entry.Term() != entries[i].Term() || entry.Index() != entries[i].Index() || !bytes.Equal(entry.Command(), entries[i].Command()) {
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
}
}
// Append entry that overwrites the second (uncommitted) entry.
entries = []*LogEntry{entry3}
resp = s.AppendEntries(newAppendEntriesRequest(2, 1, 1, 2, "ldr", entries))
if resp.Term != 2 || !resp.Success || s.(*server).log.commitIndex != 2 || !reflect.DeepEqual(s.(*server).log.entries, []*LogEntry{entry1, entry3}) {
if resp.Term() != 2 || !resp.Success() || s.(*server).log.commitIndex != 2 {
t.Fatalf("AppendEntries should have succeeded: %v/%v", resp.Term, resp.Success)
}
entries = []*LogEntry{entry1, entry3}
for i, entry := range s.(*server).log.entries {
if entry.Term() != entries[i].Term() || entry.Index() != entries[i].Index() || !bytes.Equal(entry.Command(), entries[i].Command()) {
t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success)
}
}
}
//--------------------------------------

View File

@ -1,10 +1,11 @@
package raft
import (
"code.google.com/p/goprotobuf/proto"
"github.com/goraft/raft/protobuf"
"io"
"io/ioutil"
"code.google.com/p/gogoprotobuf/proto"
"github.com/goraft/raft/protobuf"
)
// The request sent to a server to start from the snapshot.
@ -31,16 +32,16 @@ func newSnapshotRecoveryRequest(leaderName string, snapshot *Snapshot) *Snapshot
// written and any error that may have occurred.
func (req *SnapshotRecoveryRequest) Encode(w io.Writer) (int, error) {
protoPeers := make([]*protobuf.ProtoSnapshotRecoveryRequest_ProtoPeer, len(req.Peers))
protoPeers := make([]*protobuf.SnapshotRecoveryRequest_Peer, len(req.Peers))
for i, peer := range req.Peers {
protoPeers[i] = &protobuf.ProtoSnapshotRecoveryRequest_ProtoPeer{
protoPeers[i] = &protobuf.SnapshotRecoveryRequest_Peer{
Name: proto.String(peer.Name),
ConnectionString: proto.String(peer.ConnectionString),
}
}
pb := &protobuf.ProtoSnapshotRecoveryRequest{
pb := &protobuf.SnapshotRecoveryRequest{
LeaderName: proto.String(req.LeaderName),
LastIndex: proto.Uint64(req.LastIndex),
LastTerm: proto.Uint64(req.LastTerm),
@ -66,7 +67,7 @@ func (req *SnapshotRecoveryRequest) Decode(r io.Reader) (int, error) {
totalBytes := len(data)
pb := &protobuf.ProtoSnapshotRecoveryRequest{}
pb := &protobuf.SnapshotRecoveryRequest{}
if err = proto.Unmarshal(data, pb); err != nil {
return -1, err
}

View File

@ -1,10 +1,11 @@
package raft
import (
"code.google.com/p/goprotobuf/proto"
"github.com/goraft/raft/protobuf"
"io"
"io/ioutil"
"code.google.com/p/gogoprotobuf/proto"
"github.com/goraft/raft/protobuf"
)
// The response returned from a server appending entries to the log.
@ -26,7 +27,7 @@ func newSnapshotRecoveryResponse(term uint64, success bool, commitIndex uint64)
// Encode writes the response to a writer.
// Returns the number of bytes written and any error that occurs.
func (req *SnapshotRecoveryResponse) Encode(w io.Writer) (int, error) {
pb := &protobuf.ProtoSnapshotRecoveryResponse{
pb := &protobuf.SnapshotRecoveryResponse{
Term: proto.Uint64(req.Term),
Success: proto.Bool(req.Success),
CommitIndex: proto.Uint64(req.CommitIndex),
@ -49,7 +50,7 @@ func (req *SnapshotRecoveryResponse) Decode(r io.Reader) (int, error) {
totalBytes := len(data)
pb := &protobuf.ProtoSnapshotRecoveryResponse{}
pb := &protobuf.SnapshotRecoveryResponse{}
if err := proto.Unmarshal(data, pb); err != nil {
return -1, err
}

View File

@ -1,10 +1,11 @@
package raft
import (
"code.google.com/p/goprotobuf/proto"
"github.com/goraft/raft/protobuf"
"io"
"io/ioutil"
"code.google.com/p/gogoprotobuf/proto"
"github.com/goraft/raft/protobuf"
)
// The request sent to a server to start from the snapshot.
@ -26,7 +27,7 @@ func newSnapshotRequest(leaderName string, snapshot *Snapshot) *SnapshotRequest
// Encodes the SnapshotRequest to a buffer. Returns the number of bytes
// written and any error that may have occurred.
func (req *SnapshotRequest) Encode(w io.Writer) (int, error) {
pb := &protobuf.ProtoSnapshotRequest{
pb := &protobuf.SnapshotRequest{
LeaderName: proto.String(req.LeaderName),
LastIndex: proto.Uint64(req.LastIndex),
LastTerm: proto.Uint64(req.LastTerm),
@ -50,7 +51,7 @@ func (req *SnapshotRequest) Decode(r io.Reader) (int, error) {
totalBytes := len(data)
pb := &protobuf.ProtoSnapshotRequest{}
pb := &protobuf.SnapshotRequest{}
if err := proto.Unmarshal(data, pb); err != nil {
return -1, err

View File

@ -1,10 +1,11 @@
package raft
import (
"code.google.com/p/goprotobuf/proto"
"github.com/goraft/raft/protobuf"
"io"
"io/ioutil"
"code.google.com/p/gogoprotobuf/proto"
"github.com/goraft/raft/protobuf"
)
// The response returned if the follower entered snapshot state
@ -22,7 +23,7 @@ func newSnapshotResponse(success bool) *SnapshotResponse {
// Encodes the SnapshotResponse to a buffer. Returns the number of bytes
// written and any error that may have occurred.
func (resp *SnapshotResponse) Encode(w io.Writer) (int, error) {
pb := &protobuf.ProtoSnapshotResponse{
pb := &protobuf.SnapshotResponse{
Success: proto.Bool(resp.Success),
}
p, err := proto.Marshal(pb)
@ -44,7 +45,7 @@ func (resp *SnapshotResponse) Decode(r io.Reader) (int, error) {
totalBytes := len(data)
pb := &protobuf.ProtoSnapshotResponse{}
pb := &protobuf.SnapshotResponse{}
if err := proto.Unmarshal(data, pb); err != nil {
return -1, err
}