refactor do not copy entry around when doing encoding
parent
f6b1cc1407
commit
8ce022418d
|
@ -15,43 +15,38 @@ type AppendEntriesRequest struct {
|
|||
PrevLogTerm uint64
|
||||
CommitIndex uint64
|
||||
LeaderName string
|
||||
Entries []*LogEntry
|
||||
Entries []*protobuf.ProtoLogEntry
|
||||
}
|
||||
|
||||
// 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.ProtoLogEntry, 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{
|
||||
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)
|
||||
|
@ -83,17 +78,7 @@ func (req *AppendEntriesRequest) Decode(r io.Reader) (int, error) {
|
|||
req.PrevLogTerm = pb.GetPrevLogTerm()
|
||||
req.CommitIndex = pb.GetCommitIndex()
|
||||
req.LeaderName = pb.GetLeaderName()
|
||||
|
||||
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,
|
||||
}
|
||||
}
|
||||
req.Entries = pb.GetEntries()
|
||||
|
||||
return totalBytes, nil
|
||||
}
|
||||
|
|
|
@ -69,7 +69,6 @@ func RegisterCommand(command Command) {
|
|||
panic(fmt.Sprintf("raft: Cannot register nil"))
|
||||
} else if commandTypes[command.CommandName()] != nil {
|
||||
panic(fmt.Sprintf("raft: Duplicate registration: %s", command.CommandName()))
|
||||
return
|
||||
}
|
||||
commandTypes[command.CommandName()] = command
|
||||
}
|
||||
|
|
65
log.go
65
log.go
|
@ -79,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.
|
||||
|
@ -100,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 ""
|
||||
|
@ -118,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()
|
||||
}
|
||||
|
||||
//------------------------------------------------------------------------------
|
||||
|
@ -173,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)
|
||||
|
@ -228,7 +228,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
|
||||
|
@ -264,9 +264,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()
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -291,7 +291,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.
|
||||
|
@ -306,7 +306,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
|
||||
|
@ -356,17 +356,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 entires index: ", i, entryIndex)
|
||||
debugf("setCommitIndex.set.result index: %v entires index: %v", i, entryIndex)
|
||||
if entry.event != nil {
|
||||
entry.event.returnValue = returnValue
|
||||
entry.event.c <- err
|
||||
|
@ -423,9 +423,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.
|
||||
|
@ -455,7 +455,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.ProtoLogEntry) error {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
|
||||
|
@ -466,12 +466,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()
|
||||
|
@ -496,10 +501,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())
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -527,10 +532,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())
|
||||
}
|
||||
}
|
||||
|
||||
|
|
58
log_entry.go
58
log_entry.go
|
@ -12,13 +12,10 @@ import (
|
|||
|
||||
// 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.ProtoLogEntry
|
||||
Position int64 // position in the log file
|
||||
log *Log
|
||||
event *ev
|
||||
}
|
||||
|
||||
// Creates a new log entry associated with a log.
|
||||
|
@ -36,27 +33,42 @@ func newLogEntry(log *Log, event *ev, index uint64, term uint64, command Command
|
|||
}
|
||||
}
|
||||
|
||||
e := &LogEntry{
|
||||
log: log,
|
||||
Index: index,
|
||||
Term: term,
|
||||
CommandName: commandName,
|
||||
pb := &protobuf.ProtoLogEntry{
|
||||
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) {
|
||||
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
|
||||
|
||||
b, err := proto.Marshal(e.log.pLogEntry)
|
||||
b, err := proto.Marshal(e.pb)
|
||||
if err != nil {
|
||||
return -1, err
|
||||
}
|
||||
|
@ -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, nil
|
||||
}
|
||||
|
|
18
log_test.go
18
log_test.go
|
@ -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])
|
||||
}
|
||||
}
|
||||
|
|
4
peer.go
4
peer.go
|
@ -177,11 +177,11 @@ func (p *Peer) sendAppendEntriesRequest(req *AppendEntriesRequest) {
|
|||
p.mutex.Lock()
|
||||
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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -10,22 +10,22 @@ 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 io1 "io"
|
||||
import code_google_com_p_gogoprotobuf_proto2 "code.google.com/p/gogoprotobuf/proto"
|
||||
|
||||
import fmt "fmt"
|
||||
import strings "strings"
|
||||
import reflect "reflect"
|
||||
import fmt3 "fmt"
|
||||
import strings2 "strings"
|
||||
import reflect2 "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 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 fmt2 "fmt"
|
||||
import bytes "bytes"
|
||||
import fmt5 "fmt"
|
||||
import bytes1 "bytes"
|
||||
|
||||
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
|
@ -33,13 +33,13 @@ 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:"-"`
|
||||
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 []*ProtoLogEntry `protobuf:"bytes,6,rep" json:"Entries,omitempty"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
}
|
||||
|
||||
func (m *ProtoAppendEntriesRequest) Reset() { *m = ProtoAppendEntriesRequest{} }
|
||||
|
@ -80,54 +80,13 @@ func (m *ProtoAppendEntriesRequest) GetLeaderName() string {
|
|||
return ""
|
||||
}
|
||||
|
||||
func (m *ProtoAppendEntriesRequest) GetEntries() []*ProtoAppendEntriesRequest_ProtoLogEntry {
|
||||
func (m *ProtoAppendEntriesRequest) GetEntries() []*ProtoLogEntry {
|
||||
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 (m *ProtoAppendEntriesRequest_ProtoLogEntry) Reset() {
|
||||
*m = ProtoAppendEntriesRequest_ProtoLogEntry{}
|
||||
}
|
||||
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
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func init() {
|
||||
}
|
||||
func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
||||
|
@ -137,7 +96,7 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
return io1.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -156,7 +115,7 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
return io1.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -173,7 +132,7 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
return io1.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -190,7 +149,7 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
return io1.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -207,7 +166,7 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
return io1.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -224,7 +183,7 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
var stringLen uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
return io1.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -235,7 +194,7 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
}
|
||||
postIndex := index + int(stringLen)
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
return io1.ErrUnexpectedEOF
|
||||
}
|
||||
s := string(data[index:postIndex])
|
||||
m.LeaderName = &s
|
||||
|
@ -247,7 +206,7 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
return io1.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -258,9 +217,9 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
}
|
||||
postIndex := index + msglen
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
return io1.ErrUnexpectedEOF
|
||||
}
|
||||
m.Entries = append(m.Entries, &ProtoAppendEntriesRequest_ProtoLogEntry{})
|
||||
m.Entries = append(m.Entries, &ProtoLogEntry{})
|
||||
m.Entries[len(m.Entries)-1].Unmarshal(data[index:postIndex])
|
||||
index = postIndex
|
||||
default:
|
||||
|
@ -273,125 +232,7 @@ func (m *ProtoAppendEntriesRequest) Unmarshal(data []byte) error {
|
|||
}
|
||||
}
|
||||
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 (m *ProtoAppendEntriesRequest_ProtoLogEntry) 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:])
|
||||
skippy, err := code_google_com_p_gogoprotobuf_proto2.Skip(data[index:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -405,39 +246,25 @@ func (this *ProtoAppendEntriesRequest) String() string {
|
|||
if this == nil {
|
||||
return "nil"
|
||||
}
|
||||
s := strings.Join([]string{`&ProtoAppendEntriesRequest{`,
|
||||
s := strings2.Join([]string{`&ProtoAppendEntriesRequest{`,
|
||||
`Term:` + valueToStringAppendEntriesRequest(this.Term) + `,`,
|
||||
`PrevLogIndex:` + valueToStringAppendEntriesRequest(this.PrevLogIndex) + `,`,
|
||||
`PrevLogTerm:` + valueToStringAppendEntriesRequest(this.PrevLogTerm) + `,`,
|
||||
`CommitIndex:` + valueToStringAppendEntriesRequest(this.CommitIndex) + `,`,
|
||||
`LeaderName:` + valueToStringAppendEntriesRequest(this.LeaderName) + `,`,
|
||||
`Entries:` + strings.Replace(fmt.Sprintf("%v", this.Entries), "ProtoAppendEntriesRequest_ProtoLogEntry", "ProtoAppendEntriesRequest_ProtoLogEntry", 1) + `,`,
|
||||
`XXX_unrecognized:` + fmt.Sprintf("%v", this.XXX_unrecognized) + `,`,
|
||||
`}`,
|
||||
}, "")
|
||||
return s
|
||||
}
|
||||
func (this *ProtoAppendEntriesRequest_ProtoLogEntry) String() string {
|
||||
if this == nil {
|
||||
return "nil"
|
||||
}
|
||||
s := strings.Join([]string{`&ProtoAppendEntriesRequest_ProtoLogEntry{`,
|
||||
`Index:` + valueToStringAppendEntriesRequest(this.Index) + `,`,
|
||||
`Term:` + valueToStringAppendEntriesRequest(this.Term) + `,`,
|
||||
`CommandName:` + valueToStringAppendEntriesRequest(this.CommandName) + `,`,
|
||||
`Command:` + valueToStringAppendEntriesRequest(this.Command) + `,`,
|
||||
`XXX_unrecognized:` + fmt.Sprintf("%v", this.XXX_unrecognized) + `,`,
|
||||
`Entries:` + strings2.Replace(fmt3.Sprintf("%v", this.Entries), "ProtoLogEntry", "ProtoLogEntry", 1) + `,`,
|
||||
`XXX_unrecognized:` + fmt3.Sprintf("%v", this.XXX_unrecognized) + `,`,
|
||||
`}`,
|
||||
}, "")
|
||||
return s
|
||||
}
|
||||
func valueToStringAppendEntriesRequest(v interface{}) string {
|
||||
rv := reflect.ValueOf(v)
|
||||
rv := reflect2.ValueOf(v)
|
||||
if rv.IsNil() {
|
||||
return "nil"
|
||||
}
|
||||
pv := reflect.Indirect(rv).Interface()
|
||||
return fmt.Sprintf("*%v", pv)
|
||||
pv := reflect2.Indirect(rv).Interface()
|
||||
return fmt3.Sprintf("*%v", pv)
|
||||
}
|
||||
func (m *ProtoAppendEntriesRequest) Size() (n int) {
|
||||
var l int
|
||||
|
@ -469,28 +296,6 @@ func (m *ProtoAppendEntriesRequest) Size() (n int) {
|
|||
}
|
||||
return n
|
||||
}
|
||||
func (m *ProtoAppendEntriesRequest_ProtoLogEntry) Size() (n int) {
|
||||
var l int
|
||||
_ = l
|
||||
if m.Index != nil {
|
||||
n += 1 + sovAppendEntriesRequest(uint64(*m.Index))
|
||||
}
|
||||
if m.Term != nil {
|
||||
n += 1 + sovAppendEntriesRequest(uint64(*m.Term))
|
||||
}
|
||||
if m.CommandName != nil {
|
||||
l = len(*m.CommandName)
|
||||
n += 1 + l + sovAppendEntriesRequest(uint64(l))
|
||||
}
|
||||
if m.Command != nil {
|
||||
l = len(m.Command)
|
||||
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 {
|
||||
|
@ -520,9 +325,9 @@ func NewPopulatedProtoAppendEntriesRequest(r randyAppendEntriesRequest, easy boo
|
|||
this.LeaderName = &v5
|
||||
if r.Intn(10) != 0 {
|
||||
v6 := r.Intn(10)
|
||||
this.Entries = make([]*ProtoAppendEntriesRequest_ProtoLogEntry, v6)
|
||||
this.Entries = make([]*ProtoLogEntry, v6)
|
||||
for i := 0; i < v6; i++ {
|
||||
this.Entries[i] = NewPopulatedProtoAppendEntriesRequest_ProtoLogEntry(r, easy)
|
||||
this.Entries[i] = NewPopulatedProtoLogEntry(r, easy)
|
||||
}
|
||||
}
|
||||
if !easy && r.Intn(10) != 0 {
|
||||
|
@ -531,27 +336,6 @@ func NewPopulatedProtoAppendEntriesRequest(r randyAppendEntriesRequest, easy boo
|
|||
return this
|
||||
}
|
||||
|
||||
func NewPopulatedProtoAppendEntriesRequest_ProtoLogEntry(r randyAppendEntriesRequest, easy bool) *ProtoAppendEntriesRequest_ProtoLogEntry {
|
||||
this := &ProtoAppendEntriesRequest_ProtoLogEntry{}
|
||||
v7 := uint64(r.Uint32())
|
||||
this.Index = &v7
|
||||
v8 := uint64(r.Uint32())
|
||||
this.Term = &v8
|
||||
v9 := randStringAppendEntriesRequest(r)
|
||||
this.CommandName = &v9
|
||||
if r.Intn(10) != 0 {
|
||||
v10 := r.Intn(100)
|
||||
this.Command = make([]byte, v10)
|
||||
for i := 0; i < v10; i++ {
|
||||
this.Command[i] = byte(r.Intn(256))
|
||||
}
|
||||
}
|
||||
if !easy && r.Intn(10) != 0 {
|
||||
this.XXX_unrecognized = randUnrecognizedAppendEntriesRequest(r, 5)
|
||||
}
|
||||
return this
|
||||
}
|
||||
|
||||
type randyAppendEntriesRequest interface {
|
||||
Float32() float32
|
||||
Float64() float64
|
||||
|
@ -569,9 +353,9 @@ func randUTF8RuneAppendEntriesRequest(r randyAppendEntriesRequest) rune {
|
|||
return res
|
||||
}
|
||||
func randStringAppendEntriesRequest(r randyAppendEntriesRequest) string {
|
||||
v11 := r.Intn(100)
|
||||
tmps := make([]rune, v11)
|
||||
for i := 0; i < v11; i++ {
|
||||
v7 := r.Intn(100)
|
||||
tmps := make([]rune, v7)
|
||||
for i := 0; i < v7; i++ {
|
||||
tmps[i] = randUTF8RuneAppendEntriesRequest(r)
|
||||
}
|
||||
return string(tmps)
|
||||
|
@ -676,48 +460,6 @@ func (m *ProtoAppendEntriesRequest) MarshalTo(data []byte) (n int, err error) {
|
|||
}
|
||||
return i, nil
|
||||
}
|
||||
func (m *ProtoAppendEntriesRequest_ProtoLogEntry) 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 *ProtoAppendEntriesRequest_ProtoLogEntry) MarshalTo(data []byte) (n int, err error) {
|
||||
var i int
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if m.Index != nil {
|
||||
data[i] = 0x8
|
||||
i++
|
||||
i = encodeVarintAppendEntriesRequest(data, i, uint64(*m.Index))
|
||||
}
|
||||
if m.Term != nil {
|
||||
data[i] = 0x10
|
||||
i++
|
||||
i = encodeVarintAppendEntriesRequest(data, i, uint64(*m.Term))
|
||||
}
|
||||
if m.CommandName != nil {
|
||||
data[i] = 0x1a
|
||||
i++
|
||||
i = encodeVarintAppendEntriesRequest(data, i, uint64(len(*m.CommandName)))
|
||||
i += copy(data[i:], *m.CommandName)
|
||||
}
|
||||
if m.Command != nil {
|
||||
data[i] = 0x22
|
||||
i++
|
||||
i = encodeVarintAppendEntriesRequest(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 encodeFixed64AppendEntriesRequest(data []byte, offset int, v uint64) int {
|
||||
data[offset] = uint8(v)
|
||||
data[offset+1] = uint8(v >> 8)
|
||||
|
@ -749,25 +491,18 @@ func (this *ProtoAppendEntriesRequest) GoString() string {
|
|||
if this == nil {
|
||||
return "nil"
|
||||
}
|
||||
s := strings1.Join([]string{`&protobuf.ProtoAppendEntriesRequest{` + `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:` + fmt1.Sprintf("%#v", this.Entries), `XXX_unrecognized:` + fmt1.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
|
||||
return s
|
||||
}
|
||||
func (this *ProtoAppendEntriesRequest_ProtoLogEntry) GoString() string {
|
||||
if this == nil {
|
||||
return "nil"
|
||||
}
|
||||
s := strings1.Join([]string{`&protobuf.ProtoAppendEntriesRequest_ProtoLogEntry{` + `Index:` + valueToGoStringAppendEntriesRequest(this.Index, "uint64"), `Term:` + valueToGoStringAppendEntriesRequest(this.Term, "uint64"), `CommandName:` + valueToGoStringAppendEntriesRequest(this.CommandName, "string"), `Command:` + valueToGoStringAppendEntriesRequest(this.Command, "byte"), `XXX_unrecognized:` + fmt1.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
|
||||
s := strings3.Join([]string{`&protobuf.ProtoAppendEntriesRequest{` + `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 := reflect1.ValueOf(v)
|
||||
rv := reflect3.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)
|
||||
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_proto1.Extension) string {
|
||||
func extensionToGoStringAppendEntriesRequest(e map[int32]code_google_com_p_gogoprotobuf_proto3.Extension) string {
|
||||
if e == nil {
|
||||
return "nil"
|
||||
}
|
||||
|
@ -776,12 +511,12 @@ func extensionToGoStringAppendEntriesRequest(e map[int32]code_google_com_p_gogop
|
|||
for k := range e {
|
||||
keys = append(keys, int(k))
|
||||
}
|
||||
sort.Ints(keys)
|
||||
sort1.Ints(keys)
|
||||
ss := []string{}
|
||||
for _, k := range keys {
|
||||
ss = append(ss, strconv.Itoa(k)+": "+e[int32(k)].GoString())
|
||||
ss = append(ss, strconv1.Itoa(k)+": "+e[int32(k)].GoString())
|
||||
}
|
||||
s += strings1.Join(ss, ",") + "}"
|
||||
s += strings3.Join(ss, ",") + "}"
|
||||
return s
|
||||
}
|
||||
func (this *ProtoAppendEntriesRequest) VerboseEqual(that interface{}) error {
|
||||
|
@ -789,76 +524,76 @@ func (this *ProtoAppendEntriesRequest) VerboseEqual(that interface{}) error {
|
|||
if this == nil {
|
||||
return nil
|
||||
}
|
||||
return fmt2.Errorf("that == nil && this != nil")
|
||||
return fmt5.Errorf("that == nil && this != nil")
|
||||
}
|
||||
|
||||
that1, ok := that.(*ProtoAppendEntriesRequest)
|
||||
if !ok {
|
||||
return fmt2.Errorf("that is not of type *ProtoAppendEntriesRequest")
|
||||
return fmt5.Errorf("that is not of type *ProtoAppendEntriesRequest")
|
||||
}
|
||||
if that1 == nil {
|
||||
if this == nil {
|
||||
return nil
|
||||
}
|
||||
return fmt2.Errorf("that is type *ProtoAppendEntriesRequest but is nil && this != nil")
|
||||
return fmt5.Errorf("that is type *ProtoAppendEntriesRequest but is nil && this != nil")
|
||||
} else if this == nil {
|
||||
return fmt2.Errorf("that is type *ProtoAppendEntriesRequestbut is not nil && this == nil")
|
||||
return fmt5.Errorf("that is type *ProtoAppendEntriesRequestbut is not nil && this == nil")
|
||||
}
|
||||
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)
|
||||
return fmt5.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")
|
||||
return fmt5.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)
|
||||
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 fmt2.Errorf("PrevLogIndex this(%v) Not Equal that(%v)", *this.PrevLogIndex, *that1.PrevLogIndex)
|
||||
return fmt5.Errorf("PrevLogIndex this(%v) Not Equal that(%v)", *this.PrevLogIndex, *that1.PrevLogIndex)
|
||||
}
|
||||
} else if this.PrevLogIndex != nil {
|
||||
return fmt2.Errorf("this.PrevLogIndex == nil && that.PrevLogIndex != nil")
|
||||
return fmt5.Errorf("this.PrevLogIndex == nil && that.PrevLogIndex != nil")
|
||||
} else if that1.PrevLogIndex != nil {
|
||||
return fmt2.Errorf("PrevLogIndex this(%v) Not Equal that(%v)", this.PrevLogIndex, that1.PrevLogIndex)
|
||||
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 fmt2.Errorf("PrevLogTerm this(%v) Not Equal that(%v)", *this.PrevLogTerm, *that1.PrevLogTerm)
|
||||
return fmt5.Errorf("PrevLogTerm this(%v) Not Equal that(%v)", *this.PrevLogTerm, *that1.PrevLogTerm)
|
||||
}
|
||||
} else if this.PrevLogTerm != nil {
|
||||
return fmt2.Errorf("this.PrevLogTerm == nil && that.PrevLogTerm != nil")
|
||||
return fmt5.Errorf("this.PrevLogTerm == nil && that.PrevLogTerm != nil")
|
||||
} else if that1.PrevLogTerm != nil {
|
||||
return fmt2.Errorf("PrevLogTerm this(%v) Not Equal that(%v)", this.PrevLogTerm, that1.PrevLogTerm)
|
||||
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 fmt2.Errorf("CommitIndex this(%v) Not Equal that(%v)", *this.CommitIndex, *that1.CommitIndex)
|
||||
return fmt5.Errorf("CommitIndex this(%v) Not Equal that(%v)", *this.CommitIndex, *that1.CommitIndex)
|
||||
}
|
||||
} else if this.CommitIndex != nil {
|
||||
return fmt2.Errorf("this.CommitIndex == nil && that.CommitIndex != nil")
|
||||
return fmt5.Errorf("this.CommitIndex == nil && that.CommitIndex != nil")
|
||||
} else if that1.CommitIndex != nil {
|
||||
return fmt2.Errorf("CommitIndex this(%v) Not Equal that(%v)", this.CommitIndex, that1.CommitIndex)
|
||||
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 fmt2.Errorf("LeaderName this(%v) Not Equal that(%v)", *this.LeaderName, *that1.LeaderName)
|
||||
return fmt5.Errorf("LeaderName this(%v) Not Equal that(%v)", *this.LeaderName, *that1.LeaderName)
|
||||
}
|
||||
} else if this.LeaderName != nil {
|
||||
return fmt2.Errorf("this.LeaderName == nil && that.LeaderName != nil")
|
||||
return fmt5.Errorf("this.LeaderName == nil && that.LeaderName != nil")
|
||||
} else if that1.LeaderName != nil {
|
||||
return fmt2.Errorf("LeaderName this(%v) Not Equal that(%v)", this.LeaderName, that1.LeaderName)
|
||||
return fmt5.Errorf("LeaderName this(%v) Not Equal that(%v)", this.LeaderName, that1.LeaderName)
|
||||
}
|
||||
if len(this.Entries) != len(that1.Entries) {
|
||||
return fmt2.Errorf("Entries this(%v) Not Equal that(%v)", 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 fmt2.Errorf("Entries this[%v](%v) Not Equal that[%v](%v)", i, this.Entries[i], i, that1.Entries[i])
|
||||
return fmt5.Errorf("Entries this[%v](%v) Not Equal that[%v](%v)", i, this.Entries[i], i, that1.Entries[i])
|
||||
}
|
||||
}
|
||||
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)
|
||||
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
|
||||
}
|
||||
|
@ -935,117 +670,7 @@ func (this *ProtoAppendEntriesRequest) Equal(that interface{}) bool {
|
|||
return false
|
||||
}
|
||||
}
|
||||
if !bytes.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
||||
func (this *ProtoAppendEntriesRequest_ProtoLogEntry) VerboseEqual(that interface{}) error {
|
||||
if that == nil {
|
||||
if this == nil {
|
||||
return nil
|
||||
}
|
||||
return fmt2.Errorf("that == nil && this != nil")
|
||||
}
|
||||
|
||||
that1, ok := that.(*ProtoAppendEntriesRequest_ProtoLogEntry)
|
||||
if !ok {
|
||||
return fmt2.Errorf("that is not of type *ProtoAppendEntriesRequest_ProtoLogEntry")
|
||||
}
|
||||
if that1 == nil {
|
||||
if this == nil {
|
||||
return nil
|
||||
}
|
||||
return fmt2.Errorf("that is type *ProtoAppendEntriesRequest_ProtoLogEntry but is nil && this != nil")
|
||||
} else if this == nil {
|
||||
return fmt2.Errorf("that is type *ProtoAppendEntriesRequest_ProtoLogEntrybut 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 *ProtoAppendEntriesRequest_ProtoLogEntry) Equal(that interface{}) bool {
|
||||
if that == nil {
|
||||
if this == nil {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
that1, ok := that.(*ProtoAppendEntriesRequest_ProtoLogEntry)
|
||||
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) {
|
||||
if !bytes1.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package protobuf;
|
||||
|
||||
import "code.google.com/p/gogoprotobuf/gogoproto/gogo.proto";
|
||||
import "log_entry.proto";
|
||||
|
||||
option (gogoproto.gostring_all) = true;
|
||||
option (gogoproto.equal_all) = true;
|
||||
|
@ -20,13 +21,5 @@ message ProtoAppendEntriesRequest {
|
|||
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;
|
||||
}
|
|
@ -10,22 +10,22 @@ 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 io2 "io"
|
||||
import code_google_com_p_gogoprotobuf_proto4 "code.google.com/p/gogoprotobuf/proto"
|
||||
|
||||
import fmt3 "fmt"
|
||||
import strings2 "strings"
|
||||
import reflect2 "reflect"
|
||||
import fmt6 "fmt"
|
||||
import strings4 "strings"
|
||||
import reflect4 "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 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 fmt5 "fmt"
|
||||
import bytes1 "bytes"
|
||||
import fmt8 "fmt"
|
||||
import bytes2 "bytes"
|
||||
|
||||
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
|
@ -80,7 +80,7 @@ func (m *ProtoAppendEntriesResponse) Unmarshal(data []byte) error {
|
|||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io1.ErrUnexpectedEOF
|
||||
return io2.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -99,7 +99,7 @@ func (m *ProtoAppendEntriesResponse) Unmarshal(data []byte) error {
|
|||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io1.ErrUnexpectedEOF
|
||||
return io2.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -116,7 +116,7 @@ func (m *ProtoAppendEntriesResponse) Unmarshal(data []byte) error {
|
|||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io1.ErrUnexpectedEOF
|
||||
return io2.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -133,7 +133,7 @@ func (m *ProtoAppendEntriesResponse) Unmarshal(data []byte) error {
|
|||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io1.ErrUnexpectedEOF
|
||||
return io2.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -150,7 +150,7 @@ func (m *ProtoAppendEntriesResponse) Unmarshal(data []byte) error {
|
|||
var v int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io1.ErrUnexpectedEOF
|
||||
return io2.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -171,7 +171,7 @@ func (m *ProtoAppendEntriesResponse) Unmarshal(data []byte) error {
|
|||
}
|
||||
}
|
||||
index -= sizeOfWire
|
||||
skippy, err := code_google_com_p_gogoprotobuf_proto2.Skip(data[index:])
|
||||
skippy, err := code_google_com_p_gogoprotobuf_proto4.Skip(data[index:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -185,23 +185,23 @@ func (this *ProtoAppendEntriesResponse) String() string {
|
|||
if this == nil {
|
||||
return "nil"
|
||||
}
|
||||
s := strings2.Join([]string{`&ProtoAppendEntriesResponse{`,
|
||||
s := strings4.Join([]string{`&ProtoAppendEntriesResponse{`,
|
||||
`Term:` + valueToStringAppendEntriesResponses(this.Term) + `,`,
|
||||
`Index:` + valueToStringAppendEntriesResponses(this.Index) + `,`,
|
||||
`CommitIndex:` + valueToStringAppendEntriesResponses(this.CommitIndex) + `,`,
|
||||
`Success:` + valueToStringAppendEntriesResponses(this.Success) + `,`,
|
||||
`XXX_unrecognized:` + fmt3.Sprintf("%v", this.XXX_unrecognized) + `,`,
|
||||
`XXX_unrecognized:` + fmt6.Sprintf("%v", this.XXX_unrecognized) + `,`,
|
||||
`}`,
|
||||
}, "")
|
||||
return s
|
||||
}
|
||||
func valueToStringAppendEntriesResponses(v interface{}) string {
|
||||
rv := reflect2.ValueOf(v)
|
||||
rv := reflect4.ValueOf(v)
|
||||
if rv.IsNil() {
|
||||
return "nil"
|
||||
}
|
||||
pv := reflect2.Indirect(rv).Interface()
|
||||
return fmt3.Sprintf("*%v", pv)
|
||||
pv := reflect4.Indirect(rv).Interface()
|
||||
return fmt6.Sprintf("*%v", pv)
|
||||
}
|
||||
func (m *ProtoAppendEntriesResponse) Size() (n int) {
|
||||
var l int
|
||||
|
@ -396,18 +396,18 @@ func (this *ProtoAppendEntriesResponse) GoString() string {
|
|||
if this == nil {
|
||||
return "nil"
|
||||
}
|
||||
s := strings3.Join([]string{`&protobuf.ProtoAppendEntriesResponse{` + `Term:` + valueToGoStringAppendEntriesResponses(this.Term, "uint64"), `Index:` + valueToGoStringAppendEntriesResponses(this.Index, "uint64"), `CommitIndex:` + valueToGoStringAppendEntriesResponses(this.CommitIndex, "uint64"), `Success:` + valueToGoStringAppendEntriesResponses(this.Success, "bool"), `XXX_unrecognized:` + fmt4.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
|
||||
s := strings5.Join([]string{`&protobuf.ProtoAppendEntriesResponse{` + `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 := reflect3.ValueOf(v)
|
||||
rv := reflect5.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)
|
||||
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_proto3.Extension) string {
|
||||
func extensionToGoStringAppendEntriesResponses(e map[int32]code_google_com_p_gogoprotobuf_proto5.Extension) string {
|
||||
if e == nil {
|
||||
return "nil"
|
||||
}
|
||||
|
@ -416,12 +416,12 @@ func extensionToGoStringAppendEntriesResponses(e map[int32]code_google_com_p_gog
|
|||
for k := range e {
|
||||
keys = append(keys, int(k))
|
||||
}
|
||||
sort1.Ints(keys)
|
||||
sort2.Ints(keys)
|
||||
ss := []string{}
|
||||
for _, k := range keys {
|
||||
ss = append(ss, strconv1.Itoa(k)+": "+e[int32(k)].GoString())
|
||||
ss = append(ss, strconv2.Itoa(k)+": "+e[int32(k)].GoString())
|
||||
}
|
||||
s += strings3.Join(ss, ",") + "}"
|
||||
s += strings5.Join(ss, ",") + "}"
|
||||
return s
|
||||
}
|
||||
func (this *ProtoAppendEntriesResponse) VerboseEqual(that interface{}) error {
|
||||
|
@ -429,59 +429,59 @@ func (this *ProtoAppendEntriesResponse) VerboseEqual(that interface{}) error {
|
|||
if this == nil {
|
||||
return nil
|
||||
}
|
||||
return fmt5.Errorf("that == nil && this != nil")
|
||||
return fmt8.Errorf("that == nil && this != nil")
|
||||
}
|
||||
|
||||
that1, ok := that.(*ProtoAppendEntriesResponse)
|
||||
if !ok {
|
||||
return fmt5.Errorf("that is not of type *ProtoAppendEntriesResponse")
|
||||
return fmt8.Errorf("that is not of type *ProtoAppendEntriesResponse")
|
||||
}
|
||||
if that1 == nil {
|
||||
if this == nil {
|
||||
return nil
|
||||
}
|
||||
return fmt5.Errorf("that is type *ProtoAppendEntriesResponse but is nil && this != nil")
|
||||
return fmt8.Errorf("that is type *ProtoAppendEntriesResponse but is nil && this != nil")
|
||||
} else if this == nil {
|
||||
return fmt5.Errorf("that is type *ProtoAppendEntriesResponsebut is not nil && this == nil")
|
||||
return fmt8.Errorf("that is type *ProtoAppendEntriesResponsebut 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)
|
||||
return fmt8.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")
|
||||
return fmt8.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)
|
||||
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 fmt5.Errorf("Index this(%v) Not Equal that(%v)", *this.Index, *that1.Index)
|
||||
return fmt8.Errorf("Index this(%v) Not Equal that(%v)", *this.Index, *that1.Index)
|
||||
}
|
||||
} else if this.Index != nil {
|
||||
return fmt5.Errorf("this.Index == nil && that.Index != nil")
|
||||
return fmt8.Errorf("this.Index == nil && that.Index != nil")
|
||||
} else if that1.Index != nil {
|
||||
return fmt5.Errorf("Index this(%v) Not Equal that(%v)", this.Index, that1.Index)
|
||||
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 fmt5.Errorf("CommitIndex this(%v) Not Equal that(%v)", *this.CommitIndex, *that1.CommitIndex)
|
||||
return fmt8.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")
|
||||
return fmt8.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)
|
||||
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 fmt5.Errorf("Success this(%v) Not Equal that(%v)", *this.Success, *that1.Success)
|
||||
return fmt8.Errorf("Success this(%v) Not Equal that(%v)", *this.Success, *that1.Success)
|
||||
}
|
||||
} else if this.Success != nil {
|
||||
return fmt5.Errorf("this.Success == nil && that.Success != nil")
|
||||
return fmt8.Errorf("this.Success == nil && that.Success != nil")
|
||||
} else if that1.Success != nil {
|
||||
return fmt5.Errorf("Success this(%v) Not Equal that(%v)", this.Success, that1.Success)
|
||||
return fmt8.Errorf("Success this(%v) Not Equal that(%v)", this.Success, that1.Success)
|
||||
}
|
||||
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)
|
||||
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
|
||||
}
|
||||
|
@ -541,7 +541,7 @@ func (this *ProtoAppendEntriesResponse) Equal(that interface{}) bool {
|
|||
} else if that1.Success != nil {
|
||||
return false
|
||||
}
|
||||
if !bytes1.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
|
||||
if !bytes2.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
|
|
|
@ -10,22 +10,22 @@ 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 io "io"
|
||||
import code_google_com_p_gogoprotobuf_proto "code.google.com/p/gogoprotobuf/proto"
|
||||
|
||||
import fmt6 "fmt"
|
||||
import strings4 "strings"
|
||||
import reflect4 "reflect"
|
||||
import fmt "fmt"
|
||||
import strings "strings"
|
||||
import reflect "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 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 fmt8 "fmt"
|
||||
import bytes2 "bytes"
|
||||
import fmt2 "fmt"
|
||||
import bytes "bytes"
|
||||
|
||||
// Reference proto, json, and math imports to suppress error if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
|
@ -80,7 +80,7 @@ func (m *ProtoLogEntry) Unmarshal(data []byte) error {
|
|||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io2.ErrUnexpectedEOF
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -99,7 +99,7 @@ func (m *ProtoLogEntry) Unmarshal(data []byte) error {
|
|||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io2.ErrUnexpectedEOF
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -116,7 +116,7 @@ func (m *ProtoLogEntry) Unmarshal(data []byte) error {
|
|||
var v uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io2.ErrUnexpectedEOF
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -133,7 +133,7 @@ func (m *ProtoLogEntry) Unmarshal(data []byte) error {
|
|||
var stringLen uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io2.ErrUnexpectedEOF
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -144,7 +144,7 @@ func (m *ProtoLogEntry) Unmarshal(data []byte) error {
|
|||
}
|
||||
postIndex := index + int(stringLen)
|
||||
if postIndex > l {
|
||||
return io2.ErrUnexpectedEOF
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
s := string(data[index:postIndex])
|
||||
m.CommandName = &s
|
||||
|
@ -156,7 +156,7 @@ func (m *ProtoLogEntry) Unmarshal(data []byte) error {
|
|||
var byteLen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if index >= l {
|
||||
return io2.ErrUnexpectedEOF
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := data[index]
|
||||
index++
|
||||
|
@ -167,7 +167,7 @@ func (m *ProtoLogEntry) Unmarshal(data []byte) error {
|
|||
}
|
||||
postIndex := index + byteLen
|
||||
if postIndex > l {
|
||||
return io2.ErrUnexpectedEOF
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.Command = append(m.Command, data[index:postIndex]...)
|
||||
index = postIndex
|
||||
|
@ -181,7 +181,7 @@ func (m *ProtoLogEntry) Unmarshal(data []byte) error {
|
|||
}
|
||||
}
|
||||
index -= sizeOfWire
|
||||
skippy, err := code_google_com_p_gogoprotobuf_proto4.Skip(data[index:])
|
||||
skippy, err := code_google_com_p_gogoprotobuf_proto.Skip(data[index:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -195,23 +195,23 @@ func (this *ProtoLogEntry) String() string {
|
|||
if this == nil {
|
||||
return "nil"
|
||||
}
|
||||
s := strings4.Join([]string{`&ProtoLogEntry{`,
|
||||
s := strings.Join([]string{`&ProtoLogEntry{`,
|
||||
`Index:` + valueToStringLogEntry(this.Index) + `,`,
|
||||
`Term:` + valueToStringLogEntry(this.Term) + `,`,
|
||||
`CommandName:` + valueToStringLogEntry(this.CommandName) + `,`,
|
||||
`Command:` + valueToStringLogEntry(this.Command) + `,`,
|
||||
`XXX_unrecognized:` + fmt6.Sprintf("%v", this.XXX_unrecognized) + `,`,
|
||||
`XXX_unrecognized:` + fmt.Sprintf("%v", this.XXX_unrecognized) + `,`,
|
||||
`}`,
|
||||
}, "")
|
||||
return s
|
||||
}
|
||||
func valueToStringLogEntry(v interface{}) string {
|
||||
rv := reflect4.ValueOf(v)
|
||||
rv := reflect.ValueOf(v)
|
||||
if rv.IsNil() {
|
||||
return "nil"
|
||||
}
|
||||
pv := reflect4.Indirect(rv).Interface()
|
||||
return fmt6.Sprintf("*%v", pv)
|
||||
pv := reflect.Indirect(rv).Interface()
|
||||
return fmt.Sprintf("*%v", pv)
|
||||
}
|
||||
func (m *ProtoLogEntry) Size() (n int) {
|
||||
var l int
|
||||
|
@ -410,18 +410,18 @@ func (this *ProtoLogEntry) GoString() string {
|
|||
if this == nil {
|
||||
return "nil"
|
||||
}
|
||||
s := strings5.Join([]string{`&protobuf.ProtoLogEntry{` + `Index:` + valueToGoStringLogEntry(this.Index, "uint64"), `Term:` + valueToGoStringLogEntry(this.Term, "uint64"), `CommandName:` + valueToGoStringLogEntry(this.CommandName, "string"), `Command:` + valueToGoStringLogEntry(this.Command, "byte"), `XXX_unrecognized:` + fmt7.Sprintf("%#v", this.XXX_unrecognized) + `}`}, ", ")
|
||||
s := strings1.Join([]string{`&protobuf.ProtoLogEntry{` + `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 := reflect5.ValueOf(v)
|
||||
rv := reflect1.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)
|
||||
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_proto5.Extension) string {
|
||||
func extensionToGoStringLogEntry(e map[int32]code_google_com_p_gogoprotobuf_proto1.Extension) string {
|
||||
if e == nil {
|
||||
return "nil"
|
||||
}
|
||||
|
@ -430,12 +430,12 @@ func extensionToGoStringLogEntry(e map[int32]code_google_com_p_gogoprotobuf_prot
|
|||
for k := range e {
|
||||
keys = append(keys, int(k))
|
||||
}
|
||||
sort2.Ints(keys)
|
||||
sort.Ints(keys)
|
||||
ss := []string{}
|
||||
for _, k := range keys {
|
||||
ss = append(ss, strconv2.Itoa(k)+": "+e[int32(k)].GoString())
|
||||
ss = append(ss, strconv.Itoa(k)+": "+e[int32(k)].GoString())
|
||||
}
|
||||
s += strings5.Join(ss, ",") + "}"
|
||||
s += strings1.Join(ss, ",") + "}"
|
||||
return s
|
||||
}
|
||||
func (this *ProtoLogEntry) VerboseEqual(that interface{}) error {
|
||||
|
@ -443,53 +443,53 @@ func (this *ProtoLogEntry) VerboseEqual(that interface{}) error {
|
|||
if this == nil {
|
||||
return nil
|
||||
}
|
||||
return fmt8.Errorf("that == nil && this != nil")
|
||||
return fmt2.Errorf("that == nil && this != nil")
|
||||
}
|
||||
|
||||
that1, ok := that.(*ProtoLogEntry)
|
||||
if !ok {
|
||||
return fmt8.Errorf("that is not of type *ProtoLogEntry")
|
||||
return fmt2.Errorf("that is not of type *ProtoLogEntry")
|
||||
}
|
||||
if that1 == nil {
|
||||
if this == nil {
|
||||
return nil
|
||||
}
|
||||
return fmt8.Errorf("that is type *ProtoLogEntry but is nil && this != nil")
|
||||
return fmt2.Errorf("that is type *ProtoLogEntry but is nil && this != nil")
|
||||
} else if this == nil {
|
||||
return fmt8.Errorf("that is type *ProtoLogEntrybut is not nil && this == nil")
|
||||
return fmt2.Errorf("that is type *ProtoLogEntrybut is not nil && this == nil")
|
||||
}
|
||||
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)
|
||||
return fmt2.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")
|
||||
return fmt2.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)
|
||||
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 fmt8.Errorf("Term this(%v) Not Equal that(%v)", *this.Term, *that1.Term)
|
||||
return fmt2.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")
|
||||
return fmt2.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)
|
||||
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 fmt8.Errorf("CommandName this(%v) Not Equal that(%v)", *this.CommandName, *that1.CommandName)
|
||||
return fmt2.Errorf("CommandName this(%v) Not Equal that(%v)", *this.CommandName, *that1.CommandName)
|
||||
}
|
||||
} else if this.CommandName != nil {
|
||||
return fmt8.Errorf("this.CommandName == nil && that.CommandName != nil")
|
||||
return fmt2.Errorf("this.CommandName == nil && that.CommandName != nil")
|
||||
} else if that1.CommandName != nil {
|
||||
return fmt8.Errorf("CommandName this(%v) Not Equal that(%v)", this.CommandName, that1.CommandName)
|
||||
return fmt2.Errorf("CommandName this(%v) Not Equal that(%v)", this.CommandName, that1.CommandName)
|
||||
}
|
||||
if !bytes2.Equal(this.Command, that1.Command) {
|
||||
return fmt8.Errorf("Command this(%v) Not Equal that(%v)", this.Command, that1.Command)
|
||||
if !bytes.Equal(this.Command, that1.Command) {
|
||||
return fmt2.Errorf("Command this(%v) Not Equal that(%v)", this.Command, that1.Command)
|
||||
}
|
||||
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)
|
||||
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
|
||||
}
|
||||
|
@ -540,10 +540,10 @@ func (this *ProtoLogEntry) Equal(that interface{}) bool {
|
|||
} else if that1.CommandName != nil {
|
||||
return false
|
||||
}
|
||||
if !bytes2.Equal(this.Command, that1.Command) {
|
||||
if !bytes.Equal(this.Command, that1.Command) {
|
||||
return false
|
||||
}
|
||||
if !bytes2.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
|
||||
if !bytes.Equal(this.XXX_unrecognized, that1.XXX_unrecognized) {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
|
|
|
@ -1127,7 +1127,7 @@ func (s *server) TakeSnapshot() error {
|
|||
// to the slightly slow machines
|
||||
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)
|
||||
}
|
||||
|
||||
|
@ -1178,7 +1178,7 @@ func (s *server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse
|
|||
|
||||
entry := s.log.getEntry(req.LastIndex)
|
||||
|
||||
if entry != nil && entry.Term == req.LastTerm {
|
||||
if entry != nil && entry.Term() == req.LastTerm {
|
||||
return newSnapshotResponse(false)
|
||||
}
|
||||
|
||||
|
|
|
@ -1,9 +1,9 @@
|
|||
package raft
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"strconv"
|
||||
"sync"
|
||||
"testing"
|
||||
|
@ -289,23 +289,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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//--------------------------------------
|
||||
|
|
Loading…
Reference in New Issue