From a91a51d3787b0eed8849bcee5f27eaf8466042a8 Mon Sep 17 00:00:00 2001 From: gladiator Date: Tue, 26 Sep 2017 00:33:31 +0800 Subject: [PATCH 1/7] raft: add non-voter --- raft/node.go | 2 + raft/progress.go | 8 +- raft/raft.go | 107 ++++++++++-- raft/raft_test.go | 119 +++++++++++++- raft/raftpb/raft.pb.go | 357 ++++++++++++++++++++++++++++++++++------- raft/raftpb/raft.proto | 20 ++- raft/rawnode.go | 2 + 7 files changed, 526 insertions(+), 89 deletions(-) diff --git a/raft/node.go b/raft/node.go index ba25b7ddd48..f124c3b2586 100644 --- a/raft/node.go +++ b/raft/node.go @@ -334,6 +334,8 @@ func (n *node) run(r *raft) { switch cc.Type { case pb.ConfChangeAddNode: r.addNode(cc.NodeID) + case pb.ConfChangeAddNonvoter: + r.addNonvoter(cc.NodeID) case pb.ConfChangeRemoveNode: // block incoming proposal when local node is // removed diff --git a/raft/progress.go b/raft/progress.go index 77c7b52efe3..1425f91b6fe 100644 --- a/raft/progress.go +++ b/raft/progress.go @@ -14,7 +14,11 @@ package raft -import "fmt" +import ( + "fmt" + + pb "github.com/coreos/etcd/raft/raftpb" +) const ( ProgressStateProbe ProgressStateType = iota @@ -48,6 +52,8 @@ type Progress struct { // When in ProgressStateSnapshot, leader should have sent out snapshot // before and stops sending any replication message. State ProgressStateType + + Suffrage pb.SuffrageState // Paused is used in ProgressStateProbe. // When Paused is true, raft should pause sending replication message to this peer. Paused bool diff --git a/raft/raft.go b/raft/raft.go index 3e9ca275925..8ad59eacd23 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -114,7 +114,7 @@ type Config struct { // should only be set when starting a new raft cluster. Restarting raft from // previous configuration will panic if peers is set. peer is private and only // used for testing right now. - peers []uint64 + peers []pb.Server // ElectionTick is the number of Node.Tick invocations that must pass between // elections. That is, if a follower does not receive any message from the @@ -238,6 +238,8 @@ type raft struct { state StateType + suffrage pb.SuffrageState + votes map[uint64]bool msgs []pb.Message @@ -289,6 +291,9 @@ func newRaft(c *Config) *raft { panic(err) // TODO(bdarnell) } peers := c.peers + if len(cs.Nodes) > 0 && len(cs.Servers) > 0 { + panic("cannot specify both ConfState.Nodes and ConfState.Servers") + } if len(cs.Nodes) > 0 { if len(peers) > 0 { // TODO(bdarnell): the peers argument is always nil except in @@ -296,11 +301,22 @@ func newRaft(c *Config) *raft { // updated to specify their nodes through a snapshot. panic("cannot specify both newRaft(peers) and ConfState.Nodes)") } - peers = cs.Nodes + for _, n := range cs.Nodes { + peers = append(peers, pb.Server{Node: n, Suffrage: pb.Voter}) + } + } + if len(cs.Servers) > 0 { + if len(peers) > 0 { + panic("cannot specify both newRaft(peers) and ConfState.Servers)") + } + for _, s := range cs.Servers { + peers = append(peers, *s) + } } r := &raft{ id: c.ID, lead: None, + suffrage: pb.Voter, raftLog: raftlog, maxMsgSize: c.MaxSizePerMsg, maxInflight: c.MaxInflightMsgs, @@ -314,7 +330,10 @@ func newRaft(c *Config) *raft { disableProposalForwarding: c.DisableProposalForwarding, } for _, p := range peers { - r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)} + r.prs[p.Node] = &Progress{Next: 1, ins: newInflights(r.maxInflight), Suffrage: p.Suffrage} + if r.id == p.Node { + r.suffrage = p.Suffrage + } } if !isHardStateEqual(hs, emptyState) { r.loadState(hs) @@ -346,7 +365,18 @@ func (r *raft) hardState() pb.HardState { } } -func (r *raft) quorum() int { return len(r.prs)/2 + 1 } +// TODO(shuaili): optimize, maybe record voterCount in raft +func (r *raft) voterCount() int { + count := 0 + for _, p := range r.prs { + if p.Suffrage == pb.Voter { + count++ + } + } + return count +} + +func (r *raft) quorum() int { return r.voterCount()/2 + 1 } func (r *raft) nodes() []uint64 { nodes := make([]uint64, 0, len(r.prs)) @@ -504,9 +534,11 @@ func (r *raft) bcastHeartbeatWithCtx(ctx []byte) { // r.bcastAppend). func (r *raft) maybeCommit() bool { // TODO(bmizerany): optimize.. Currently naive - mis := make(uint64Slice, 0, len(r.prs)) - for id := range r.prs { - mis = append(mis, r.prs[id].Match) + mis := make(uint64Slice, 0, r.voterCount()) + for _, p := range r.prs { + if p.Suffrage == pb.Voter { + mis = append(mis, p.Match) + } } sort.Sort(sort.Reverse(mis)) mci := mis[r.quorum()-1] @@ -528,7 +560,7 @@ func (r *raft) reset(term uint64) { r.votes = make(map[uint64]bool) for id := range r.prs { - r.prs[id] = &Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight)} + r.prs[id] = &Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight), Suffrage: r.prs[id].Suffrage} if id == r.id { r.prs[id].Match = r.raftLog.lastIndex() } @@ -555,7 +587,9 @@ func (r *raft) tickElection() { if r.promotable() && r.pastElectionTimeout() { r.electionElapsed = 0 - r.Step(pb.Message{From: r.id, Type: pb.MsgHup}) + if r.suffrage == pb.Voter { + r.Step(pb.Message{From: r.id, Type: pb.MsgHup}) + } } } @@ -676,6 +710,9 @@ func (r *raft) campaign(t CampaignType) { if id == r.id { continue } + if r.prs[id].Suffrage != pb.Voter { + continue + } r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d", r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), voteMsg, id, r.Term) @@ -990,6 +1027,10 @@ func stepLeader(r *raft, m pb.Message) { } r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr) case pb.MsgTransferLeader: + if pr.Suffrage != pb.Voter { + r.logger.Debugf("%x is not Voter. Ignored transferring leadership", r.id) + return + } leadTransferee := m.From lastLeadTransferee := r.leadTransferee if lastLeadTransferee != None { @@ -1171,14 +1212,27 @@ func (r *raft) restore(s pb.Snapshot) bool { r.raftLog.restore(s) r.prs = make(map[uint64]*Progress) + if len(s.Metadata.ConfState.Nodes) > 0 && len(s.Metadata.ConfState.Servers) > 0 { + panic("cannot specify both ConfState.Nodes and ConfState.Servers") + } for _, n := range s.Metadata.ConfState.Nodes { match, next := uint64(0), r.raftLog.lastIndex()+1 if n == r.id { match = next - 1 + r.suffrage = pb.Voter } - r.setProgress(n, match, next) + r.setProgress(n, match, next, pb.Voter) r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n]) } + for _, server := range s.Metadata.ConfState.Servers { + match, next := uint64(0), r.raftLog.lastIndex()+1 + if server.Node == r.id { + match = next - 1 + r.suffrage = server.Suffrage + } + r.setProgress(server.Node, match, next, server.Suffrage) + r.logger.Infof("%x restored progress of %x [%s]", r.id, server.Node, r.prs[server.Node]) + } return true } @@ -1190,14 +1244,33 @@ func (r *raft) promotable() bool { } func (r *raft) addNode(id uint64) { + r.addNodeWithSuffrage(id, pb.Voter) +} + +func (r *raft) addNonvoter(id uint64) { + r.addNodeWithSuffrage(id, pb.Nonvoter) +} + +func (r *raft) addNodeWithSuffrage(id uint64, suffrage pb.SuffrageState) { r.pendingConf = false if _, ok := r.prs[id]; ok { - // Ignore any redundant addNode calls (which can happen because the - // initial bootstrapping entries are applied twice). - return + if r.prs[id].Suffrage == suffrage { + // Ignore any redundant addNode calls (which can happen because the + // initial bootstrapping entries are applied twice). + return + } + if suffrage != pb.Voter { + // addNode can only change suffrage from Nonvoter to Voter + return + } + r.prs[id].Suffrage = suffrage + } else { + r.setProgress(id, 0, r.raftLog.lastIndex()+1, suffrage) } - r.setProgress(id, 0, r.raftLog.lastIndex()+1) + if r.id == id { + r.suffrage = suffrage + } // When a node is first added, we should mark it as recently active. // Otherwise, CheckQuorum may cause us to step down if it is invoked // before the added node has a chance to communicate with us. @@ -1226,8 +1299,8 @@ func (r *raft) removeNode(id uint64) { func (r *raft) resetPendingConf() { r.pendingConf = false } -func (r *raft) setProgress(id, match, next uint64) { - r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)} +func (r *raft) setProgress(id, match, next uint64, suffrage pb.SuffrageState) { + r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight), Suffrage: suffrage} } func (r *raft) delProgress(id uint64) { @@ -1267,7 +1340,7 @@ func (r *raft) checkQuorumActive() bool { continue } - if r.prs[id].RecentActive { + if r.prs[id].RecentActive && r.prs[id].Suffrage == pb.Voter { act++ } diff --git a/raft/raft_test.go b/raft/raft_test.go index efd511f72c6..ce14ed2d2dc 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -348,6 +348,54 @@ func testLeaderElection(t *testing.T, preVote bool) { } } +func TestNonvoterElectionTimeout(t *testing.T) { + n1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) + n1.prs[3].Suffrage = pb.Nonvoter + + n2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) + n2.prs[3].Suffrage = pb.Nonvoter + + n3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) + n3.suffrage = pb.Nonvoter + n3.prs[3].Suffrage = pb.Nonvoter + + n1.becomeFollower(1, None) + n2.becomeFollower(1, None) + n3.becomeFollower(1, None) + + nt := newNetwork(n1, n2, n3) + + // Nonvoter can't start election + setRandomizedElectionTimeout(n3, n3.electionTimeout) + for i := 0; i < n3.electionTimeout; i++ { + n3.tick() + } + if n1.state != StateFollower { + t.Errorf("peer 1 state: %s, want %s", n1.state, StateFollower) + } + if n2.state != StateFollower { + t.Errorf("peer 2 state: %s, want %s", n2.state, StateFollower) + } + if n3.state != StateFollower { + t.Errorf("peer 3 state: %s, want %s", n3.state, StateFollower) + } + + nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + if n1.state != StateLeader { + t.Errorf("peer 1 state: %s, want %s", n1.state, StateLeader) + } + + // node 3 become Voter + n1.addNode(3) + n2.addNode(3) + n3.addNode(3) + + nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup}) + if n3.state != StateLeader { + t.Errorf("peer 3 state: %s, want %s", n3.state, StateLeader) + } +} + func TestLeaderCycle(t *testing.T) { testLeaderCycle(t, false) } @@ -1058,7 +1106,7 @@ func TestCommit(t *testing.T) { sm := newTestRaft(1, []uint64{1}, 5, 1, storage) for j := 0; j < len(tt.matches); j++ { - sm.setProgress(uint64(j)+1, tt.matches[j], tt.matches[j]+1) + sm.setProgress(uint64(j)+1, tt.matches[j], tt.matches[j]+1, pb.Voter) } sm.maybeCommit() if g := sm.raftLog.committed; g != tt.w { @@ -2326,6 +2374,42 @@ func TestRestore(t *testing.T) { } } +func TestRestoreWithNonvoter(t *testing.T) { + s := pb.Snapshot{ + Metadata: pb.SnapshotMetadata{ + Index: 11, // magic number + Term: 11, // magic number + ConfState: pb.ConfState{Servers: []*pb.Server{ + {Node: 1, Suffrage: pb.Voter}, + {Node: 2, Suffrage: pb.Voter}, + {Node: 3, Suffrage: pb.Nonvoter}, + }}, + }, + } + + storage := NewMemoryStorage() + sm := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, storage) + if ok := sm.restore(s); !ok { + t.Fatal("restore fail, want succeed") + } + + if sm.raftLog.lastIndex() != s.Metadata.Index { + t.Errorf("log.lastIndex = %d, want %d", sm.raftLog.lastIndex(), s.Metadata.Index) + } + if mustTerm(sm.raftLog.term(s.Metadata.Index)) != s.Metadata.Term { + t.Errorf("log.lastTerm = %d, want %d", mustTerm(sm.raftLog.term(s.Metadata.Index)), s.Metadata.Term) + } + for _, server := range s.Metadata.ConfState.Servers { + if sm.prs[server.Node].Suffrage != server.Suffrage { + t.Errorf("sm.Node %x suffrage = %s, want %s", server.Node, sm.prs[server.Node], server.Suffrage) + } + } + + if ok := sm.restore(s); ok { + t.Fatal("restore succeed, want fail") + } +} + func TestRestoreIgnoreSnapshot(t *testing.T) { previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}, {Term: 1, Index: 3}} commit := uint64(1) @@ -2569,6 +2653,23 @@ func TestAddNode(t *testing.T) { } } +func TestAddNonvoter(t *testing.T) { + r := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) + r.pendingConf = true + r.addNonvoter(2) + if r.pendingConf { + t.Errorf("pendingConf = %v, want false", r.pendingConf) + } + nodes := r.nodes() + wnodes := []uint64{1, 2} + if !reflect.DeepEqual(nodes, wnodes) { + t.Errorf("nodes = %v, want %v", nodes, wnodes) + } + if r.prs[2].Suffrage != pb.Nonvoter { + t.Fatalf("node 2 has suffrage %s, want %s", r.prs[2].Suffrage, pb.Nonvoter) + } +} + // TestAddNodeCheckQuorum tests that addNode does not trigger a leader election // immediately when checkQuorum is set. func TestAddNodeCheckQuorum(t *testing.T) { @@ -3278,7 +3379,7 @@ func entsWithConfig(configFunc func(*Config), terms ...uint64) *raft { for i, term := range terms { storage.Append([]pb.Entry{{Index: uint64(i + 1), Term: term}}) } - cfg := newTestConfig(1, []uint64{}, 5, 1, storage) + cfg := newTestConfig(1, []uint64{1, 2, 3, 4, 5}, 5, 1, storage) if configFunc != nil { configFunc(cfg) } @@ -3293,7 +3394,7 @@ func entsWithConfig(configFunc func(*Config), terms ...uint64) *raft { func votedWithConfig(configFunc func(*Config), vote, term uint64) *raft { storage := NewMemoryStorage() storage.SetHardState(pb.HardState{Vote: vote, Term: term}) - cfg := newTestConfig(1, []uint64{}, 5, 1, storage) + cfg := newTestConfig(1, []uint64{1, 2, 3, 4, 5}, 5, 1, storage) if configFunc != nil { configFunc(cfg) } @@ -3338,10 +3439,14 @@ func newNetworkWithConfig(configFunc func(*Config), peers ...stateMachine) *netw sm := newRaft(cfg) npeers[id] = sm case *raft: + suffrages := make(map[uint64]pb.SuffrageState, size) + for i, pr := range p.(*raft).prs { + suffrages[i] = pr.Suffrage + } v.id = id v.prs = make(map[uint64]*Progress) for i := 0; i < size; i++ { - v.prs[peerAddrs[i]] = &Progress{} + v.prs[peerAddrs[i]] = &Progress{Suffrage: suffrages[peerAddrs[i]]} } v.reset(v.Term) npeers[id] = v @@ -3448,9 +3553,13 @@ func setRandomizedElectionTimeout(r *raft, v int) { } func newTestConfig(id uint64, peers []uint64, election, heartbeat int, storage Storage) *Config { + new_peers := make([]pb.Server, 0) + for _, p := range peers { + new_peers = append(new_peers, pb.Server{Node: p, Suffrage: pb.Voter}) + } return &Config{ ID: id, - peers: peers, + peers: new_peers, ElectionTick: election, HeartbeatTick: heartbeat, Storage: storage, diff --git a/raft/raftpb/raft.pb.go b/raft/raftpb/raft.pb.go index 4c6e79d58a0..5341d9bf730 100644 --- a/raft/raftpb/raft.pb.go +++ b/raft/raftpb/raft.pb.go @@ -14,6 +14,7 @@ Snapshot Message HardState + Server ConfState ConfChange */ @@ -159,23 +160,60 @@ func (x *MessageType) UnmarshalJSON(data []byte) error { } func (MessageType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{1} } +type SuffrageState int32 + +const ( + Nonvoter SuffrageState = 0 + Voter SuffrageState = 1 +) + +var SuffrageState_name = map[int32]string{ + 0: "Nonvoter", + 1: "Voter", +} +var SuffrageState_value = map[string]int32{ + "Nonvoter": 0, + "Voter": 1, +} + +func (x SuffrageState) Enum() *SuffrageState { + p := new(SuffrageState) + *p = x + return p +} +func (x SuffrageState) String() string { + return proto.EnumName(SuffrageState_name, int32(x)) +} +func (x *SuffrageState) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(SuffrageState_value, data, "SuffrageState") + if err != nil { + return err + } + *x = SuffrageState(value) + return nil +} +func (SuffrageState) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{2} } + type ConfChangeType int32 const ( - ConfChangeAddNode ConfChangeType = 0 - ConfChangeRemoveNode ConfChangeType = 1 - ConfChangeUpdateNode ConfChangeType = 2 + ConfChangeAddNode ConfChangeType = 0 + ConfChangeRemoveNode ConfChangeType = 1 + ConfChangeUpdateNode ConfChangeType = 2 + ConfChangeAddNonvoter ConfChangeType = 3 ) var ConfChangeType_name = map[int32]string{ 0: "ConfChangeAddNode", 1: "ConfChangeRemoveNode", 2: "ConfChangeUpdateNode", + 3: "ConfChangeAddNonvoter", } var ConfChangeType_value = map[string]int32{ - "ConfChangeAddNode": 0, - "ConfChangeRemoveNode": 1, - "ConfChangeUpdateNode": 2, + "ConfChangeAddNode": 0, + "ConfChangeRemoveNode": 1, + "ConfChangeUpdateNode": 2, + "ConfChangeAddNonvoter": 3, } func (x ConfChangeType) Enum() *ConfChangeType { @@ -194,7 +232,7 @@ func (x *ConfChangeType) UnmarshalJSON(data []byte) error { *x = ConfChangeType(value) return nil } -func (ConfChangeType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{2} } +func (ConfChangeType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{3} } type Entry struct { Term uint64 `protobuf:"varint,2,opt,name=Term" json:"Term"` @@ -265,15 +303,27 @@ func (m *HardState) String() string { return proto.CompactTextString( func (*HardState) ProtoMessage() {} func (*HardState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{4} } +type Server struct { + Node uint64 `protobuf:"varint,1,opt,name=node" json:"node"` + Suffrage SuffrageState `protobuf:"varint,2,opt,name=suffrage,enum=raftpb.SuffrageState" json:"suffrage"` + XXX_unrecognized []byte `json:"-"` +} + +func (m *Server) Reset() { *m = Server{} } +func (m *Server) String() string { return proto.CompactTextString(m) } +func (*Server) ProtoMessage() {} +func (*Server) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{5} } + type ConfState struct { - Nodes []uint64 `protobuf:"varint,1,rep,name=nodes" json:"nodes,omitempty"` - XXX_unrecognized []byte `json:"-"` + Nodes []uint64 `protobuf:"varint,1,rep,name=nodes" json:"nodes,omitempty"` + Servers []*Server `protobuf:"bytes,2,rep,name=servers" json:"servers,omitempty"` + XXX_unrecognized []byte `json:"-"` } func (m *ConfState) Reset() { *m = ConfState{} } func (m *ConfState) String() string { return proto.CompactTextString(m) } func (*ConfState) ProtoMessage() {} -func (*ConfState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{5} } +func (*ConfState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{6} } type ConfChange struct { ID uint64 `protobuf:"varint,1,opt,name=ID" json:"ID"` @@ -286,7 +336,7 @@ type ConfChange struct { func (m *ConfChange) Reset() { *m = ConfChange{} } func (m *ConfChange) String() string { return proto.CompactTextString(m) } func (*ConfChange) ProtoMessage() {} -func (*ConfChange) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{6} } +func (*ConfChange) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{7} } func init() { proto.RegisterType((*Entry)(nil), "raftpb.Entry") @@ -294,10 +344,12 @@ func init() { proto.RegisterType((*Snapshot)(nil), "raftpb.Snapshot") proto.RegisterType((*Message)(nil), "raftpb.Message") proto.RegisterType((*HardState)(nil), "raftpb.HardState") + proto.RegisterType((*Server)(nil), "raftpb.Server") proto.RegisterType((*ConfState)(nil), "raftpb.ConfState") proto.RegisterType((*ConfChange)(nil), "raftpb.ConfChange") proto.RegisterEnum("raftpb.EntryType", EntryType_name, EntryType_value) proto.RegisterEnum("raftpb.MessageType", MessageType_name, MessageType_value) + proto.RegisterEnum("raftpb.SuffrageState", SuffrageState_name, SuffrageState_value) proto.RegisterEnum("raftpb.ConfChangeType", ConfChangeType_name, ConfChangeType_value) } func (m *Entry) Marshal() (dAtA []byte, err error) { @@ -515,6 +567,33 @@ func (m *HardState) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *Server) 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 *Server) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintRaft(dAtA, i, uint64(m.Node)) + dAtA[i] = 0x10 + i++ + i = encodeVarintRaft(dAtA, i, uint64(m.Suffrage)) + if m.XXX_unrecognized != nil { + i += copy(dAtA[i:], m.XXX_unrecognized) + } + return i, nil +} + func (m *ConfState) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -537,6 +616,18 @@ func (m *ConfState) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintRaft(dAtA, i, uint64(num)) } } + if len(m.Servers) > 0 { + for _, msg := range m.Servers { + dAtA[i] = 0x12 + i++ + i = encodeVarintRaft(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) } @@ -692,6 +783,17 @@ func (m *HardState) Size() (n int) { return n } +func (m *Server) Size() (n int) { + var l int + _ = l + n += 1 + sovRaft(uint64(m.Node)) + n += 1 + sovRaft(uint64(m.Suffrage)) + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + func (m *ConfState) Size() (n int) { var l int _ = l @@ -700,6 +802,12 @@ func (m *ConfState) Size() (n int) { n += 1 + sovRaft(uint64(e)) } } + if len(m.Servers) > 0 { + for _, e := range m.Servers { + l = e.Size() + n += 1 + l + sovRaft(uint64(l)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -1528,6 +1636,95 @@ func (m *HardState) Unmarshal(dAtA []byte) error { } return nil } +func (m *Server) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + 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) + if wireType == 4 { + return fmt.Errorf("proto: Server: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Server: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Node", wireType) + } + m.Node = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Node |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Suffrage", wireType) + } + m.Suffrage = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Suffrage |= (SuffrageState(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipRaft(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRaft + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ConfState) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -1619,6 +1816,37 @@ func (m *ConfState) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field Nodes", wireType) } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Servers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Servers = append(m.Servers, &Server{}) + if err := m.Servers[len(m.Servers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRaft(dAtA[iNdEx:]) @@ -1888,55 +2116,60 @@ var ( func init() { proto.RegisterFile("raft.proto", fileDescriptorRaft) } var fileDescriptorRaft = []byte{ - // 790 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x54, 0xcd, 0x6e, 0xdb, 0x46, - 0x10, 0x16, 0x29, 0xea, 0x6f, 0x28, 0xcb, 0xab, 0xb5, 0x5a, 0x2c, 0x0c, 0x43, 0x55, 0x85, 0x1e, - 0x04, 0x17, 0x76, 0x5b, 0x1d, 0x7a, 0xe8, 0xcd, 0x96, 0x0a, 0x58, 0x40, 0x65, 0xb8, 0xb2, 0xdc, - 0x43, 0x83, 0x20, 0x58, 0x8b, 0x2b, 0x4a, 0x89, 0xc9, 0x25, 0x96, 0x2b, 0xc7, 0xbe, 0x04, 0x79, - 0x80, 0x3c, 0x40, 0x2e, 0x79, 0x1f, 0x1f, 0x0d, 0xe4, 0x1e, 0xc4, 0xce, 0x8b, 0x04, 0xbb, 0x5c, - 0x4a, 0x94, 0x74, 0xdb, 0xf9, 0xbe, 0xe1, 0xcc, 0x37, 0xdf, 0xce, 0x12, 0x40, 0xd0, 0xa9, 0x3c, - 0x8e, 0x04, 0x97, 0x1c, 0x17, 0xd5, 0x39, 0xba, 0xde, 0x6f, 0xf8, 0xdc, 0xe7, 0x1a, 0xfa, 0x4d, - 0x9d, 0x12, 0xb6, 0xfd, 0x0e, 0x0a, 0x7f, 0x87, 0x52, 0xdc, 0xe3, 0x5f, 0xc1, 0x19, 0xdf, 0x47, - 0x8c, 0x58, 0x2d, 0xab, 0x53, 0xeb, 0xd6, 0x8f, 0x93, 0xaf, 0x8e, 0x35, 0xa9, 0x88, 0x53, 0xe7, - 0xe1, 0xcb, 0x4f, 0xb9, 0x91, 0x4e, 0xc2, 0x04, 0x9c, 0x31, 0x13, 0x01, 0xb1, 0x5b, 0x56, 0xc7, - 0x59, 0x32, 0x4c, 0x04, 0x78, 0x1f, 0x0a, 0x83, 0xd0, 0x63, 0x77, 0x24, 0x9f, 0xa1, 0x12, 0x08, - 0x63, 0x70, 0xfa, 0x54, 0x52, 0xe2, 0xb4, 0xac, 0x4e, 0x75, 0xa4, 0xcf, 0xed, 0xf7, 0x16, 0xa0, - 0xcb, 0x90, 0x46, 0xf1, 0x8c, 0xcb, 0x21, 0x93, 0xd4, 0xa3, 0x92, 0xe2, 0x3f, 0x01, 0x26, 0x3c, - 0x9c, 0xbe, 0x8a, 0x25, 0x95, 0x89, 0x22, 0x77, 0xa5, 0xa8, 0xc7, 0xc3, 0xe9, 0xa5, 0x22, 0x4c, - 0xf1, 0xca, 0x24, 0x05, 0x54, 0xf3, 0xb9, 0x6e, 0x9e, 0xd5, 0x95, 0x40, 0x4a, 0xb2, 0x54, 0x92, - 0xb3, 0xba, 0x34, 0xd2, 0xfe, 0x1f, 0xca, 0xa9, 0x02, 0x25, 0x51, 0x29, 0xd0, 0x3d, 0xab, 0x23, - 0x7d, 0xc6, 0x7f, 0x41, 0x39, 0x30, 0xca, 0x74, 0x61, 0xb7, 0x4b, 0x52, 0x2d, 0x9b, 0xca, 0x4d, - 0xdd, 0x65, 0x7e, 0xfb, 0x53, 0x1e, 0x4a, 0x43, 0x16, 0xc7, 0xd4, 0x67, 0xf8, 0x08, 0x1c, 0xb9, - 0x72, 0x78, 0x2f, 0xad, 0x61, 0xe8, 0xac, 0xc7, 0x2a, 0x0d, 0x37, 0xc0, 0x96, 0x7c, 0x6d, 0x12, - 0x5b, 0x72, 0x35, 0xc6, 0x54, 0xf0, 0x8d, 0x31, 0x14, 0xb2, 0x1c, 0xd0, 0xd9, 0x1c, 0x10, 0x37, - 0xa1, 0x74, 0xc3, 0x7d, 0x7d, 0x61, 0x85, 0x0c, 0x99, 0x82, 0x2b, 0xdb, 0x8a, 0xdb, 0xb6, 0x1d, - 0x41, 0x89, 0x85, 0x52, 0xcc, 0x59, 0x4c, 0x4a, 0xad, 0x7c, 0xc7, 0xed, 0xee, 0xac, 0x6d, 0x46, - 0x5a, 0xca, 0xe4, 0xe0, 0x03, 0x28, 0x4e, 0x78, 0x10, 0xcc, 0x25, 0x29, 0x67, 0x6a, 0x19, 0x0c, - 0x77, 0xa1, 0x1c, 0x1b, 0xc7, 0x48, 0x45, 0x3b, 0x89, 0x36, 0x9d, 0x4c, 0x1d, 0x4c, 0xf3, 0x54, - 0x45, 0xc1, 0x5e, 0xb3, 0x89, 0x24, 0xd0, 0xb2, 0x3a, 0xe5, 0xb4, 0x62, 0x82, 0xe1, 0x5f, 0x00, - 0x92, 0xd3, 0xd9, 0x3c, 0x94, 0xc4, 0xcd, 0xf4, 0xcc, 0xe0, 0x98, 0x40, 0x69, 0xc2, 0x43, 0xc9, - 0xee, 0x24, 0xa9, 0xea, 0x8b, 0x4d, 0xc3, 0xf6, 0x4b, 0xa8, 0x9c, 0x51, 0xe1, 0x25, 0xeb, 0x93, - 0x3a, 0x68, 0x6d, 0x39, 0x48, 0xc0, 0xb9, 0xe5, 0x92, 0xad, 0xef, 0xbb, 0x42, 0x32, 0x03, 0xe7, - 0xb7, 0x07, 0x6e, 0xff, 0x0c, 0x95, 0xe5, 0xba, 0xe2, 0x06, 0x14, 0x42, 0xee, 0xb1, 0x98, 0x58, - 0xad, 0x7c, 0xc7, 0x19, 0x25, 0x41, 0xfb, 0x83, 0x05, 0xa0, 0x72, 0x7a, 0x33, 0x1a, 0xfa, 0xfa, - 0xd6, 0x07, 0xfd, 0x35, 0x05, 0xf6, 0xa0, 0x8f, 0x7f, 0x37, 0x8f, 0xd3, 0xd6, 0xab, 0xf3, 0x63, - 0xf6, 0x29, 0x24, 0xdf, 0x6d, 0xbd, 0xd0, 0x03, 0x28, 0x9e, 0x73, 0x8f, 0x0d, 0xfa, 0xeb, 0xba, - 0x12, 0x4c, 0x19, 0xd2, 0x33, 0x86, 0x24, 0x8f, 0x31, 0x0d, 0x0f, 0xff, 0x80, 0xca, 0xf2, 0xc9, - 0xe3, 0x5d, 0x70, 0x75, 0x70, 0xce, 0x45, 0x40, 0x6f, 0x50, 0x0e, 0xef, 0xc1, 0xae, 0x06, 0x56, - 0x8d, 0x91, 0x75, 0xf8, 0xd9, 0x06, 0x37, 0xb3, 0xc4, 0x18, 0xa0, 0x38, 0x8c, 0xfd, 0xb3, 0x45, - 0x84, 0x72, 0xd8, 0x85, 0xd2, 0x30, 0xf6, 0x4f, 0x19, 0x95, 0xc8, 0x32, 0xc1, 0x85, 0xe0, 0x11, - 0xb2, 0x4d, 0xd6, 0x49, 0x14, 0xa1, 0x3c, 0xae, 0x01, 0x24, 0xe7, 0x11, 0x8b, 0x23, 0xe4, 0x98, - 0xc4, 0xff, 0xb8, 0x64, 0xa8, 0xa0, 0x44, 0x98, 0x40, 0xb3, 0x45, 0xc3, 0xaa, 0x85, 0x41, 0x25, - 0x8c, 0xa0, 0xaa, 0x9a, 0x31, 0x2a, 0xe4, 0xb5, 0xea, 0x52, 0xc6, 0x0d, 0x40, 0x59, 0x44, 0x7f, - 0x54, 0xc1, 0x18, 0x6a, 0xc3, 0xd8, 0xbf, 0x0a, 0x05, 0xa3, 0x93, 0x19, 0xbd, 0xbe, 0x61, 0x08, - 0x70, 0x1d, 0x76, 0x4c, 0x21, 0x75, 0x41, 0x8b, 0x18, 0xb9, 0x26, 0xad, 0x37, 0x63, 0x93, 0x37, - 0xff, 0x2e, 0xb8, 0x58, 0x04, 0xa8, 0x8a, 0x7f, 0x80, 0xfa, 0x30, 0xf6, 0xc7, 0x82, 0x86, 0xf1, - 0x94, 0x89, 0x7f, 0x18, 0xf5, 0x98, 0x40, 0x3b, 0xe6, 0xeb, 0xf1, 0x3c, 0x60, 0x7c, 0x21, 0xcf, - 0xf9, 0x5b, 0x54, 0x33, 0x62, 0x46, 0x8c, 0x7a, 0xfa, 0x87, 0x87, 0x76, 0x8d, 0x98, 0x25, 0xa2, - 0xc5, 0x20, 0x33, 0xef, 0x85, 0x60, 0x7a, 0xc4, 0xba, 0xe9, 0x6a, 0x62, 0x9d, 0x83, 0x0f, 0x5f, - 0x40, 0x6d, 0xfd, 0x7a, 0x95, 0x8e, 0x15, 0x72, 0xe2, 0x79, 0xea, 0x2e, 0x51, 0x0e, 0x13, 0x68, - 0xac, 0xe0, 0x11, 0x0b, 0xf8, 0x2d, 0xd3, 0x8c, 0xb5, 0xce, 0x5c, 0x45, 0x1e, 0x95, 0x09, 0x63, - 0x9f, 0x92, 0x87, 0xa7, 0x66, 0xee, 0xf1, 0xa9, 0x99, 0x7b, 0x78, 0x6e, 0x5a, 0x8f, 0xcf, 0x4d, - 0xeb, 0xeb, 0x73, 0xd3, 0xfa, 0xf8, 0xad, 0x99, 0xfb, 0x1e, 0x00, 0x00, 0xff, 0xff, 0xcf, 0x30, - 0x01, 0x41, 0x3a, 0x06, 0x00, 0x00, + // 871 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x54, 0x4d, 0x6f, 0xe3, 0x44, + 0x18, 0x8e, 0x13, 0xe7, 0xeb, 0x4d, 0x9a, 0x4e, 0xdf, 0x4d, 0xd1, 0xb0, 0x5a, 0x85, 0x28, 0xe2, + 0x10, 0x15, 0x6d, 0x81, 0x1c, 0x40, 0xe2, 0xb6, 0x6d, 0x91, 0x5a, 0x41, 0xaa, 0xc5, 0xed, 0xee, + 0x01, 0x84, 0xd0, 0x34, 0x9e, 0xb8, 0x81, 0xda, 0x63, 0xcd, 0x4c, 0xca, 0xee, 0x05, 0xf1, 0x03, + 0xf8, 0x01, 0x5c, 0xf8, 0x3f, 0x3d, 0xae, 0xc4, 0x1d, 0xb1, 0xe5, 0x8f, 0xa0, 0x19, 0x8f, 0x1d, + 0x3b, 0xb9, 0xcd, 0x3c, 0xcf, 0xeb, 0xe7, 0x7d, 0xde, 0x0f, 0x0f, 0x80, 0x64, 0x4b, 0x7d, 0x9c, + 0x4a, 0xa1, 0x05, 0xb6, 0xcc, 0x39, 0xbd, 0x79, 0x3a, 0x8c, 0x44, 0x24, 0x2c, 0xf4, 0xa9, 0x39, + 0x65, 0xec, 0xe4, 0x37, 0x68, 0x7e, 0x9d, 0x68, 0xf9, 0x16, 0x3f, 0x01, 0xff, 0xfa, 0x6d, 0xca, + 0xa9, 0x37, 0xf6, 0xa6, 0x83, 0xd9, 0xc1, 0x71, 0xf6, 0xd5, 0xb1, 0x25, 0x0d, 0x71, 0xe2, 0x3f, + 0xfc, 0xf3, 0x51, 0x2d, 0xb0, 0x41, 0x48, 0xc1, 0xbf, 0xe6, 0x32, 0xa6, 0xf5, 0xb1, 0x37, 0xf5, + 0x0b, 0x86, 0xcb, 0x18, 0x9f, 0x42, 0xf3, 0x22, 0x09, 0xf9, 0x1b, 0xda, 0x28, 0x51, 0x19, 0x84, + 0x08, 0xfe, 0x19, 0xd3, 0x8c, 0xfa, 0x63, 0x6f, 0xda, 0x0f, 0xec, 0x79, 0xf2, 0xbb, 0x07, 0xe4, + 0x2a, 0x61, 0xa9, 0xba, 0x15, 0x7a, 0xce, 0x35, 0x0b, 0x99, 0x66, 0xf8, 0x05, 0xc0, 0x42, 0x24, + 0xcb, 0x9f, 0x94, 0x66, 0x3a, 0x73, 0xd4, 0xdb, 0x38, 0x3a, 0x15, 0xc9, 0xf2, 0xca, 0x10, 0x4e, + 0xbc, 0xbb, 0xc8, 0x01, 0x93, 0x7c, 0x65, 0x93, 0x97, 0x7d, 0x65, 0x90, 0xb1, 0xac, 0x8d, 0xe5, + 0xb2, 0x2f, 0x8b, 0x4c, 0xbe, 0x87, 0x4e, 0xee, 0xc0, 0x58, 0x34, 0x0e, 0x6c, 0xce, 0x7e, 0x60, + 0xcf, 0xf8, 0x15, 0x74, 0x62, 0xe7, 0xcc, 0x0a, 0xf7, 0x66, 0x34, 0xf7, 0xb2, 0xed, 0xdc, 0xe9, + 0x16, 0xf1, 0x93, 0xbf, 0x1a, 0xd0, 0x9e, 0x73, 0xa5, 0x58, 0xc4, 0xf1, 0x39, 0xf8, 0x7a, 0xd3, + 0xe1, 0x27, 0xb9, 0x86, 0xa3, 0xcb, 0x3d, 0x36, 0x61, 0x38, 0x84, 0xba, 0x16, 0x95, 0x4a, 0xea, + 0x5a, 0x98, 0x32, 0x96, 0x52, 0x6c, 0x95, 0x61, 0x90, 0xa2, 0x40, 0x7f, 0xbb, 0x40, 0x1c, 0x41, + 0xfb, 0x4e, 0x44, 0x76, 0x60, 0xcd, 0x12, 0x99, 0x83, 0x9b, 0xb6, 0xb5, 0x76, 0xdb, 0xf6, 0x1c, + 0xda, 0x3c, 0xd1, 0x72, 0xc5, 0x15, 0x6d, 0x8f, 0x1b, 0xd3, 0xde, 0x6c, 0xaf, 0xb2, 0x19, 0xb9, + 0x94, 0x8b, 0xc1, 0x67, 0xd0, 0x5a, 0x88, 0x38, 0x5e, 0x69, 0xda, 0x29, 0x69, 0x39, 0x0c, 0x67, + 0xd0, 0x51, 0xae, 0x63, 0xb4, 0x6b, 0x3b, 0x49, 0xb6, 0x3b, 0x99, 0x77, 0x30, 0x8f, 0x33, 0x8a, + 0x92, 0xff, 0xcc, 0x17, 0x9a, 0xc2, 0xd8, 0x9b, 0x76, 0x72, 0xc5, 0x0c, 0xc3, 0x8f, 0x01, 0xb2, + 0xd3, 0xf9, 0x2a, 0xd1, 0xb4, 0x57, 0xca, 0x59, 0xc2, 0x91, 0x42, 0x7b, 0x21, 0x12, 0xcd, 0xdf, + 0x68, 0xda, 0xb7, 0x83, 0xcd, 0xaf, 0x93, 0x1f, 0xa1, 0x7b, 0xce, 0x64, 0x98, 0xad, 0x4f, 0xde, + 0x41, 0x6f, 0xa7, 0x83, 0x14, 0xfc, 0x7b, 0xa1, 0x79, 0x75, 0xdf, 0x0d, 0x52, 0x2a, 0xb8, 0xb1, + 0x5b, 0xf0, 0xe4, 0x07, 0x68, 0x5d, 0x71, 0x79, 0xcf, 0xa5, 0x51, 0x48, 0x44, 0xc8, 0xab, 0xda, + 0x06, 0xc1, 0x2f, 0xa1, 0xa3, 0xd6, 0xcb, 0xa5, 0x64, 0x51, 0xa6, 0x3f, 0x98, 0x1d, 0x16, 0x4d, + 0x71, 0x78, 0x79, 0xdd, 0x8b, 0xe0, 0xc9, 0x37, 0xd0, 0x2d, 0xfe, 0x05, 0x1c, 0x42, 0xd3, 0xa8, + 0x29, 0xea, 0x8d, 0x1b, 0x53, 0x3f, 0xc8, 0x2e, 0x38, 0x85, 0xb6, 0xb2, 0xf9, 0x15, 0xad, 0xdb, + 0xe9, 0x0d, 0x0a, 0x69, 0x0b, 0x07, 0x39, 0x3d, 0xf9, 0xc3, 0x03, 0x30, 0x6a, 0xa7, 0xb7, 0x2c, + 0x89, 0xec, 0xf2, 0x5d, 0x9c, 0x55, 0xcc, 0xd6, 0x2f, 0xce, 0xf0, 0x33, 0xf7, 0x46, 0x64, 0x36, + 0x3f, 0x28, 0xff, 0x91, 0xd9, 0x77, 0x3b, 0x0f, 0xc5, 0x33, 0x68, 0x5d, 0x8a, 0x90, 0x5f, 0x9c, + 0x55, 0xdb, 0x93, 0x61, 0x66, 0x2e, 0xa7, 0x6e, 0x2e, 0xd9, 0x9b, 0x90, 0x5f, 0x8f, 0x3e, 0x87, + 0x6e, 0xf1, 0xf2, 0xe0, 0x3e, 0xf4, 0xec, 0xe5, 0x52, 0xc8, 0x98, 0xdd, 0x91, 0x1a, 0x3e, 0x81, + 0x7d, 0x0b, 0x6c, 0x12, 0x13, 0xef, 0xe8, 0xef, 0x3a, 0xf4, 0x4a, 0xff, 0x12, 0x02, 0xb4, 0xe6, + 0x2a, 0x3a, 0x5f, 0xa7, 0xa4, 0x86, 0x3d, 0x68, 0xcf, 0x55, 0x74, 0xc2, 0x99, 0x26, 0x9e, 0xbb, + 0xbc, 0x94, 0x22, 0x25, 0x75, 0x17, 0xf5, 0x22, 0x4d, 0x49, 0x03, 0x07, 0x00, 0xd9, 0x39, 0xe0, + 0x2a, 0x25, 0xbe, 0x0b, 0x7c, 0x2d, 0x34, 0x27, 0x4d, 0x63, 0xc2, 0x5d, 0x2c, 0xdb, 0x72, 0xac, + 0xd9, 0x5b, 0xd2, 0x46, 0x02, 0x7d, 0x93, 0x8c, 0x33, 0xa9, 0x6f, 0x4c, 0x96, 0x0e, 0x0e, 0x81, + 0x94, 0x11, 0xfb, 0x51, 0x17, 0x11, 0x06, 0x73, 0x15, 0xbd, 0x4a, 0x24, 0x67, 0x8b, 0x5b, 0x76, + 0x73, 0xc7, 0x09, 0xe0, 0x01, 0xec, 0x39, 0x21, 0x33, 0xca, 0xb5, 0x22, 0x3d, 0x17, 0x76, 0x7a, + 0xcb, 0x17, 0xbf, 0x7c, 0xb7, 0x16, 0x72, 0x1d, 0x93, 0x3e, 0x1e, 0xc2, 0xc1, 0x5c, 0x45, 0xd7, + 0x92, 0x25, 0x6a, 0xc9, 0xe5, 0xb7, 0x9c, 0x85, 0x5c, 0x92, 0x3d, 0xf7, 0xf5, 0xf5, 0x2a, 0xe6, + 0x62, 0xad, 0x2f, 0xc5, 0xaf, 0x64, 0xe0, 0xcc, 0x04, 0x9c, 0x85, 0xf6, 0xdd, 0x25, 0xfb, 0xce, + 0x4c, 0x81, 0x58, 0x33, 0xc4, 0xd5, 0xfb, 0x52, 0x72, 0x5b, 0xe2, 0x81, 0xcb, 0xea, 0xee, 0x36, + 0x06, 0x8f, 0xa6, 0xb0, 0x57, 0xd9, 0x42, 0xec, 0x43, 0xe7, 0x52, 0x24, 0x66, 0xf7, 0x25, 0xa9, + 0x61, 0x17, 0x9a, 0xaf, 0xed, 0xd1, 0x3b, 0xba, 0x87, 0x41, 0x75, 0x11, 0x8c, 0xe3, 0x0d, 0xf2, + 0x22, 0x0c, 0xcd, 0xd4, 0x49, 0x0d, 0x29, 0x0c, 0x37, 0x70, 0xc0, 0x63, 0x71, 0xcf, 0x2d, 0xe3, + 0x55, 0x99, 0x57, 0x69, 0xc8, 0x74, 0xc6, 0xd4, 0xf1, 0x43, 0x38, 0xdc, 0x92, 0x72, 0x16, 0x1a, + 0x27, 0xf4, 0xe1, 0xfd, 0xa8, 0xf6, 0xee, 0xfd, 0xa8, 0xf6, 0xf0, 0x38, 0xf2, 0xde, 0x3d, 0x8e, + 0xbc, 0x7f, 0x1f, 0x47, 0xde, 0x9f, 0xff, 0x8d, 0x6a, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0x10, + 0x3a, 0xce, 0xfa, 0x06, 0x07, 0x00, 0x00, } diff --git a/raft/raftpb/raft.proto b/raft/raftpb/raft.proto index 806a43634fd..da1afbabf20 100644 --- a/raft/raftpb/raft.proto +++ b/raft/raftpb/raft.proto @@ -75,14 +75,26 @@ message HardState { optional uint64 commit = 3 [(gogoproto.nullable) = false]; } +enum SuffrageState { + Nonvoter = 0; + Voter = 1; +} + +message Server { + optional uint64 node = 1 [(gogoproto.nullable) = false]; + optional SuffrageState suffrage = 2 [(gogoproto.nullable) = false]; +} + message ConfState { - repeated uint64 nodes = 1; + repeated uint64 nodes = 1; + repeated Server servers = 2; } enum ConfChangeType { - ConfChangeAddNode = 0; - ConfChangeRemoveNode = 1; - ConfChangeUpdateNode = 2; + ConfChangeAddNode = 0; + ConfChangeRemoveNode = 1; + ConfChangeUpdateNode = 2; + ConfChangeAddNonvoter = 3; } message ConfChange { diff --git a/raft/rawnode.go b/raft/rawnode.go index b950d5169a5..a89ae4c80a9 100644 --- a/raft/rawnode.go +++ b/raft/rawnode.go @@ -175,6 +175,8 @@ func (rn *RawNode) ApplyConfChange(cc pb.ConfChange) *pb.ConfState { switch cc.Type { case pb.ConfChangeAddNode: rn.raft.addNode(cc.NodeID) + case pb.ConfChangeAddNonvoter: + rn.raft.addNonvoter(cc.NodeID) case pb.ConfChangeRemoveNode: rn.raft.removeNode(cc.NodeID) case pb.ConfChangeUpdateNode: From c098dbb6463a9864390e8204090d03931d85998a Mon Sep 17 00:00:00 2001 From: gladiator Date: Tue, 26 Sep 2017 11:16:11 +0800 Subject: [PATCH 2/7] raft: promotable check raft suffrage --- raft/raft.go | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/raft/raft.go b/raft/raft.go index 8ad59eacd23..2afaaca83d3 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -587,9 +587,7 @@ func (r *raft) tickElection() { if r.promotable() && r.pastElectionTimeout() { r.electionElapsed = 0 - if r.suffrage == pb.Voter { - r.Step(pb.Message{From: r.id, Type: pb.MsgHup}) - } + r.Step(pb.Message{From: r.id, Type: pb.MsgHup}) } } @@ -1237,10 +1235,10 @@ func (r *raft) restore(s pb.Snapshot) bool { } // promotable indicates whether state machine can be promoted to leader, -// which is true when its own id is in progress list. +// which is true when its own id is in progress list and suffrage is Voter func (r *raft) promotable() bool { _, ok := r.prs[r.id] - return ok + return ok && r.suffrage == pb.Voter } func (r *raft) addNode(id uint64) { From b159733cff301b594517a998858b83780bcf0c75 Mon Sep 17 00:00:00 2001 From: gladiator Date: Sat, 30 Sep 2017 17:53:43 +0800 Subject: [PATCH 3/7] raft: use learner instead of suffrage --- raft/node.go | 4 +- raft/progress.go | 9 +- raft/raft.go | 114 ++++++------ raft/raft_test.go | 56 +++--- raft/raftpb/raft.pb.go | 414 +++++++++++++---------------------------- raft/raftpb/raft.proto | 22 +-- raft/rawnode.go | 4 +- 7 files changed, 225 insertions(+), 398 deletions(-) diff --git a/raft/node.go b/raft/node.go index f124c3b2586..2895dbe588b 100644 --- a/raft/node.go +++ b/raft/node.go @@ -334,8 +334,8 @@ func (n *node) run(r *raft) { switch cc.Type { case pb.ConfChangeAddNode: r.addNode(cc.NodeID) - case pb.ConfChangeAddNonvoter: - r.addNonvoter(cc.NodeID) + case pb.ConfChangeAddLearner: + r.addLearner(cc.NodeID) case pb.ConfChangeRemoveNode: // block incoming proposal when local node is // removed diff --git a/raft/progress.go b/raft/progress.go index 1425f91b6fe..3c8555a1f6d 100644 --- a/raft/progress.go +++ b/raft/progress.go @@ -14,11 +14,7 @@ package raft -import ( - "fmt" - - pb "github.com/coreos/etcd/raft/raftpb" -) +import "fmt" const ( ProgressStateProbe ProgressStateType = iota @@ -53,7 +49,6 @@ type Progress struct { // before and stops sending any replication message. State ProgressStateType - Suffrage pb.SuffrageState // Paused is used in ProgressStateProbe. // When Paused is true, raft should pause sending replication message to this peer. Paused bool @@ -82,6 +77,8 @@ type Progress struct { // be freed by calling inflights.freeTo with the index of the last // received entry. ins *inflights + + isLearner bool } func (pr *Progress) resetState(state ProgressStateType) { diff --git a/raft/raft.go b/raft/raft.go index 2afaaca83d3..c015f70d342 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -114,7 +114,9 @@ type Config struct { // should only be set when starting a new raft cluster. Restarting raft from // previous configuration will panic if peers is set. peer is private and only // used for testing right now. - peers []pb.Server + peers []uint64 + + learners []uint64 // ElectionTick is the number of Node.Tick invocations that must pass between // elections. That is, if a follower does not receive any message from the @@ -238,7 +240,7 @@ type raft struct { state StateType - suffrage pb.SuffrageState + isLearner bool votes map[uint64]bool @@ -290,33 +292,22 @@ func newRaft(c *Config) *raft { if err != nil { panic(err) // TODO(bdarnell) } - peers := c.peers - if len(cs.Nodes) > 0 && len(cs.Servers) > 0 { - panic("cannot specify both ConfState.Nodes and ConfState.Servers") - } - if len(cs.Nodes) > 0 { - if len(peers) > 0 { + voters := c.peers + learners := c.learners + if len(cs.Nodes) > 0 || len(cs.Learners) > 0 { + if len(voters) > 0 || len(learners) > 0 { // TODO(bdarnell): the peers argument is always nil except in // tests; the argument should be removed and these tests should be // updated to specify their nodes through a snapshot. - panic("cannot specify both newRaft(peers) and ConfState.Nodes)") - } - for _, n := range cs.Nodes { - peers = append(peers, pb.Server{Node: n, Suffrage: pb.Voter}) - } - } - if len(cs.Servers) > 0 { - if len(peers) > 0 { - panic("cannot specify both newRaft(peers) and ConfState.Servers)") - } - for _, s := range cs.Servers { - peers = append(peers, *s) + panic("cannot specify both newRaft(peers, learners) and ConfState.(Nodes, Learners)") } + voters = cs.Nodes + learners = cs.Learners } r := &raft{ id: c.ID, lead: None, - suffrage: pb.Voter, + isLearner: false, raftLog: raftlog, maxMsgSize: c.MaxSizePerMsg, maxInflight: c.MaxInflightMsgs, @@ -329,10 +320,16 @@ func newRaft(c *Config) *raft { readOnly: newReadOnly(c.ReadOnlyOption), disableProposalForwarding: c.DisableProposalForwarding, } - for _, p := range peers { - r.prs[p.Node] = &Progress{Next: 1, ins: newInflights(r.maxInflight), Suffrage: p.Suffrage} - if r.id == p.Node { - r.suffrage = p.Suffrage + for _, n := range voters { + r.prs[n] = &Progress{Next: 1, ins: newInflights(r.maxInflight), isLearner: false} + } + for _, n := range learners { + if _, has := r.prs[n]; has { + panic(fmt.Sprintf("cannot specify both Voter and Learner for node: %x", n)) + } + r.prs[n] = &Progress{Next: 1, ins: newInflights(r.maxInflight), isLearner: true} + if r.id == n { + r.isLearner = true } } if !isHardStateEqual(hs, emptyState) { @@ -369,7 +366,7 @@ func (r *raft) hardState() pb.HardState { func (r *raft) voterCount() int { count := 0 for _, p := range r.prs { - if p.Suffrage == pb.Voter { + if !p.isLearner { count++ } } @@ -536,7 +533,7 @@ func (r *raft) maybeCommit() bool { // TODO(bmizerany): optimize.. Currently naive mis := make(uint64Slice, 0, r.voterCount()) for _, p := range r.prs { - if p.Suffrage == pb.Voter { + if !p.isLearner { mis = append(mis, p.Match) } } @@ -560,7 +557,7 @@ func (r *raft) reset(term uint64) { r.votes = make(map[uint64]bool) for id := range r.prs { - r.prs[id] = &Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight), Suffrage: r.prs[id].Suffrage} + r.prs[id] = &Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight), isLearner: r.prs[id].isLearner} if id == r.id { r.prs[id].Match = r.raftLog.lastIndex() } @@ -708,7 +705,7 @@ func (r *raft) campaign(t CampaignType) { if id == r.id { continue } - if r.prs[id].Suffrage != pb.Voter { + if r.prs[id].isLearner { continue } r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d", @@ -1025,8 +1022,8 @@ func stepLeader(r *raft, m pb.Message) { } r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr) case pb.MsgTransferLeader: - if pr.Suffrage != pb.Voter { - r.logger.Debugf("%x is not Voter. Ignored transferring leadership", r.id) + if pr.isLearner { + r.logger.Debugf("%x is Learner. Ignored transferring leadership", r.id) return } leadTransferee := m.From @@ -1210,64 +1207,57 @@ func (r *raft) restore(s pb.Snapshot) bool { r.raftLog.restore(s) r.prs = make(map[uint64]*Progress) - if len(s.Metadata.ConfState.Nodes) > 0 && len(s.Metadata.ConfState.Servers) > 0 { - panic("cannot specify both ConfState.Nodes and ConfState.Servers") - } - for _, n := range s.Metadata.ConfState.Nodes { + r.restoreNode(s.Metadata.ConfState.Nodes, false) + r.restoreNode(s.Metadata.ConfState.Learners, true) + return true +} + +func (r *raft) restoreNode(nodes []uint64, isLearner bool) { + for _, n := range nodes { match, next := uint64(0), r.raftLog.lastIndex()+1 if n == r.id { match = next - 1 - r.suffrage = pb.Voter + r.isLearner = isLearner } - r.setProgress(n, match, next, pb.Voter) + r.setProgress(n, match, next, isLearner) r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n]) } - for _, server := range s.Metadata.ConfState.Servers { - match, next := uint64(0), r.raftLog.lastIndex()+1 - if server.Node == r.id { - match = next - 1 - r.suffrage = server.Suffrage - } - r.setProgress(server.Node, match, next, server.Suffrage) - r.logger.Infof("%x restored progress of %x [%s]", r.id, server.Node, r.prs[server.Node]) - } - return true } // promotable indicates whether state machine can be promoted to leader, -// which is true when its own id is in progress list and suffrage is Voter +// which is true when its own id is in progress list and its not learner. func (r *raft) promotable() bool { _, ok := r.prs[r.id] - return ok && r.suffrage == pb.Voter + return ok && !r.isLearner } func (r *raft) addNode(id uint64) { - r.addNodeWithSuffrage(id, pb.Voter) + r.addLearnerNode(id, false) } -func (r *raft) addNonvoter(id uint64) { - r.addNodeWithSuffrage(id, pb.Nonvoter) +func (r *raft) addLearner(id uint64) { + r.addLearnerNode(id, true) } -func (r *raft) addNodeWithSuffrage(id uint64, suffrage pb.SuffrageState) { +func (r *raft) addLearnerNode(id uint64, isLearner bool) { r.pendingConf = false if _, ok := r.prs[id]; ok { - if r.prs[id].Suffrage == suffrage { + if r.prs[id].isLearner == isLearner { // Ignore any redundant addNode calls (which can happen because the // initial bootstrapping entries are applied twice). return } - if suffrage != pb.Voter { - // addNode can only change suffrage from Nonvoter to Voter + if isLearner { + // can only change Learner to Voter return } - r.prs[id].Suffrage = suffrage + r.prs[id].isLearner = isLearner } else { - r.setProgress(id, 0, r.raftLog.lastIndex()+1, suffrage) + r.setProgress(id, 0, r.raftLog.lastIndex()+1, isLearner) } if r.id == id { - r.suffrage = suffrage + r.isLearner = isLearner } // When a node is first added, we should mark it as recently active. // Otherwise, CheckQuorum may cause us to step down if it is invoked @@ -1297,8 +1287,8 @@ func (r *raft) removeNode(id uint64) { func (r *raft) resetPendingConf() { r.pendingConf = false } -func (r *raft) setProgress(id, match, next uint64, suffrage pb.SuffrageState) { - r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight), Suffrage: suffrage} +func (r *raft) setProgress(id, match, next uint64, isLearner bool) { + r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight), isLearner: isLearner} } func (r *raft) delProgress(id uint64) { @@ -1338,7 +1328,7 @@ func (r *raft) checkQuorumActive() bool { continue } - if r.prs[id].RecentActive && r.prs[id].Suffrage == pb.Voter { + if r.prs[id].RecentActive && !r.prs[id].isLearner { act++ } diff --git a/raft/raft_test.go b/raft/raft_test.go index ce14ed2d2dc..cf6c35a36ff 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -350,14 +350,14 @@ func testLeaderElection(t *testing.T, preVote bool) { func TestNonvoterElectionTimeout(t *testing.T) { n1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - n1.prs[3].Suffrage = pb.Nonvoter + n1.prs[3].isLearner = true n2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - n2.prs[3].Suffrage = pb.Nonvoter + n2.prs[3].isLearner = true n3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - n3.suffrage = pb.Nonvoter - n3.prs[3].Suffrage = pb.Nonvoter + n3.isLearner = true + n3.prs[3].isLearner = true n1.becomeFollower(1, None) n2.becomeFollower(1, None) @@ -1106,7 +1106,7 @@ func TestCommit(t *testing.T) { sm := newTestRaft(1, []uint64{1}, 5, 1, storage) for j := 0; j < len(tt.matches); j++ { - sm.setProgress(uint64(j)+1, tt.matches[j], tt.matches[j]+1, pb.Voter) + sm.setProgress(uint64(j)+1, tt.matches[j], tt.matches[j]+1, false) } sm.maybeCommit() if g := sm.raftLog.committed; g != tt.w { @@ -2377,13 +2377,9 @@ func TestRestore(t *testing.T) { func TestRestoreWithNonvoter(t *testing.T) { s := pb.Snapshot{ Metadata: pb.SnapshotMetadata{ - Index: 11, // magic number - Term: 11, // magic number - ConfState: pb.ConfState{Servers: []*pb.Server{ - {Node: 1, Suffrage: pb.Voter}, - {Node: 2, Suffrage: pb.Voter}, - {Node: 3, Suffrage: pb.Nonvoter}, - }}, + Index: 11, // magic number + Term: 11, // magic number + ConfState: pb.ConfState{Nodes: []uint64{1, 2}, Learners: []uint64{3}}, }, } @@ -2399,9 +2395,18 @@ func TestRestoreWithNonvoter(t *testing.T) { if mustTerm(sm.raftLog.term(s.Metadata.Index)) != s.Metadata.Term { t.Errorf("log.lastTerm = %d, want %d", mustTerm(sm.raftLog.term(s.Metadata.Index)), s.Metadata.Term) } - for _, server := range s.Metadata.ConfState.Servers { - if sm.prs[server.Node].Suffrage != server.Suffrage { - t.Errorf("sm.Node %x suffrage = %s, want %s", server.Node, sm.prs[server.Node], server.Suffrage) + sg := sm.nodes() + if len(sg) != len(s.Metadata.ConfState.Nodes)+len(s.Metadata.ConfState.Learners) { + t.Errorf("sm.Nodes = %+v, length not equal with %+v", sg, s.Metadata.ConfState) + } + for _, n := range s.Metadata.ConfState.Nodes { + if sm.prs[n].isLearner { + t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs[n], true) + } + } + for _, n := range s.Metadata.ConfState.Learners { + if !sm.prs[n].isLearner { + t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs[n], false) } } @@ -2656,7 +2661,7 @@ func TestAddNode(t *testing.T) { func TestAddNonvoter(t *testing.T) { r := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) r.pendingConf = true - r.addNonvoter(2) + r.addLearner(2) if r.pendingConf { t.Errorf("pendingConf = %v, want false", r.pendingConf) } @@ -2665,8 +2670,8 @@ func TestAddNonvoter(t *testing.T) { if !reflect.DeepEqual(nodes, wnodes) { t.Errorf("nodes = %v, want %v", nodes, wnodes) } - if r.prs[2].Suffrage != pb.Nonvoter { - t.Fatalf("node 2 has suffrage %s, want %s", r.prs[2].Suffrage, pb.Nonvoter) + if !r.prs[2].isLearner { + t.Fatalf("node 2 has suffrage %t, want %t", r.prs[2].isLearner, true) } } @@ -3439,14 +3444,17 @@ func newNetworkWithConfig(configFunc func(*Config), peers ...stateMachine) *netw sm := newRaft(cfg) npeers[id] = sm case *raft: - suffrages := make(map[uint64]pb.SuffrageState, size) + learners := make(map[uint64]bool, 0) for i, pr := range p.(*raft).prs { - suffrages[i] = pr.Suffrage + if pr.isLearner { + learners[i] = true + } } v.id = id v.prs = make(map[uint64]*Progress) for i := 0; i < size; i++ { - v.prs[peerAddrs[i]] = &Progress{Suffrage: suffrages[peerAddrs[i]]} + _, isLearner := learners[peerAddrs[i]] + v.prs[peerAddrs[i]] = &Progress{isLearner: isLearner} } v.reset(v.Term) npeers[id] = v @@ -3553,13 +3561,9 @@ func setRandomizedElectionTimeout(r *raft, v int) { } func newTestConfig(id uint64, peers []uint64, election, heartbeat int, storage Storage) *Config { - new_peers := make([]pb.Server, 0) - for _, p := range peers { - new_peers = append(new_peers, pb.Server{Node: p, Suffrage: pb.Voter}) - } return &Config{ ID: id, - peers: new_peers, + peers: peers, ElectionTick: election, HeartbeatTick: heartbeat, Storage: storage, diff --git a/raft/raftpb/raft.pb.go b/raft/raftpb/raft.pb.go index 5341d9bf730..9830ca65426 100644 --- a/raft/raftpb/raft.pb.go +++ b/raft/raftpb/raft.pb.go @@ -14,7 +14,6 @@ Snapshot Message HardState - Server ConfState ConfChange */ @@ -160,60 +159,26 @@ func (x *MessageType) UnmarshalJSON(data []byte) error { } func (MessageType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{1} } -type SuffrageState int32 - -const ( - Nonvoter SuffrageState = 0 - Voter SuffrageState = 1 -) - -var SuffrageState_name = map[int32]string{ - 0: "Nonvoter", - 1: "Voter", -} -var SuffrageState_value = map[string]int32{ - "Nonvoter": 0, - "Voter": 1, -} - -func (x SuffrageState) Enum() *SuffrageState { - p := new(SuffrageState) - *p = x - return p -} -func (x SuffrageState) String() string { - return proto.EnumName(SuffrageState_name, int32(x)) -} -func (x *SuffrageState) UnmarshalJSON(data []byte) error { - value, err := proto.UnmarshalJSONEnum(SuffrageState_value, data, "SuffrageState") - if err != nil { - return err - } - *x = SuffrageState(value) - return nil -} -func (SuffrageState) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{2} } - type ConfChangeType int32 const ( - ConfChangeAddNode ConfChangeType = 0 - ConfChangeRemoveNode ConfChangeType = 1 - ConfChangeUpdateNode ConfChangeType = 2 - ConfChangeAddNonvoter ConfChangeType = 3 + ConfChangeAddNode ConfChangeType = 0 + ConfChangeRemoveNode ConfChangeType = 1 + ConfChangeUpdateNode ConfChangeType = 2 + ConfChangeAddLearner ConfChangeType = 3 ) var ConfChangeType_name = map[int32]string{ 0: "ConfChangeAddNode", 1: "ConfChangeRemoveNode", 2: "ConfChangeUpdateNode", - 3: "ConfChangeAddNonvoter", + 3: "ConfChangeAddLearner", } var ConfChangeType_value = map[string]int32{ - "ConfChangeAddNode": 0, - "ConfChangeRemoveNode": 1, - "ConfChangeUpdateNode": 2, - "ConfChangeAddNonvoter": 3, + "ConfChangeAddNode": 0, + "ConfChangeRemoveNode": 1, + "ConfChangeUpdateNode": 2, + "ConfChangeAddLearner": 3, } func (x ConfChangeType) Enum() *ConfChangeType { @@ -232,7 +197,7 @@ func (x *ConfChangeType) UnmarshalJSON(data []byte) error { *x = ConfChangeType(value) return nil } -func (ConfChangeType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{3} } +func (ConfChangeType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{2} } type Entry struct { Term uint64 `protobuf:"varint,2,opt,name=Term" json:"Term"` @@ -303,27 +268,16 @@ func (m *HardState) String() string { return proto.CompactTextString( func (*HardState) ProtoMessage() {} func (*HardState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{4} } -type Server struct { - Node uint64 `protobuf:"varint,1,opt,name=node" json:"node"` - Suffrage SuffrageState `protobuf:"varint,2,opt,name=suffrage,enum=raftpb.SuffrageState" json:"suffrage"` - XXX_unrecognized []byte `json:"-"` -} - -func (m *Server) Reset() { *m = Server{} } -func (m *Server) String() string { return proto.CompactTextString(m) } -func (*Server) ProtoMessage() {} -func (*Server) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{5} } - type ConfState struct { - Nodes []uint64 `protobuf:"varint,1,rep,name=nodes" json:"nodes,omitempty"` - Servers []*Server `protobuf:"bytes,2,rep,name=servers" json:"servers,omitempty"` - XXX_unrecognized []byte `json:"-"` + Nodes []uint64 `protobuf:"varint,1,rep,name=nodes" json:"nodes,omitempty"` + Learners []uint64 `protobuf:"varint,2,rep,name=learners" json:"learners,omitempty"` + XXX_unrecognized []byte `json:"-"` } func (m *ConfState) Reset() { *m = ConfState{} } func (m *ConfState) String() string { return proto.CompactTextString(m) } func (*ConfState) ProtoMessage() {} -func (*ConfState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{6} } +func (*ConfState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{5} } type ConfChange struct { ID uint64 `protobuf:"varint,1,opt,name=ID" json:"ID"` @@ -336,7 +290,7 @@ type ConfChange struct { func (m *ConfChange) Reset() { *m = ConfChange{} } func (m *ConfChange) String() string { return proto.CompactTextString(m) } func (*ConfChange) ProtoMessage() {} -func (*ConfChange) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{7} } +func (*ConfChange) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{6} } func init() { proto.RegisterType((*Entry)(nil), "raftpb.Entry") @@ -344,12 +298,10 @@ func init() { proto.RegisterType((*Snapshot)(nil), "raftpb.Snapshot") proto.RegisterType((*Message)(nil), "raftpb.Message") proto.RegisterType((*HardState)(nil), "raftpb.HardState") - proto.RegisterType((*Server)(nil), "raftpb.Server") proto.RegisterType((*ConfState)(nil), "raftpb.ConfState") proto.RegisterType((*ConfChange)(nil), "raftpb.ConfChange") proto.RegisterEnum("raftpb.EntryType", EntryType_name, EntryType_value) proto.RegisterEnum("raftpb.MessageType", MessageType_name, MessageType_value) - proto.RegisterEnum("raftpb.SuffrageState", SuffrageState_name, SuffrageState_value) proto.RegisterEnum("raftpb.ConfChangeType", ConfChangeType_name, ConfChangeType_value) } func (m *Entry) Marshal() (dAtA []byte, err error) { @@ -567,33 +519,6 @@ func (m *HardState) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *Server) 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 *Server) MarshalTo(dAtA []byte) (int, error) { - var i int - _ = i - var l int - _ = l - dAtA[i] = 0x8 - i++ - i = encodeVarintRaft(dAtA, i, uint64(m.Node)) - dAtA[i] = 0x10 - i++ - i = encodeVarintRaft(dAtA, i, uint64(m.Suffrage)) - if m.XXX_unrecognized != nil { - i += copy(dAtA[i:], m.XXX_unrecognized) - } - return i, nil -} - func (m *ConfState) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -616,16 +541,11 @@ func (m *ConfState) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintRaft(dAtA, i, uint64(num)) } } - if len(m.Servers) > 0 { - for _, msg := range m.Servers { - dAtA[i] = 0x12 + if len(m.Learners) > 0 { + for _, num := range m.Learners { + dAtA[i] = 0x10 i++ - i = encodeVarintRaft(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + i = encodeVarintRaft(dAtA, i, uint64(num)) } } if m.XXX_unrecognized != nil { @@ -783,17 +703,6 @@ func (m *HardState) Size() (n int) { return n } -func (m *Server) Size() (n int) { - var l int - _ = l - n += 1 + sovRaft(uint64(m.Node)) - n += 1 + sovRaft(uint64(m.Suffrage)) - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } - return n -} - func (m *ConfState) Size() (n int) { var l int _ = l @@ -802,10 +711,9 @@ func (m *ConfState) Size() (n int) { n += 1 + sovRaft(uint64(e)) } } - if len(m.Servers) > 0 { - for _, e := range m.Servers { - l = e.Size() - n += 1 + l + sovRaft(uint64(l)) + if len(m.Learners) > 0 { + for _, e := range m.Learners { + n += 1 + sovRaft(uint64(e)) } } if m.XXX_unrecognized != nil { @@ -1636,95 +1544,6 @@ func (m *HardState) Unmarshal(dAtA []byte) error { } return nil } -func (m *Server) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRaft - } - 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) - if wireType == 4 { - return fmt.Errorf("proto: Server: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Server: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Node", wireType) - } - m.Node = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRaft - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Node |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Suffrage", wireType) - } - m.Suffrage = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRaft - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Suffrage |= (SuffrageState(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - default: - iNdEx = preIndex - skippy, err := skipRaft(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthRaft - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} func (m *ConfState) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -1817,36 +1636,67 @@ func (m *ConfState) Unmarshal(dAtA []byte) error { return fmt.Errorf("proto: wrong wireType = %d for field Nodes", wireType) } case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Servers", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRaft + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { + m.Learners = append(m.Learners, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + packedLen + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - msglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Learners = append(m.Learners, v) } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Learners", wireType) } - if msglen < 0 { - return ErrInvalidLengthRaft - } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Servers = append(m.Servers, &Server{}) - if err := m.Servers[len(m.Servers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRaft(dAtA[iNdEx:]) @@ -2116,60 +1966,56 @@ var ( func init() { proto.RegisterFile("raft.proto", fileDescriptorRaft) } var fileDescriptorRaft = []byte{ - // 871 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x54, 0x4d, 0x6f, 0xe3, 0x44, - 0x18, 0x8e, 0x13, 0xe7, 0xeb, 0x4d, 0x9a, 0x4e, 0xdf, 0x4d, 0xd1, 0xb0, 0x5a, 0x85, 0x28, 0xe2, - 0x10, 0x15, 0x6d, 0x81, 0x1c, 0x40, 0xe2, 0xb6, 0x6d, 0x91, 0x5a, 0x41, 0xaa, 0xc5, 0xed, 0xee, - 0x01, 0x84, 0xd0, 0x34, 0x9e, 0xb8, 0x81, 0xda, 0x63, 0xcd, 0x4c, 0xca, 0xee, 0x05, 0xf1, 0x03, - 0xf8, 0x01, 0x5c, 0xf8, 0x3f, 0x3d, 0xae, 0xc4, 0x1d, 0xb1, 0xe5, 0x8f, 0xa0, 0x19, 0x8f, 0x1d, - 0x3b, 0xb9, 0xcd, 0x3c, 0xcf, 0xeb, 0xe7, 0x7d, 0xde, 0x0f, 0x0f, 0x80, 0x64, 0x4b, 0x7d, 0x9c, - 0x4a, 0xa1, 0x05, 0xb6, 0xcc, 0x39, 0xbd, 0x79, 0x3a, 0x8c, 0x44, 0x24, 0x2c, 0xf4, 0xa9, 0x39, - 0x65, 0xec, 0xe4, 0x37, 0x68, 0x7e, 0x9d, 0x68, 0xf9, 0x16, 0x3f, 0x01, 0xff, 0xfa, 0x6d, 0xca, - 0xa9, 0x37, 0xf6, 0xa6, 0x83, 0xd9, 0xc1, 0x71, 0xf6, 0xd5, 0xb1, 0x25, 0x0d, 0x71, 0xe2, 0x3f, - 0xfc, 0xf3, 0x51, 0x2d, 0xb0, 0x41, 0x48, 0xc1, 0xbf, 0xe6, 0x32, 0xa6, 0xf5, 0xb1, 0x37, 0xf5, - 0x0b, 0x86, 0xcb, 0x18, 0x9f, 0x42, 0xf3, 0x22, 0x09, 0xf9, 0x1b, 0xda, 0x28, 0x51, 0x19, 0x84, - 0x08, 0xfe, 0x19, 0xd3, 0x8c, 0xfa, 0x63, 0x6f, 0xda, 0x0f, 0xec, 0x79, 0xf2, 0xbb, 0x07, 0xe4, - 0x2a, 0x61, 0xa9, 0xba, 0x15, 0x7a, 0xce, 0x35, 0x0b, 0x99, 0x66, 0xf8, 0x05, 0xc0, 0x42, 0x24, - 0xcb, 0x9f, 0x94, 0x66, 0x3a, 0x73, 0xd4, 0xdb, 0x38, 0x3a, 0x15, 0xc9, 0xf2, 0xca, 0x10, 0x4e, - 0xbc, 0xbb, 0xc8, 0x01, 0x93, 0x7c, 0x65, 0x93, 0x97, 0x7d, 0x65, 0x90, 0xb1, 0xac, 0x8d, 0xe5, - 0xb2, 0x2f, 0x8b, 0x4c, 0xbe, 0x87, 0x4e, 0xee, 0xc0, 0x58, 0x34, 0x0e, 0x6c, 0xce, 0x7e, 0x60, - 0xcf, 0xf8, 0x15, 0x74, 0x62, 0xe7, 0xcc, 0x0a, 0xf7, 0x66, 0x34, 0xf7, 0xb2, 0xed, 0xdc, 0xe9, - 0x16, 0xf1, 0x93, 0xbf, 0x1a, 0xd0, 0x9e, 0x73, 0xa5, 0x58, 0xc4, 0xf1, 0x39, 0xf8, 0x7a, 0xd3, - 0xe1, 0x27, 0xb9, 0x86, 0xa3, 0xcb, 0x3d, 0x36, 0x61, 0x38, 0x84, 0xba, 0x16, 0x95, 0x4a, 0xea, - 0x5a, 0x98, 0x32, 0x96, 0x52, 0x6c, 0x95, 0x61, 0x90, 0xa2, 0x40, 0x7f, 0xbb, 0x40, 0x1c, 0x41, - 0xfb, 0x4e, 0x44, 0x76, 0x60, 0xcd, 0x12, 0x99, 0x83, 0x9b, 0xb6, 0xb5, 0x76, 0xdb, 0xf6, 0x1c, - 0xda, 0x3c, 0xd1, 0x72, 0xc5, 0x15, 0x6d, 0x8f, 0x1b, 0xd3, 0xde, 0x6c, 0xaf, 0xb2, 0x19, 0xb9, - 0x94, 0x8b, 0xc1, 0x67, 0xd0, 0x5a, 0x88, 0x38, 0x5e, 0x69, 0xda, 0x29, 0x69, 0x39, 0x0c, 0x67, - 0xd0, 0x51, 0xae, 0x63, 0xb4, 0x6b, 0x3b, 0x49, 0xb6, 0x3b, 0x99, 0x77, 0x30, 0x8f, 0x33, 0x8a, - 0x92, 0xff, 0xcc, 0x17, 0x9a, 0xc2, 0xd8, 0x9b, 0x76, 0x72, 0xc5, 0x0c, 0xc3, 0x8f, 0x01, 0xb2, - 0xd3, 0xf9, 0x2a, 0xd1, 0xb4, 0x57, 0xca, 0x59, 0xc2, 0x91, 0x42, 0x7b, 0x21, 0x12, 0xcd, 0xdf, - 0x68, 0xda, 0xb7, 0x83, 0xcd, 0xaf, 0x93, 0x1f, 0xa1, 0x7b, 0xce, 0x64, 0x98, 0xad, 0x4f, 0xde, - 0x41, 0x6f, 0xa7, 0x83, 0x14, 0xfc, 0x7b, 0xa1, 0x79, 0x75, 0xdf, 0x0d, 0x52, 0x2a, 0xb8, 0xb1, - 0x5b, 0xf0, 0xe4, 0x07, 0x68, 0x5d, 0x71, 0x79, 0xcf, 0xa5, 0x51, 0x48, 0x44, 0xc8, 0xab, 0xda, - 0x06, 0xc1, 0x2f, 0xa1, 0xa3, 0xd6, 0xcb, 0xa5, 0x64, 0x51, 0xa6, 0x3f, 0x98, 0x1d, 0x16, 0x4d, - 0x71, 0x78, 0x79, 0xdd, 0x8b, 0xe0, 0xc9, 0x37, 0xd0, 0x2d, 0xfe, 0x05, 0x1c, 0x42, 0xd3, 0xa8, - 0x29, 0xea, 0x8d, 0x1b, 0x53, 0x3f, 0xc8, 0x2e, 0x38, 0x85, 0xb6, 0xb2, 0xf9, 0x15, 0xad, 0xdb, - 0xe9, 0x0d, 0x0a, 0x69, 0x0b, 0x07, 0x39, 0x3d, 0xf9, 0xc3, 0x03, 0x30, 0x6a, 0xa7, 0xb7, 0x2c, - 0x89, 0xec, 0xf2, 0x5d, 0x9c, 0x55, 0xcc, 0xd6, 0x2f, 0xce, 0xf0, 0x33, 0xf7, 0x46, 0x64, 0x36, - 0x3f, 0x28, 0xff, 0x91, 0xd9, 0x77, 0x3b, 0x0f, 0xc5, 0x33, 0x68, 0x5d, 0x8a, 0x90, 0x5f, 0x9c, - 0x55, 0xdb, 0x93, 0x61, 0x66, 0x2e, 0xa7, 0x6e, 0x2e, 0xd9, 0x9b, 0x90, 0x5f, 0x8f, 0x3e, 0x87, - 0x6e, 0xf1, 0xf2, 0xe0, 0x3e, 0xf4, 0xec, 0xe5, 0x52, 0xc8, 0x98, 0xdd, 0x91, 0x1a, 0x3e, 0x81, - 0x7d, 0x0b, 0x6c, 0x12, 0x13, 0xef, 0xe8, 0xef, 0x3a, 0xf4, 0x4a, 0xff, 0x12, 0x02, 0xb4, 0xe6, - 0x2a, 0x3a, 0x5f, 0xa7, 0xa4, 0x86, 0x3d, 0x68, 0xcf, 0x55, 0x74, 0xc2, 0x99, 0x26, 0x9e, 0xbb, - 0xbc, 0x94, 0x22, 0x25, 0x75, 0x17, 0xf5, 0x22, 0x4d, 0x49, 0x03, 0x07, 0x00, 0xd9, 0x39, 0xe0, - 0x2a, 0x25, 0xbe, 0x0b, 0x7c, 0x2d, 0x34, 0x27, 0x4d, 0x63, 0xc2, 0x5d, 0x2c, 0xdb, 0x72, 0xac, - 0xd9, 0x5b, 0xd2, 0x46, 0x02, 0x7d, 0x93, 0x8c, 0x33, 0xa9, 0x6f, 0x4c, 0x96, 0x0e, 0x0e, 0x81, - 0x94, 0x11, 0xfb, 0x51, 0x17, 0x11, 0x06, 0x73, 0x15, 0xbd, 0x4a, 0x24, 0x67, 0x8b, 0x5b, 0x76, - 0x73, 0xc7, 0x09, 0xe0, 0x01, 0xec, 0x39, 0x21, 0x33, 0xca, 0xb5, 0x22, 0x3d, 0x17, 0x76, 0x7a, - 0xcb, 0x17, 0xbf, 0x7c, 0xb7, 0x16, 0x72, 0x1d, 0x93, 0x3e, 0x1e, 0xc2, 0xc1, 0x5c, 0x45, 0xd7, - 0x92, 0x25, 0x6a, 0xc9, 0xe5, 0xb7, 0x9c, 0x85, 0x5c, 0x92, 0x3d, 0xf7, 0xf5, 0xf5, 0x2a, 0xe6, - 0x62, 0xad, 0x2f, 0xc5, 0xaf, 0x64, 0xe0, 0xcc, 0x04, 0x9c, 0x85, 0xf6, 0xdd, 0x25, 0xfb, 0xce, - 0x4c, 0x81, 0x58, 0x33, 0xc4, 0xd5, 0xfb, 0x52, 0x72, 0x5b, 0xe2, 0x81, 0xcb, 0xea, 0xee, 0x36, - 0x06, 0x8f, 0xa6, 0xb0, 0x57, 0xd9, 0x42, 0xec, 0x43, 0xe7, 0x52, 0x24, 0x66, 0xf7, 0x25, 0xa9, - 0x61, 0x17, 0x9a, 0xaf, 0xed, 0xd1, 0x3b, 0xba, 0x87, 0x41, 0x75, 0x11, 0x8c, 0xe3, 0x0d, 0xf2, - 0x22, 0x0c, 0xcd, 0xd4, 0x49, 0x0d, 0x29, 0x0c, 0x37, 0x70, 0xc0, 0x63, 0x71, 0xcf, 0x2d, 0xe3, - 0x55, 0x99, 0x57, 0x69, 0xc8, 0x74, 0xc6, 0xd4, 0xf1, 0x43, 0x38, 0xdc, 0x92, 0x72, 0x16, 0x1a, - 0x27, 0xf4, 0xe1, 0xfd, 0xa8, 0xf6, 0xee, 0xfd, 0xa8, 0xf6, 0xf0, 0x38, 0xf2, 0xde, 0x3d, 0x8e, - 0xbc, 0x7f, 0x1f, 0x47, 0xde, 0x9f, 0xff, 0x8d, 0x6a, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0x10, - 0x3a, 0xce, 0xfa, 0x06, 0x07, 0x00, 0x00, + // 813 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x54, 0xcd, 0x8e, 0xe3, 0x44, + 0x10, 0x8e, 0x1d, 0xe7, 0xaf, 0x9c, 0xc9, 0xf4, 0xf4, 0x06, 0xd4, 0x1a, 0xad, 0x42, 0x14, 0x71, + 0x88, 0x06, 0xed, 0x00, 0x73, 0xe0, 0x80, 0xc4, 0x61, 0x26, 0x41, 0x9a, 0x48, 0x9b, 0xd1, 0x92, + 0xcd, 0x72, 0x40, 0x42, 0xa8, 0x27, 0xae, 0x38, 0x81, 0xd8, 0x6d, 0xb5, 0x3b, 0xcb, 0xee, 0x05, + 0xf1, 0x00, 0x3c, 0x00, 0x17, 0xde, 0x67, 0x8e, 0x2b, 0x71, 0x47, 0xcc, 0xf0, 0x22, 0xa8, 0xdb, + 0xed, 0xc4, 0x4e, 0x6e, 0x5d, 0xdf, 0x57, 0x5d, 0xf5, 0xd5, 0xd7, 0x65, 0x03, 0x48, 0xbe, 0x54, + 0x97, 0x89, 0x14, 0x4a, 0xd0, 0xba, 0x3e, 0x27, 0xf7, 0xe7, 0xdd, 0x50, 0x84, 0xc2, 0x40, 0x9f, + 0xeb, 0x53, 0xc6, 0x0e, 0x7e, 0x83, 0xda, 0xb7, 0xb1, 0x92, 0xef, 0xe9, 0x67, 0xe0, 0xcd, 0xdf, + 0x27, 0xc8, 0x9c, 0xbe, 0x33, 0xec, 0x5c, 0x9d, 0x5d, 0x66, 0xb7, 0x2e, 0x0d, 0xa9, 0x89, 0x1b, + 0xef, 0xe1, 0x9f, 0x4f, 0x2a, 0x33, 0x93, 0x44, 0x19, 0x78, 0x73, 0x94, 0x11, 0x73, 0xfb, 0xce, + 0xd0, 0xdb, 0x31, 0x28, 0x23, 0x7a, 0x0e, 0xb5, 0x49, 0x1c, 0xe0, 0x3b, 0x56, 0x2d, 0x50, 0x19, + 0x44, 0x29, 0x78, 0x63, 0xae, 0x38, 0xf3, 0xfa, 0xce, 0xb0, 0x3d, 0x33, 0xe7, 0xc1, 0xef, 0x0e, + 0x90, 0xd7, 0x31, 0x4f, 0xd2, 0x95, 0x50, 0x53, 0x54, 0x3c, 0xe0, 0x8a, 0xd3, 0xaf, 0x00, 0x16, + 0x22, 0x5e, 0xfe, 0x94, 0x2a, 0xae, 0x32, 0x45, 0xfe, 0x5e, 0xd1, 0x48, 0xc4, 0xcb, 0xd7, 0x9a, + 0xb0, 0xc5, 0x5b, 0x8b, 0x1c, 0xd0, 0xcd, 0xd7, 0xa6, 0x79, 0x51, 0x57, 0x06, 0x69, 0xc9, 0x4a, + 0x4b, 0x2e, 0xea, 0x32, 0xc8, 0xe0, 0x07, 0x68, 0xe6, 0x0a, 0xb4, 0x44, 0xad, 0xc0, 0xf4, 0x6c, + 0xcf, 0xcc, 0x99, 0x7e, 0x0d, 0xcd, 0xc8, 0x2a, 0x33, 0x85, 0xfd, 0x2b, 0x96, 0x6b, 0x39, 0x54, + 0x6e, 0xeb, 0xee, 0xf2, 0x07, 0x7f, 0x55, 0xa1, 0x31, 0xc5, 0x34, 0xe5, 0x21, 0xd2, 0x17, 0xe0, + 0xa9, 0xbd, 0xc3, 0xcf, 0xf2, 0x1a, 0x96, 0x2e, 0x7a, 0xac, 0xd3, 0x68, 0x17, 0x5c, 0x25, 0x4a, + 0x93, 0xb8, 0x4a, 0xe8, 0x31, 0x96, 0x52, 0x1c, 0x8c, 0xa1, 0x91, 0xdd, 0x80, 0xde, 0xe1, 0x80, + 0xb4, 0x07, 0x8d, 0x8d, 0x08, 0xcd, 0x83, 0xd5, 0x0a, 0x64, 0x0e, 0xee, 0x6d, 0xab, 0x1f, 0xdb, + 0xf6, 0x02, 0x1a, 0x18, 0x2b, 0xb9, 0xc6, 0x94, 0x35, 0xfa, 0xd5, 0xa1, 0x7f, 0x75, 0x52, 0xda, + 0x8c, 0xbc, 0x94, 0xcd, 0xa1, 0xcf, 0xa1, 0xbe, 0x10, 0x51, 0xb4, 0x56, 0xac, 0x59, 0xa8, 0x65, + 0x31, 0x7a, 0x05, 0xcd, 0xd4, 0x3a, 0xc6, 0x5a, 0xc6, 0x49, 0x72, 0xe8, 0x64, 0xee, 0x60, 0x9e, + 0xa7, 0x2b, 0x4a, 0xfc, 0x19, 0x17, 0x8a, 0x41, 0xdf, 0x19, 0x36, 0xf3, 0x8a, 0x19, 0x46, 0x3f, + 0x05, 0xc8, 0x4e, 0xb7, 0xeb, 0x58, 0x31, 0xbf, 0xd0, 0xb3, 0x80, 0x53, 0x06, 0x8d, 0x85, 0x88, + 0x15, 0xbe, 0x53, 0xac, 0x6d, 0x1e, 0x36, 0x0f, 0x07, 0x3f, 0x42, 0xeb, 0x96, 0xcb, 0x20, 0x5b, + 0x9f, 0xdc, 0x41, 0xe7, 0xc8, 0x41, 0x06, 0xde, 0x5b, 0xa1, 0xb0, 0xbc, 0xef, 0x1a, 0x29, 0x0c, + 0x5c, 0x3d, 0x1e, 0x78, 0xf0, 0x0d, 0xb4, 0x76, 0xeb, 0x4a, 0xbb, 0x50, 0x8b, 0x45, 0x80, 0x29, + 0x73, 0xfa, 0xd5, 0xa1, 0x37, 0xcb, 0x02, 0x7a, 0x0e, 0xcd, 0x0d, 0x72, 0x19, 0xa3, 0x4c, 0x99, + 0x6b, 0x88, 0x5d, 0x3c, 0xf8, 0xc3, 0x01, 0xd0, 0xf7, 0x47, 0x2b, 0x1e, 0x87, 0x66, 0x23, 0x26, + 0xe3, 0x92, 0x3a, 0x77, 0x32, 0xa6, 0x5f, 0xd8, 0x0f, 0xd7, 0x35, 0x6b, 0xf5, 0x71, 0xf1, 0x33, + 0xc9, 0xee, 0x1d, 0x7d, 0xbd, 0xcf, 0xa1, 0x7e, 0x27, 0x02, 0x9c, 0x8c, 0xcb, 0x9a, 0x33, 0x4c, + 0x9b, 0x35, 0xb2, 0x66, 0x65, 0x1f, 0x6a, 0x1e, 0x5e, 0x7c, 0x09, 0xad, 0xdd, 0xef, 0x80, 0x9e, + 0x82, 0x6f, 0x82, 0x3b, 0x21, 0x23, 0xbe, 0x21, 0x15, 0xfa, 0x0c, 0x4e, 0x0d, 0xb0, 0x6f, 0x4c, + 0x9c, 0x8b, 0xbf, 0x5d, 0xf0, 0x0b, 0x0b, 0x4e, 0x01, 0xea, 0xd3, 0x34, 0xbc, 0xdd, 0x26, 0xa4, + 0x42, 0x7d, 0x68, 0x4c, 0xd3, 0xf0, 0x06, 0xb9, 0x22, 0x8e, 0x0d, 0x5e, 0x49, 0x91, 0x10, 0xd7, + 0x66, 0x5d, 0x27, 0x09, 0xa9, 0xd2, 0x0e, 0x40, 0x76, 0x9e, 0x61, 0x9a, 0x10, 0xcf, 0x26, 0x7e, + 0x2f, 0x14, 0x92, 0x9a, 0x16, 0x61, 0x03, 0xc3, 0xd6, 0x2d, 0xab, 0x97, 0x89, 0x34, 0x28, 0x81, + 0xb6, 0x6e, 0x86, 0x5c, 0xaa, 0x7b, 0xdd, 0xa5, 0x49, 0xbb, 0x40, 0x8a, 0x88, 0xb9, 0xd4, 0xa2, + 0x14, 0x3a, 0xd3, 0x34, 0x7c, 0x13, 0x4b, 0xe4, 0x8b, 0x15, 0xbf, 0xdf, 0x20, 0x01, 0x7a, 0x06, + 0x27, 0xb6, 0x90, 0x7e, 0xbc, 0x6d, 0x4a, 0x7c, 0x9b, 0x36, 0x5a, 0xe1, 0xe2, 0x97, 0xef, 0xb6, + 0x42, 0x6e, 0x23, 0xd2, 0xa6, 0x1f, 0xc1, 0xd9, 0x34, 0x0d, 0xe7, 0x92, 0xc7, 0xe9, 0x12, 0xe5, + 0x4b, 0xe4, 0x01, 0x4a, 0x72, 0x62, 0x6f, 0xcf, 0xd7, 0x11, 0x8a, 0xad, 0xba, 0x13, 0xbf, 0x92, + 0x8e, 0x15, 0x33, 0x43, 0x1e, 0x98, 0x9f, 0x21, 0x39, 0xb5, 0x62, 0x76, 0x88, 0x11, 0x43, 0xec, + 0xbc, 0xaf, 0x24, 0x9a, 0x11, 0xcf, 0x6c, 0x57, 0x1b, 0x9b, 0x1c, 0x7a, 0xb1, 0x85, 0x4e, 0xf9, + 0x79, 0xb5, 0x8e, 0x3d, 0x72, 0x1d, 0x04, 0xfa, 0x2d, 0x49, 0x85, 0x32, 0xe8, 0xee, 0xe1, 0x19, + 0x46, 0xe2, 0x2d, 0x1a, 0xc6, 0x29, 0x33, 0x6f, 0x92, 0x80, 0xab, 0x8c, 0x71, 0xcb, 0xcc, 0x75, + 0x10, 0xbc, 0xcc, 0xb6, 0x91, 0x54, 0x6f, 0xd8, 0xc3, 0x63, 0xaf, 0xf2, 0xe1, 0xb1, 0x57, 0x79, + 0x78, 0xea, 0x39, 0x1f, 0x9e, 0x7a, 0xce, 0xbf, 0x4f, 0x3d, 0xe7, 0xcf, 0xff, 0x7a, 0x95, 0xff, + 0x03, 0x00, 0x00, 0xff, 0xff, 0x3f, 0x1a, 0xcf, 0x3b, 0x70, 0x06, 0x00, 0x00, } diff --git a/raft/raftpb/raft.proto b/raft/raftpb/raft.proto index da1afbabf20..3ee8c0cfc89 100644 --- a/raft/raftpb/raft.proto +++ b/raft/raftpb/raft.proto @@ -75,26 +75,16 @@ message HardState { optional uint64 commit = 3 [(gogoproto.nullable) = false]; } -enum SuffrageState { - Nonvoter = 0; - Voter = 1; -} - -message Server { - optional uint64 node = 1 [(gogoproto.nullable) = false]; - optional SuffrageState suffrage = 2 [(gogoproto.nullable) = false]; -} - message ConfState { - repeated uint64 nodes = 1; - repeated Server servers = 2; + repeated uint64 nodes = 1; // Voters + repeated uint64 learners = 2; // Nonvoters } enum ConfChangeType { - ConfChangeAddNode = 0; - ConfChangeRemoveNode = 1; - ConfChangeUpdateNode = 2; - ConfChangeAddNonvoter = 3; + ConfChangeAddNode = 0; + ConfChangeRemoveNode = 1; + ConfChangeUpdateNode = 2; + ConfChangeAddLearner = 3; } message ConfChange { diff --git a/raft/rawnode.go b/raft/rawnode.go index a89ae4c80a9..b0208e64651 100644 --- a/raft/rawnode.go +++ b/raft/rawnode.go @@ -175,8 +175,8 @@ func (rn *RawNode) ApplyConfChange(cc pb.ConfChange) *pb.ConfState { switch cc.Type { case pb.ConfChangeAddNode: rn.raft.addNode(cc.NodeID) - case pb.ConfChangeAddNonvoter: - rn.raft.addNonvoter(cc.NodeID) + case pb.ConfChangeAddLearner: + rn.raft.addLearner(cc.NodeID) case pb.ConfChangeRemoveNode: rn.raft.removeNode(cc.NodeID) case pb.ConfChangeUpdateNode: From ee003a5137f5bc24ac0005af2a82a4508a15c106 Mon Sep 17 00:00:00 2001 From: gladiator Date: Sun, 1 Oct 2017 13:43:28 +0800 Subject: [PATCH 4/7] raft: add TestLearnerElectionTimeout --- raft/raft_test.go | 55 +++++++++++++++++++++++------------------------ 1 file changed, 27 insertions(+), 28 deletions(-) diff --git a/raft/raft_test.go b/raft/raft_test.go index cf6c35a36ff..deb3a2f8be7 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -348,27 +348,21 @@ func testLeaderElection(t *testing.T, preVote bool) { } } -func TestNonvoterElectionTimeout(t *testing.T) { - n1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - n1.prs[3].isLearner = true - - n2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - n2.prs[3].isLearner = true - - n3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage()) - n3.isLearner = true - n3.prs[3].isLearner = true +func TestLearnerElectionTimeout(t *testing.T) { + n1 := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) + n2 := newTestRaft(2, []uint64{1, 2}, 10, 1, NewMemoryStorage()) + n2.isLearner = true + n2.prs[2].isLearner = true n1.becomeFollower(1, None) n2.becomeFollower(1, None) - n3.becomeFollower(1, None) - nt := newNetwork(n1, n2, n3) + nt := newNetwork(n1, n2) - // Nonvoter can't start election - setRandomizedElectionTimeout(n3, n3.electionTimeout) - for i := 0; i < n3.electionTimeout; i++ { - n3.tick() + // Learner can't start election + setRandomizedElectionTimeout(n2, n2.electionTimeout) + for i := 0; i < n2.electionTimeout; i++ { + n2.tick() } if n1.state != StateFollower { t.Errorf("peer 1 state: %s, want %s", n1.state, StateFollower) @@ -376,23 +370,28 @@ func TestNonvoterElectionTimeout(t *testing.T) { if n2.state != StateFollower { t.Errorf("peer 2 state: %s, want %s", n2.state, StateFollower) } - if n3.state != StateFollower { - t.Errorf("peer 3 state: %s, want %s", n3.state, StateFollower) - } + // n1 should become leader nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) if n1.state != StateLeader { t.Errorf("peer 1 state: %s, want %s", n1.state, StateLeader) } + if n2.state != StateFollower { + t.Errorf("peer 2 state: %s, want %s", n2.state, StateFollower) + } - // node 3 become Voter - n1.addNode(3) - n2.addNode(3) - n3.addNode(3) + n2.addNode(2) + if n2.isLearner { + t.Errorf("peer 2 isLearner: %t, want %t", n2.isLearner, false) + } - nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup}) - if n3.state != StateLeader { - t.Errorf("peer 3 state: %s, want %s", n3.state, StateLeader) + // n2 start election, should become leader + nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup}) + if n1.state != StateFollower { + t.Errorf("peer 1 state: %s, want %s", n1.state, StateFollower) + } + if n2.state != StateLeader { + t.Errorf("peer 2 state: %s, want %s", n2.state, StateLeader) } } @@ -2374,7 +2373,7 @@ func TestRestore(t *testing.T) { } } -func TestRestoreWithNonvoter(t *testing.T) { +func TestRestoreWithLearner(t *testing.T) { s := pb.Snapshot{ Metadata: pb.SnapshotMetadata{ Index: 11, // magic number @@ -2658,7 +2657,7 @@ func TestAddNode(t *testing.T) { } } -func TestAddNonvoter(t *testing.T) { +func TestAddLearner(t *testing.T) { r := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage()) r.pendingConf = true r.addLearner(2) From aff1c5066ce9ec37e3e22278a3c60ba8c6bc397c Mon Sep 17 00:00:00 2001 From: gladiator Date: Thu, 12 Oct 2017 11:56:34 +0800 Subject: [PATCH 5/7] raft: ignore vote request for learner --- raft/raft.go | 6 ++++++ raft/raft_test.go | 5 +++++ 2 files changed, 11 insertions(+) diff --git a/raft/raft.go b/raft/raft.go index c015f70d342..0d9a4574445 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -819,6 +819,12 @@ func (r *raft) Step(m pb.Message) error { } case pb.MsgVote, pb.MsgPreVote: + if r.isLearner { + // TODO: learner may need to vote, in case of node down when confchange. + r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored %s from %x [logterm: %d, index: %d] at term %d: learner can not vote", + r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term) + return nil + } // The m.Term > r.Term clause is for MsgPreVote. For MsgVote m.Term should // always equal r.Term. if (r.Vote == None || m.Term > r.Term || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) { diff --git a/raft/raft_test.go b/raft/raft_test.go index deb3a2f8be7..c3fda581469 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -350,6 +350,8 @@ func testLeaderElection(t *testing.T, preVote bool) { func TestLearnerElectionTimeout(t *testing.T) { n1 := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage()) + n1.prs[2].isLearner = true + n2 := newTestRaft(2, []uint64{1, 2}, 10, 1, NewMemoryStorage()) n2.isLearner = true n2.prs[2].isLearner = true @@ -380,6 +382,9 @@ func TestLearnerElectionTimeout(t *testing.T) { t.Errorf("peer 2 state: %s, want %s", n2.state, StateFollower) } + nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat}) + + n1.addNode(2) n2.addNode(2) if n2.isLearner { t.Errorf("peer 2 isLearner: %t, want %t", n2.isLearner, false) From abb497a35c8535b5b84ae0b4c9a46ec0e66b0387 Mon Sep 17 00:00:00 2001 From: gladiator Date: Wed, 18 Oct 2017 12:05:13 +0800 Subject: [PATCH 6/7] raft: ConfChangeAddLearnerNode --- raft/node.go | 2 +- raft/raft.go | 35 +++++++----- raft/raftpb/raft.pb.go | 122 ++++++++++++++++++++--------------------- raft/raftpb/raft.proto | 8 +-- raft/rawnode.go | 2 +- 5 files changed, 87 insertions(+), 82 deletions(-) diff --git a/raft/node.go b/raft/node.go index 2895dbe588b..0fe29a0eea4 100644 --- a/raft/node.go +++ b/raft/node.go @@ -334,7 +334,7 @@ func (n *node) run(r *raft) { switch cc.Type { case pb.ConfChangeAddNode: r.addNode(cc.NodeID) - case pb.ConfChangeAddLearner: + case pb.ConfChangeAddLearnerNode: r.addLearner(cc.NodeID) case pb.ConfChangeRemoveNode: // block incoming proposal when local node is diff --git a/raft/raft.go b/raft/raft.go index 0d9a4574445..7e3ffc10936 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -116,6 +116,7 @@ type Config struct { // used for testing right now. peers []uint64 + // learners can not promote or vote. learners []uint64 // ElectionTick is the number of Node.Tick invocations that must pass between @@ -292,16 +293,16 @@ func newRaft(c *Config) *raft { if err != nil { panic(err) // TODO(bdarnell) } - voters := c.peers + peers := c.peers learners := c.learners if len(cs.Nodes) > 0 || len(cs.Learners) > 0 { - if len(voters) > 0 || len(learners) > 0 { + if len(peers) > 0 || len(learners) > 0 { // TODO(bdarnell): the peers argument is always nil except in // tests; the argument should be removed and these tests should be // updated to specify their nodes through a snapshot. panic("cannot specify both newRaft(peers, learners) and ConfState.(Nodes, Learners)") } - voters = cs.Nodes + peers = cs.Nodes learners = cs.Learners } r := &raft{ @@ -320,15 +321,15 @@ func newRaft(c *Config) *raft { readOnly: newReadOnly(c.ReadOnlyOption), disableProposalForwarding: c.DisableProposalForwarding, } - for _, n := range voters { - r.prs[n] = &Progress{Next: 1, ins: newInflights(r.maxInflight), isLearner: false} + for _, p := range peers { + r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)} } - for _, n := range learners { - if _, has := r.prs[n]; has { - panic(fmt.Sprintf("cannot specify both Voter and Learner for node: %x", n)) + for _, p := range learners { + if _, has := r.prs[p]; has { + panic(fmt.Sprintf("cannot specify both Voter and Learner for node: %x", p)) } - r.prs[n] = &Progress{Next: 1, ins: newInflights(r.maxInflight), isLearner: true} - if r.id == n { + r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight), isLearner: true} + if r.id == p { r.isLearner = true } } @@ -1231,21 +1232,24 @@ func (r *raft) restoreNode(nodes []uint64, isLearner bool) { } // promotable indicates whether state machine can be promoted to leader, -// which is true when its own id is in progress list and its not learner. +// which is true when its own id is in progress list and it is not a learner. func (r *raft) promotable() bool { + if r.isLearner { + return false + } _, ok := r.prs[r.id] - return ok && !r.isLearner + return ok } func (r *raft) addNode(id uint64) { - r.addLearnerNode(id, false) + r.addNodeOrLearnerNode(id, false) } func (r *raft) addLearner(id uint64) { - r.addLearnerNode(id, true) + r.addNodeOrLearnerNode(id, true) } -func (r *raft) addLearnerNode(id uint64, isLearner bool) { +func (r *raft) addNodeOrLearnerNode(id uint64, isLearner bool) { r.pendingConf = false if _, ok := r.prs[id]; ok { if r.prs[id].isLearner == isLearner { @@ -1255,6 +1259,7 @@ func (r *raft) addLearnerNode(id uint64, isLearner bool) { } if isLearner { // can only change Learner to Voter + r.logger.Infof("%x ignore addLearner for %x [%s]", r.id, id, r.prs[id]) return } r.prs[id].isLearner = isLearner diff --git a/raft/raftpb/raft.pb.go b/raft/raftpb/raft.pb.go index 9830ca65426..6e896c3143e 100644 --- a/raft/raftpb/raft.pb.go +++ b/raft/raftpb/raft.pb.go @@ -162,23 +162,23 @@ func (MessageType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, type ConfChangeType int32 const ( - ConfChangeAddNode ConfChangeType = 0 - ConfChangeRemoveNode ConfChangeType = 1 - ConfChangeUpdateNode ConfChangeType = 2 - ConfChangeAddLearner ConfChangeType = 3 + ConfChangeAddNode ConfChangeType = 0 + ConfChangeRemoveNode ConfChangeType = 1 + ConfChangeUpdateNode ConfChangeType = 2 + ConfChangeAddLearnerNode ConfChangeType = 3 ) var ConfChangeType_name = map[int32]string{ 0: "ConfChangeAddNode", 1: "ConfChangeRemoveNode", 2: "ConfChangeUpdateNode", - 3: "ConfChangeAddLearner", + 3: "ConfChangeAddLearnerNode", } var ConfChangeType_value = map[string]int32{ - "ConfChangeAddNode": 0, - "ConfChangeRemoveNode": 1, - "ConfChangeUpdateNode": 2, - "ConfChangeAddLearner": 3, + "ConfChangeAddNode": 0, + "ConfChangeRemoveNode": 1, + "ConfChangeUpdateNode": 2, + "ConfChangeAddLearnerNode": 3, } func (x ConfChangeType) Enum() *ConfChangeType { @@ -1966,56 +1966,56 @@ var ( func init() { proto.RegisterFile("raft.proto", fileDescriptorRaft) } var fileDescriptorRaft = []byte{ - // 813 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x54, 0xcd, 0x8e, 0xe3, 0x44, - 0x10, 0x8e, 0x1d, 0xe7, 0xaf, 0x9c, 0xc9, 0xf4, 0xf4, 0x06, 0xd4, 0x1a, 0xad, 0x42, 0x14, 0x71, - 0x88, 0x06, 0xed, 0x00, 0x73, 0xe0, 0x80, 0xc4, 0x61, 0x26, 0x41, 0x9a, 0x48, 0x9b, 0xd1, 0x92, - 0xcd, 0x72, 0x40, 0x42, 0xa8, 0x27, 0xae, 0x38, 0x81, 0xd8, 0x6d, 0xb5, 0x3b, 0xcb, 0xee, 0x05, - 0xf1, 0x00, 0x3c, 0x00, 0x17, 0xde, 0x67, 0x8e, 0x2b, 0x71, 0x47, 0xcc, 0xf0, 0x22, 0xa8, 0xdb, - 0xed, 0xc4, 0x4e, 0x6e, 0x5d, 0xdf, 0x57, 0x5d, 0xf5, 0xd5, 0xd7, 0x65, 0x03, 0x48, 0xbe, 0x54, - 0x97, 0x89, 0x14, 0x4a, 0xd0, 0xba, 0x3e, 0x27, 0xf7, 0xe7, 0xdd, 0x50, 0x84, 0xc2, 0x40, 0x9f, - 0xeb, 0x53, 0xc6, 0x0e, 0x7e, 0x83, 0xda, 0xb7, 0xb1, 0x92, 0xef, 0xe9, 0x67, 0xe0, 0xcd, 0xdf, - 0x27, 0xc8, 0x9c, 0xbe, 0x33, 0xec, 0x5c, 0x9d, 0x5d, 0x66, 0xb7, 0x2e, 0x0d, 0xa9, 0x89, 0x1b, - 0xef, 0xe1, 0x9f, 0x4f, 0x2a, 0x33, 0x93, 0x44, 0x19, 0x78, 0x73, 0x94, 0x11, 0x73, 0xfb, 0xce, - 0xd0, 0xdb, 0x31, 0x28, 0x23, 0x7a, 0x0e, 0xb5, 0x49, 0x1c, 0xe0, 0x3b, 0x56, 0x2d, 0x50, 0x19, - 0x44, 0x29, 0x78, 0x63, 0xae, 0x38, 0xf3, 0xfa, 0xce, 0xb0, 0x3d, 0x33, 0xe7, 0xc1, 0xef, 0x0e, - 0x90, 0xd7, 0x31, 0x4f, 0xd2, 0x95, 0x50, 0x53, 0x54, 0x3c, 0xe0, 0x8a, 0xd3, 0xaf, 0x00, 0x16, - 0x22, 0x5e, 0xfe, 0x94, 0x2a, 0xae, 0x32, 0x45, 0xfe, 0x5e, 0xd1, 0x48, 0xc4, 0xcb, 0xd7, 0x9a, - 0xb0, 0xc5, 0x5b, 0x8b, 0x1c, 0xd0, 0xcd, 0xd7, 0xa6, 0x79, 0x51, 0x57, 0x06, 0x69, 0xc9, 0x4a, - 0x4b, 0x2e, 0xea, 0x32, 0xc8, 0xe0, 0x07, 0x68, 0xe6, 0x0a, 0xb4, 0x44, 0xad, 0xc0, 0xf4, 0x6c, - 0xcf, 0xcc, 0x99, 0x7e, 0x0d, 0xcd, 0xc8, 0x2a, 0x33, 0x85, 0xfd, 0x2b, 0x96, 0x6b, 0x39, 0x54, - 0x6e, 0xeb, 0xee, 0xf2, 0x07, 0x7f, 0x55, 0xa1, 0x31, 0xc5, 0x34, 0xe5, 0x21, 0xd2, 0x17, 0xe0, - 0xa9, 0xbd, 0xc3, 0xcf, 0xf2, 0x1a, 0x96, 0x2e, 0x7a, 0xac, 0xd3, 0x68, 0x17, 0x5c, 0x25, 0x4a, - 0x93, 0xb8, 0x4a, 0xe8, 0x31, 0x96, 0x52, 0x1c, 0x8c, 0xa1, 0x91, 0xdd, 0x80, 0xde, 0xe1, 0x80, - 0xb4, 0x07, 0x8d, 0x8d, 0x08, 0xcd, 0x83, 0xd5, 0x0a, 0x64, 0x0e, 0xee, 0x6d, 0xab, 0x1f, 0xdb, - 0xf6, 0x02, 0x1a, 0x18, 0x2b, 0xb9, 0xc6, 0x94, 0x35, 0xfa, 0xd5, 0xa1, 0x7f, 0x75, 0x52, 0xda, - 0x8c, 0xbc, 0x94, 0xcd, 0xa1, 0xcf, 0xa1, 0xbe, 0x10, 0x51, 0xb4, 0x56, 0xac, 0x59, 0xa8, 0x65, - 0x31, 0x7a, 0x05, 0xcd, 0xd4, 0x3a, 0xc6, 0x5a, 0xc6, 0x49, 0x72, 0xe8, 0x64, 0xee, 0x60, 0x9e, - 0xa7, 0x2b, 0x4a, 0xfc, 0x19, 0x17, 0x8a, 0x41, 0xdf, 0x19, 0x36, 0xf3, 0x8a, 0x19, 0x46, 0x3f, - 0x05, 0xc8, 0x4e, 0xb7, 0xeb, 0x58, 0x31, 0xbf, 0xd0, 0xb3, 0x80, 0x53, 0x06, 0x8d, 0x85, 0x88, - 0x15, 0xbe, 0x53, 0xac, 0x6d, 0x1e, 0x36, 0x0f, 0x07, 0x3f, 0x42, 0xeb, 0x96, 0xcb, 0x20, 0x5b, - 0x9f, 0xdc, 0x41, 0xe7, 0xc8, 0x41, 0x06, 0xde, 0x5b, 0xa1, 0xb0, 0xbc, 0xef, 0x1a, 0x29, 0x0c, - 0x5c, 0x3d, 0x1e, 0x78, 0xf0, 0x0d, 0xb4, 0x76, 0xeb, 0x4a, 0xbb, 0x50, 0x8b, 0x45, 0x80, 0x29, - 0x73, 0xfa, 0xd5, 0xa1, 0x37, 0xcb, 0x02, 0x7a, 0x0e, 0xcd, 0x0d, 0x72, 0x19, 0xa3, 0x4c, 0x99, - 0x6b, 0x88, 0x5d, 0x3c, 0xf8, 0xc3, 0x01, 0xd0, 0xf7, 0x47, 0x2b, 0x1e, 0x87, 0x66, 0x23, 0x26, - 0xe3, 0x92, 0x3a, 0x77, 0x32, 0xa6, 0x5f, 0xd8, 0x0f, 0xd7, 0x35, 0x6b, 0xf5, 0x71, 0xf1, 0x33, - 0xc9, 0xee, 0x1d, 0x7d, 0xbd, 0xcf, 0xa1, 0x7e, 0x27, 0x02, 0x9c, 0x8c, 0xcb, 0x9a, 0x33, 0x4c, - 0x9b, 0x35, 0xb2, 0x66, 0x65, 0x1f, 0x6a, 0x1e, 0x5e, 0x7c, 0x09, 0xad, 0xdd, 0xef, 0x80, 0x9e, - 0x82, 0x6f, 0x82, 0x3b, 0x21, 0x23, 0xbe, 0x21, 0x15, 0xfa, 0x0c, 0x4e, 0x0d, 0xb0, 0x6f, 0x4c, - 0x9c, 0x8b, 0xbf, 0x5d, 0xf0, 0x0b, 0x0b, 0x4e, 0x01, 0xea, 0xd3, 0x34, 0xbc, 0xdd, 0x26, 0xa4, - 0x42, 0x7d, 0x68, 0x4c, 0xd3, 0xf0, 0x06, 0xb9, 0x22, 0x8e, 0x0d, 0x5e, 0x49, 0x91, 0x10, 0xd7, - 0x66, 0x5d, 0x27, 0x09, 0xa9, 0xd2, 0x0e, 0x40, 0x76, 0x9e, 0x61, 0x9a, 0x10, 0xcf, 0x26, 0x7e, - 0x2f, 0x14, 0x92, 0x9a, 0x16, 0x61, 0x03, 0xc3, 0xd6, 0x2d, 0xab, 0x97, 0x89, 0x34, 0x28, 0x81, - 0xb6, 0x6e, 0x86, 0x5c, 0xaa, 0x7b, 0xdd, 0xa5, 0x49, 0xbb, 0x40, 0x8a, 0x88, 0xb9, 0xd4, 0xa2, - 0x14, 0x3a, 0xd3, 0x34, 0x7c, 0x13, 0x4b, 0xe4, 0x8b, 0x15, 0xbf, 0xdf, 0x20, 0x01, 0x7a, 0x06, - 0x27, 0xb6, 0x90, 0x7e, 0xbc, 0x6d, 0x4a, 0x7c, 0x9b, 0x36, 0x5a, 0xe1, 0xe2, 0x97, 0xef, 0xb6, - 0x42, 0x6e, 0x23, 0xd2, 0xa6, 0x1f, 0xc1, 0xd9, 0x34, 0x0d, 0xe7, 0x92, 0xc7, 0xe9, 0x12, 0xe5, - 0x4b, 0xe4, 0x01, 0x4a, 0x72, 0x62, 0x6f, 0xcf, 0xd7, 0x11, 0x8a, 0xad, 0xba, 0x13, 0xbf, 0x92, - 0x8e, 0x15, 0x33, 0x43, 0x1e, 0x98, 0x9f, 0x21, 0x39, 0xb5, 0x62, 0x76, 0x88, 0x11, 0x43, 0xec, - 0xbc, 0xaf, 0x24, 0x9a, 0x11, 0xcf, 0x6c, 0x57, 0x1b, 0x9b, 0x1c, 0x7a, 0xb1, 0x85, 0x4e, 0xf9, - 0x79, 0xb5, 0x8e, 0x3d, 0x72, 0x1d, 0x04, 0xfa, 0x2d, 0x49, 0x85, 0x32, 0xe8, 0xee, 0xe1, 0x19, - 0x46, 0xe2, 0x2d, 0x1a, 0xc6, 0x29, 0x33, 0x6f, 0x92, 0x80, 0xab, 0x8c, 0x71, 0xcb, 0xcc, 0x75, - 0x10, 0xbc, 0xcc, 0xb6, 0x91, 0x54, 0x6f, 0xd8, 0xc3, 0x63, 0xaf, 0xf2, 0xe1, 0xb1, 0x57, 0x79, - 0x78, 0xea, 0x39, 0x1f, 0x9e, 0x7a, 0xce, 0xbf, 0x4f, 0x3d, 0xe7, 0xcf, 0xff, 0x7a, 0x95, 0xff, - 0x03, 0x00, 0x00, 0xff, 0xff, 0x3f, 0x1a, 0xcf, 0x3b, 0x70, 0x06, 0x00, 0x00, + // 815 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x54, 0xcd, 0x6e, 0x23, 0x45, + 0x10, 0xf6, 0x8c, 0xc7, 0x7f, 0x35, 0x8e, 0xd3, 0xa9, 0x35, 0xa8, 0x15, 0x45, 0xc6, 0xb2, 0x38, + 0x58, 0x41, 0x1b, 0x20, 0x07, 0x0e, 0x48, 0x1c, 0x36, 0x09, 0x52, 0x22, 0xad, 0xa3, 0xc5, 0x9b, + 0xe5, 0x80, 0x84, 0x50, 0xc7, 0x53, 0x9e, 0x18, 0x32, 0xd3, 0xa3, 0x9e, 0xf6, 0xb2, 0xb9, 0x20, + 0x1e, 0x80, 0x07, 0xe0, 0xc2, 0xfb, 0xe4, 0xb8, 0x12, 0x77, 0xc4, 0x86, 0x17, 0x41, 0xdd, 0xd3, + 0x63, 0xcf, 0x24, 0xb7, 0xae, 0xef, 0xab, 0xae, 0xfa, 0xea, 0xeb, 0x9a, 0x01, 0x50, 0x62, 0xa9, + 0x8f, 0x32, 0x25, 0xb5, 0xc4, 0xb6, 0x39, 0x67, 0xd7, 0xfb, 0xc3, 0x58, 0xc6, 0xd2, 0x42, 0x9f, + 0x9b, 0x53, 0xc1, 0x4e, 0x7e, 0x83, 0xd6, 0xb7, 0xa9, 0x56, 0x77, 0xf8, 0x19, 0x04, 0x57, 0x77, + 0x19, 0x71, 0x6f, 0xec, 0x4d, 0x07, 0xc7, 0x7b, 0x47, 0xc5, 0xad, 0x23, 0x4b, 0x1a, 0xe2, 0x24, + 0xb8, 0xff, 0xe7, 0x93, 0xc6, 0xdc, 0x26, 0x21, 0x87, 0xe0, 0x8a, 0x54, 0xc2, 0xfd, 0xb1, 0x37, + 0x0d, 0x36, 0x0c, 0xa9, 0x04, 0xf7, 0xa1, 0x75, 0x91, 0x46, 0xf4, 0x8e, 0x37, 0x2b, 0x54, 0x01, + 0x21, 0x42, 0x70, 0x26, 0xb4, 0xe0, 0xc1, 0xd8, 0x9b, 0xf6, 0xe7, 0xf6, 0x3c, 0xf9, 0xdd, 0x03, + 0xf6, 0x3a, 0x15, 0x59, 0x7e, 0x23, 0xf5, 0x8c, 0xb4, 0x88, 0x84, 0x16, 0xf8, 0x15, 0xc0, 0x42, + 0xa6, 0xcb, 0x9f, 0x72, 0x2d, 0x74, 0xa1, 0x28, 0xdc, 0x2a, 0x3a, 0x95, 0xe9, 0xf2, 0xb5, 0x21, + 0x5c, 0xf1, 0xde, 0xa2, 0x04, 0x4c, 0xf3, 0x95, 0x6d, 0x5e, 0xd5, 0x55, 0x40, 0x46, 0xb2, 0x36, + 0x92, 0xab, 0xba, 0x2c, 0x32, 0xf9, 0x01, 0xba, 0xa5, 0x02, 0x23, 0xd1, 0x28, 0xb0, 0x3d, 0xfb, + 0x73, 0x7b, 0xc6, 0xaf, 0xa1, 0x9b, 0x38, 0x65, 0xb6, 0x70, 0x78, 0xcc, 0x4b, 0x2d, 0x8f, 0x95, + 0xbb, 0xba, 0x9b, 0xfc, 0xc9, 0x5f, 0x4d, 0xe8, 0xcc, 0x28, 0xcf, 0x45, 0x4c, 0xf8, 0x1c, 0x02, + 0xbd, 0x75, 0xf8, 0x59, 0x59, 0xc3, 0xd1, 0x55, 0x8f, 0x4d, 0x1a, 0x0e, 0xc1, 0xd7, 0xb2, 0x36, + 0x89, 0xaf, 0xa5, 0x19, 0x63, 0xa9, 0xe4, 0xa3, 0x31, 0x0c, 0xb2, 0x19, 0x30, 0x78, 0x3c, 0x20, + 0x8e, 0xa0, 0x73, 0x2b, 0x63, 0xfb, 0x60, 0xad, 0x0a, 0x59, 0x82, 0x5b, 0xdb, 0xda, 0x4f, 0x6d, + 0x7b, 0x0e, 0x1d, 0x4a, 0xb5, 0x5a, 0x51, 0xce, 0x3b, 0xe3, 0xe6, 0x34, 0x3c, 0xde, 0xa9, 0x6d, + 0x46, 0x59, 0xca, 0xe5, 0xe0, 0x01, 0xb4, 0x17, 0x32, 0x49, 0x56, 0x9a, 0x77, 0x2b, 0xb5, 0x1c, + 0x86, 0xc7, 0xd0, 0xcd, 0x9d, 0x63, 0xbc, 0x67, 0x9d, 0x64, 0x8f, 0x9d, 0x2c, 0x1d, 0x2c, 0xf3, + 0x4c, 0x45, 0x45, 0x3f, 0xd3, 0x42, 0x73, 0x18, 0x7b, 0xd3, 0x6e, 0x59, 0xb1, 0xc0, 0xf0, 0x53, + 0x80, 0xe2, 0x74, 0xbe, 0x4a, 0x35, 0x0f, 0x2b, 0x3d, 0x2b, 0x38, 0x72, 0xe8, 0x2c, 0x64, 0xaa, + 0xe9, 0x9d, 0xe6, 0x7d, 0xfb, 0xb0, 0x65, 0x38, 0xf9, 0x11, 0x7a, 0xe7, 0x42, 0x45, 0xc5, 0xfa, + 0x94, 0x0e, 0x7a, 0x4f, 0x1c, 0xe4, 0x10, 0xbc, 0x95, 0x9a, 0xea, 0xfb, 0x6e, 0x90, 0xca, 0xc0, + 0xcd, 0xa7, 0x03, 0x4f, 0xbe, 0x81, 0xde, 0x66, 0x5d, 0x71, 0x08, 0xad, 0x54, 0x46, 0x94, 0x73, + 0x6f, 0xdc, 0x9c, 0x06, 0xf3, 0x22, 0xc0, 0x7d, 0xe8, 0xde, 0x92, 0x50, 0x29, 0xa9, 0x9c, 0xfb, + 0x96, 0xd8, 0xc4, 0x93, 0x3f, 0x3c, 0x00, 0x73, 0xff, 0xf4, 0x46, 0xa4, 0xb1, 0xdd, 0x88, 0x8b, + 0xb3, 0x9a, 0x3a, 0xff, 0xe2, 0x0c, 0xbf, 0x70, 0x1f, 0xae, 0x6f, 0xd7, 0xea, 0xe3, 0xea, 0x67, + 0x52, 0xdc, 0x7b, 0xf2, 0xf5, 0x1e, 0x40, 0xfb, 0x52, 0x46, 0x74, 0x71, 0x56, 0xd7, 0x5c, 0x60, + 0xc6, 0xac, 0x53, 0x67, 0x56, 0xf1, 0xa1, 0x96, 0xe1, 0xe1, 0x97, 0xd0, 0xdb, 0xfc, 0x0e, 0x70, + 0x17, 0x42, 0x1b, 0x5c, 0x4a, 0x95, 0x88, 0x5b, 0xd6, 0xc0, 0x67, 0xb0, 0x6b, 0x81, 0x6d, 0x63, + 0xe6, 0x1d, 0xfe, 0xed, 0x43, 0x58, 0x59, 0x70, 0x04, 0x68, 0xcf, 0xf2, 0xf8, 0x7c, 0x9d, 0xb1, + 0x06, 0x86, 0xd0, 0x99, 0xe5, 0xf1, 0x09, 0x09, 0xcd, 0x3c, 0x17, 0xbc, 0x52, 0x32, 0x63, 0xbe, + 0xcb, 0x7a, 0x91, 0x65, 0xac, 0x89, 0x03, 0x80, 0xe2, 0x3c, 0xa7, 0x3c, 0x63, 0x81, 0x4b, 0xfc, + 0x5e, 0x6a, 0x62, 0x2d, 0x23, 0xc2, 0x05, 0x96, 0x6d, 0x3b, 0xd6, 0x2c, 0x13, 0xeb, 0x20, 0x83, + 0xbe, 0x69, 0x46, 0x42, 0xe9, 0x6b, 0xd3, 0xa5, 0x8b, 0x43, 0x60, 0x55, 0xc4, 0x5e, 0xea, 0x21, + 0xc2, 0x60, 0x96, 0xc7, 0x6f, 0x52, 0x45, 0x62, 0x71, 0x23, 0xae, 0x6f, 0x89, 0x01, 0xee, 0xc1, + 0x8e, 0x2b, 0x64, 0x1e, 0x6f, 0x9d, 0xb3, 0xd0, 0xa5, 0x9d, 0xde, 0xd0, 0xe2, 0x97, 0xef, 0xd6, + 0x52, 0xad, 0x13, 0xd6, 0xc7, 0x8f, 0x60, 0x6f, 0x96, 0xc7, 0x57, 0x4a, 0xa4, 0xf9, 0x92, 0xd4, + 0x4b, 0x12, 0x11, 0x29, 0xb6, 0xe3, 0x6e, 0x5f, 0xad, 0x12, 0x92, 0x6b, 0x7d, 0x29, 0x7f, 0x65, + 0x03, 0x27, 0x66, 0x4e, 0x22, 0xb2, 0x3f, 0x43, 0xb6, 0xeb, 0xc4, 0x6c, 0x10, 0x2b, 0x86, 0xb9, + 0x79, 0x5f, 0x29, 0xb2, 0x23, 0xee, 0xb9, 0xae, 0x2e, 0xb6, 0x39, 0x78, 0x78, 0x07, 0x83, 0xfa, + 0xf3, 0x1a, 0x1d, 0x5b, 0xe4, 0x45, 0x14, 0x99, 0xb7, 0x64, 0x0d, 0xe4, 0x30, 0xdc, 0xc2, 0x73, + 0x4a, 0xe4, 0x5b, 0xb2, 0x8c, 0x57, 0x67, 0xde, 0x64, 0x91, 0xd0, 0x05, 0xe3, 0xe3, 0x01, 0xf0, + 0x5a, 0xa9, 0x97, 0xc5, 0x36, 0x5a, 0xb6, 0x79, 0xc2, 0xef, 0x3f, 0x8c, 0x1a, 0xef, 0x3f, 0x8c, + 0x1a, 0xf7, 0x0f, 0x23, 0xef, 0xfd, 0xc3, 0xc8, 0xfb, 0xf7, 0x61, 0xe4, 0xfd, 0xf9, 0xdf, 0xa8, + 0xf1, 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x86, 0x52, 0x5b, 0xe0, 0x74, 0x06, 0x00, 0x00, } diff --git a/raft/raftpb/raft.proto b/raft/raftpb/raft.proto index 3ee8c0cfc89..562e9cc84a8 100644 --- a/raft/raftpb/raft.proto +++ b/raft/raftpb/raft.proto @@ -81,10 +81,10 @@ message ConfState { } enum ConfChangeType { - ConfChangeAddNode = 0; - ConfChangeRemoveNode = 1; - ConfChangeUpdateNode = 2; - ConfChangeAddLearner = 3; + ConfChangeAddNode = 0; + ConfChangeRemoveNode = 1; + ConfChangeUpdateNode = 2; + ConfChangeAddLearnerNode = 3; } message ConfChange { diff --git a/raft/rawnode.go b/raft/rawnode.go index b0208e64651..0c33fee96e9 100644 --- a/raft/rawnode.go +++ b/raft/rawnode.go @@ -175,7 +175,7 @@ func (rn *RawNode) ApplyConfChange(cc pb.ConfChange) *pb.ConfState { switch cc.Type { case pb.ConfChangeAddNode: rn.raft.addNode(cc.NodeID) - case pb.ConfChangeAddLearner: + case pb.ConfChangeAddLearnerNode: rn.raft.addLearner(cc.NodeID) case pb.ConfChangeRemoveNode: rn.raft.removeNode(cc.NodeID) From 2d2c6b96d2495787de24dec8b66f1c98f312fde8 Mon Sep 17 00:00:00 2001 From: gladiator Date: Thu, 19 Oct 2017 16:04:33 +0800 Subject: [PATCH 7/7] raft: address comments --- raft/raft.go | 2 +- raft/raft_test.go | 4 ++-- raft/raftpb/raft.proto | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/raft/raft.go b/raft/raft.go index 7e3ffc10936..acd6dddaa12 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -1259,7 +1259,7 @@ func (r *raft) addNodeOrLearnerNode(id uint64, isLearner bool) { } if isLearner { // can only change Learner to Voter - r.logger.Infof("%x ignore addLearner for %x [%s]", r.id, id, r.prs[id]) + r.logger.Infof("%x ignore addLearner for existing node %x [%s]", r.id, id, r.prs[id]) return } r.prs[id].isLearner = isLearner diff --git a/raft/raft_test.go b/raft/raft_test.go index c3fda581469..0dc40366945 100644 --- a/raft/raft_test.go +++ b/raft/raft_test.go @@ -3388,7 +3388,7 @@ func entsWithConfig(configFunc func(*Config), terms ...uint64) *raft { for i, term := range terms { storage.Append([]pb.Entry{{Index: uint64(i + 1), Term: term}}) } - cfg := newTestConfig(1, []uint64{1, 2, 3, 4, 5}, 5, 1, storage) + cfg := newTestConfig(1, []uint64{}, 5, 1, storage) if configFunc != nil { configFunc(cfg) } @@ -3403,7 +3403,7 @@ func entsWithConfig(configFunc func(*Config), terms ...uint64) *raft { func votedWithConfig(configFunc func(*Config), vote, term uint64) *raft { storage := NewMemoryStorage() storage.SetHardState(pb.HardState{Vote: vote, Term: term}) - cfg := newTestConfig(1, []uint64{1, 2, 3, 4, 5}, 5, 1, storage) + cfg := newTestConfig(1, []uint64{}, 5, 1, storage) if configFunc != nil { configFunc(cfg) } diff --git a/raft/raftpb/raft.proto b/raft/raftpb/raft.proto index 562e9cc84a8..644ce7b8f2f 100644 --- a/raft/raftpb/raft.proto +++ b/raft/raftpb/raft.proto @@ -76,8 +76,8 @@ message HardState { } message ConfState { - repeated uint64 nodes = 1; // Voters - repeated uint64 learners = 2; // Nonvoters + repeated uint64 nodes = 1; + repeated uint64 learners = 2; } enum ConfChangeType {