Browse Source

raftpb: clean up naming in ConfChange

Tobias Schottdorf 6 years ago
parent
commit
b9c051e7a7

+ 2 - 3
clientv3/snapshot/v3_snapshot.go

@@ -28,6 +28,7 @@ import (
 	"strings"
 	"time"
 
+	bolt "go.etcd.io/bbolt"
 	"go.etcd.io/etcd/clientv3"
 	"go.etcd.io/etcd/etcdserver"
 	"go.etcd.io/etcd/etcdserver/api/membership"
@@ -43,8 +44,6 @@ import (
 	"go.etcd.io/etcd/raft/raftpb"
 	"go.etcd.io/etcd/wal"
 	"go.etcd.io/etcd/wal/walpb"
-
-	bolt "go.etcd.io/bbolt"
 	"go.uber.org/zap"
 )
 
@@ -482,7 +481,7 @@ func (s *v3Manager) saveWALAndSnap() error {
 			Index: commit,
 			Term:  term,
 			ConfState: raftpb.ConfState{
-				Nodes: nodeIDs,
+				Voters: nodeIDs,
 			},
 		},
 	}

+ 1 - 2
etcdserver/api/snap/snapshotter_test.go

@@ -24,7 +24,6 @@ import (
 	"testing"
 
 	"go.etcd.io/etcd/raft/raftpb"
-
 	"go.uber.org/zap"
 )
 
@@ -32,7 +31,7 @@ var testSnap = &raftpb.Snapshot{
 	Data: []byte("some snapshot"),
 	Metadata: raftpb.SnapshotMetadata{
 		ConfState: raftpb.ConfState{
-			Nodes: []uint64{1, 2, 3},
+			Voters: []uint64{1, 2, 3},
 		},
 		Index: 1,
 		Term:  1,

+ 1 - 1
etcdserver/raft.go

@@ -647,7 +647,7 @@ func restartAsStandaloneNode(cfg ServerConfig, snapshot *raftpb.Snapshot) (types
 func getIDs(lg *zap.Logger, snap *raftpb.Snapshot, ents []raftpb.Entry) []uint64 {
 	ids := make(map[uint64]bool)
 	if snap != nil {
-		for _, id := range snap.Metadata.ConfState.Nodes {
+		for _, id := range snap.Metadata.ConfState.Voters {
 			ids[id] = true
 		}
 	}

+ 6 - 6
etcdserver/raft_test.go

@@ -46,17 +46,17 @@ func TestGetIDs(t *testing.T) {
 		widSet []uint64
 	}{
 		{nil, []raftpb.Entry{}, []uint64{}},
-		{&raftpb.ConfState{Nodes: []uint64{1}},
+		{&raftpb.ConfState{Voters: []uint64{1}},
 			[]raftpb.Entry{}, []uint64{1}},
-		{&raftpb.ConfState{Nodes: []uint64{1}},
+		{&raftpb.ConfState{Voters: []uint64{1}},
 			[]raftpb.Entry{addEntry}, []uint64{1, 2}},
-		{&raftpb.ConfState{Nodes: []uint64{1}},
+		{&raftpb.ConfState{Voters: []uint64{1}},
 			[]raftpb.Entry{addEntry, removeEntry}, []uint64{1}},
-		{&raftpb.ConfState{Nodes: []uint64{1}},
+		{&raftpb.ConfState{Voters: []uint64{1}},
 			[]raftpb.Entry{addEntry, normalEntry}, []uint64{1, 2}},
-		{&raftpb.ConfState{Nodes: []uint64{1}},
+		{&raftpb.ConfState{Voters: []uint64{1}},
 			[]raftpb.Entry{addEntry, normalEntry, updateEntry}, []uint64{1, 2}},
-		{&raftpb.ConfState{Nodes: []uint64{1}},
+		{&raftpb.ConfState{Voters: []uint64{1}},
 			[]raftpb.Entry{addEntry, removeEntry, normalEntry}, []uint64{1}},
 	}
 

+ 1 - 1
etcdserver/server_test.go

@@ -1016,7 +1016,7 @@ func TestSnapshot(t *testing.T) {
 		}
 	}()
 
-	srv.snapshot(1, raftpb.ConfState{Nodes: []uint64{1}})
+	srv.snapshot(1, raftpb.ConfState{Voters: []uint64{1}})
 	<-ch
 	<-ch
 }

+ 1 - 1
raft/node.go

@@ -366,7 +366,7 @@ func (n *node) run(rn *RawNode) {
 			// very sound and likely has bugs.
 			if _, okAfter := r.prs.Progress[r.id]; okBefore && !okAfter {
 				var found bool
-				for _, sl := range [][]uint64{cs.Nodes, cs.NodesJoint} {
+				for _, sl := range [][]uint64{cs.Voters, cs.VotersOutgoing} {
 					for _, id := range sl {
 						if id == r.id {
 							found = true

+ 2 - 2
raft/node_test.go

@@ -691,7 +691,7 @@ func TestNodeRestart(t *testing.T) {
 func TestNodeRestartFromSnapshot(t *testing.T) {
 	snap := raftpb.Snapshot{
 		Metadata: raftpb.SnapshotMetadata{
-			ConfState: raftpb.ConfState{Nodes: []uint64{1, 2}},
+			ConfState: raftpb.ConfState{Voters: []uint64{1, 2}},
 			Index:     2,
 			Term:      1,
 		},
@@ -845,7 +845,7 @@ func TestNodeProposeAddLearnerNode(t *testing.T) {
 						t.Errorf("apply conf change should return new added learner: %v", state.String())
 					}
 
-					if len(state.Nodes) != 1 {
+					if len(state.Voters) != 1 {
 						t.Errorf("add learner should not change the nodes: %v", state.String())
 					}
 					t.Logf("apply raft conf %v changed to: %v", cc, state.String())

+ 11 - 11
raft/raft.go

@@ -329,14 +329,14 @@ func newRaft(c *Config) *raft {
 	}
 	peers := c.peers
 	learners := c.learners
-	if len(cs.Nodes) > 0 || len(cs.Learners) > 0 {
+	if len(cs.Voters) > 0 || len(cs.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)")
+			panic("cannot specify both newRaft(peers, learners) and ConfState.(Voters, Learners)")
 		}
-		peers = cs.Nodes
+		peers = cs.Voters
 		learners = cs.Learners
 	}
 	r := &raft{
@@ -1384,7 +1384,7 @@ func (r *raft) restore(s pb.Snapshot) bool {
 	found := false
 	cs := s.Metadata.ConfState
 	for _, set := range [][]uint64{
-		cs.Nodes,
+		cs.Voters,
 		cs.Learners,
 	} {
 		for _, id := range set {
@@ -1415,7 +1415,7 @@ func (r *raft) restore(s pb.Snapshot) bool {
 
 	// Reset the configuration and add the (potentially updated) peers in anew.
 	r.prs = tracker.MakeProgressTracker(r.prs.MaxInflight)
-	for _, id := range s.Metadata.ConfState.Nodes {
+	for _, id := range s.Metadata.ConfState.Voters {
 		r.applyConfChange(pb.ConfChange{NodeID: id, Type: pb.ConfChangeAddNode}.AsV2())
 	}
 	for _, id := range s.Metadata.ConfState.Learners {
@@ -1463,11 +1463,11 @@ func (r *raft) applyConfChange(cc pb.ConfChangeV2) pb.ConfState {
 	// Now that the configuration is updated, handle any side effects.
 
 	cs := pb.ConfState{
-		Nodes:        r.prs.Voters[0].Slice(),
-		NodesJoint:   r.prs.Voters[1].Slice(),
-		Learners:     quorum.MajorityConfig(r.prs.Learners).Slice(),
-		LearnersNext: quorum.MajorityConfig(r.prs.LearnersNext).Slice(),
-		AutoLeave:    r.prs.AutoLeave,
+		Voters:         r.prs.Voters[0].Slice(),
+		VotersOutgoing: r.prs.Voters[1].Slice(),
+		Learners:       quorum.MajorityConfig(r.prs.Learners).Slice(),
+		LearnersNext:   quorum.MajorityConfig(r.prs.LearnersNext).Slice(),
+		AutoLeave:      r.prs.AutoLeave,
 	}
 	pr, ok := r.prs.Progress[r.id]
 
@@ -1490,7 +1490,7 @@ func (r *raft) applyConfChange(cc pb.ConfChangeV2) pb.ConfState {
 
 	// The remaining steps only make sense if this node is the leader and there
 	// are other nodes.
-	if r.state != StateLeader || len(cs.Nodes) == 0 {
+	if r.state != StateLeader || len(cs.Voters) == 0 {
 		return cs
 	}
 	if r.maybeCommit() {

+ 2 - 2
raft/raft_snap_test.go

@@ -26,7 +26,7 @@ var (
 		Metadata: pb.SnapshotMetadata{
 			Index:     11, // magic number
 			Term:      11, // magic number
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2}},
 		},
 	}
 )
@@ -145,7 +145,7 @@ func TestSnapshotSucceedViaAppResp(t *testing.T) {
 	n1.applyConfChange(pb.ConfChange{NodeID: 2, Type: pb.ConfChangeAddNode}.AsV2())
 	s1.snapshot = pb.Snapshot{
 		Metadata: pb.SnapshotMetadata{
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2}},
 			Index:     s1.lastIndex(),
 			Term:      s1.ents[len(s1.ents)-1].Term,
 		},

+ 17 - 17
raft/raft_test.go

@@ -2458,7 +2458,7 @@ func TestBcastBeat(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     offset,
 			Term:      1,
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2, 3}},
 		},
 	}
 	storage := NewMemoryStorage()
@@ -2709,7 +2709,7 @@ func TestRestore(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     11, // magic number
 			Term:      11, // magic number
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2, 3}},
 		},
 	}
 
@@ -2726,8 +2726,8 @@ func TestRestore(t *testing.T) {
 		t.Errorf("log.lastTerm = %d, want %d", mustTerm(sm.raftLog.term(s.Metadata.Index)), s.Metadata.Term)
 	}
 	sg := sm.prs.VoterNodes()
-	if !reflect.DeepEqual(sg, s.Metadata.ConfState.Nodes) {
-		t.Errorf("sm.Nodes = %+v, want %+v", sg, s.Metadata.ConfState.Nodes)
+	if !reflect.DeepEqual(sg, s.Metadata.ConfState.Voters) {
+		t.Errorf("sm.Voters = %+v, want %+v", sg, s.Metadata.ConfState.Voters)
 	}
 
 	if ok := sm.restore(s); ok {
@@ -2741,7 +2741,7 @@ func TestRestoreWithLearner(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     11, // magic number
 			Term:      11, // magic number
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2}, Learners: []uint64{3}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2}, Learners: []uint64{3}},
 		},
 	}
 
@@ -2758,14 +2758,14 @@ func TestRestoreWithLearner(t *testing.T) {
 		t.Errorf("log.lastTerm = %d, want %d", mustTerm(sm.raftLog.term(s.Metadata.Index)), s.Metadata.Term)
 	}
 	sg := sm.prs.VoterNodes()
-	if len(sg) != len(s.Metadata.ConfState.Nodes) {
-		t.Errorf("sm.Nodes = %+v, length not equal with %+v", sg, s.Metadata.ConfState.Nodes)
+	if len(sg) != len(s.Metadata.ConfState.Voters) {
+		t.Errorf("sm.Voters = %+v, length not equal with %+v", sg, s.Metadata.ConfState.Voters)
 	}
 	lns := sm.prs.LearnerNodes()
 	if len(lns) != len(s.Metadata.ConfState.Learners) {
 		t.Errorf("sm.LearnerNodes = %+v, length not equal with %+v", sg, s.Metadata.ConfState.Learners)
 	}
-	for _, n := range s.Metadata.ConfState.Nodes {
+	for _, n := range s.Metadata.ConfState.Voters {
 		if sm.prs.Progress[n].IsLearner {
 			t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs.Progress[n], false)
 		}
@@ -2794,7 +2794,7 @@ func TestRestoreVoterToLearner(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     11, // magic number
 			Term:      11, // magic number
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2}, Learners: []uint64{3}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2}, Learners: []uint64{3}},
 		},
 	}
 
@@ -2816,7 +2816,7 @@ func TestRestoreLearnerPromotion(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     11, // magic number
 			Term:      11, // magic number
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2, 3}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2, 3}},
 		},
 	}
 
@@ -2843,7 +2843,7 @@ func TestLearnerReceiveSnapshot(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     11, // magic number
 			Term:      11, // magic number
-			ConfState: pb.ConfState{Nodes: []uint64{1}, Learners: []uint64{2}},
+			ConfState: pb.ConfState{Voters: []uint64{1}, Learners: []uint64{2}},
 		},
 	}
 
@@ -2881,7 +2881,7 @@ func TestRestoreIgnoreSnapshot(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     commit,
 			Term:      1,
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2}},
 		},
 	}
 
@@ -2909,7 +2909,7 @@ func TestProvideSnap(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     11, // magic number
 			Term:      11, // magic number
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2}},
 		},
 	}
 	storage := NewMemoryStorage()
@@ -2939,7 +2939,7 @@ func TestIgnoreProvidingSnap(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     11, // magic number
 			Term:      11, // magic number
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2}},
 		},
 	}
 	storage := NewMemoryStorage()
@@ -2967,7 +2967,7 @@ func TestRestoreFromSnapMsg(t *testing.T) {
 		Metadata: pb.SnapshotMetadata{
 			Index:     11, // magic number
 			Term:      11, // magic number
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2}},
 		},
 	}
 	m := pb.Message{Type: pb.MsgSnap, From: 1, Term: 2, Snapshot: s}
@@ -2992,7 +2992,7 @@ func TestSlowNodeRestore(t *testing.T) {
 	}
 	lead := nt.peers[1].(*raft)
 	nextEnts(lead, nt.storage[1])
-	nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Nodes: lead.prs.VoterNodes()}, nil)
+	nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Voters: lead.prs.VoterNodes()}, nil)
 	nt.storage[1].Compact(lead.raftLog.applied)
 
 	nt.recover()
@@ -3469,7 +3469,7 @@ func TestLeaderTransferAfterSnapshot(t *testing.T) {
 	nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
 	lead := nt.peers[1].(*raft)
 	nextEnts(lead, nt.storage[1])
-	nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Nodes: lead.prs.VoterNodes()}, nil)
+	nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Voters: lead.prs.VoterNodes()}, nil)
 	nt.storage[1].Compact(lead.raftLog.applied)
 
 	nt.recover()

+ 104 - 88
raft/raftpb/raft.pb.go

@@ -171,13 +171,19 @@ type ConfChangeTransition int32
 
 const (
 	// Automatically use the simple protocol if possible, otherwise fall back
-	// to ConfChangeJointImplicit.
+	// to ConfChangeJointImplicit. Most applications will want to use this.
 	ConfChangeTransitionAuto ConfChangeTransition = 0
-	// Use joint consensus, but transition out of the joint configuration
-	// automatically by proposing a no-op configuration change.
+	// Use joint consensus unconditionally, and transition out of them
+	// automatically (by proposing a zero configuration change).
+	//
+	// This option is suitable for applications that want to minimize the time
+	// spent in the joint configuration and do not store the joint configuration
+	// in the state machine (outside of InitialState).
 	ConfChangeTransitionJointImplicit ConfChangeTransition = 1
 	// Use joint consensus and remain in the joint configuration until the
-	// application proposes a no-op configuration change.
+	// application proposes a no-op configuration change. This is suitable for
+	// applications that want to explicitly control the transitions, for example
+	// to use a custom payload (via the Context field).
 	ConfChangeTransitionJointExplicit ConfChangeTransition = 2
 )
 
@@ -320,12 +326,21 @@ func (*HardState) ProtoMessage()               {}
 func (*HardState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{4} }
 
 type ConfState struct {
-	Nodes            []uint64 `protobuf:"varint,1,rep,name=nodes" json:"nodes,omitempty"`
-	NodesJoint       []uint64 `protobuf:"varint,3,rep,name=nodes_joint,json=nodesJoint" json:"nodes_joint,omitempty"`
-	Learners         []uint64 `protobuf:"varint,2,rep,name=learners" json:"learners,omitempty"`
-	LearnersNext     []uint64 `protobuf:"varint,4,rep,name=learners_next,json=learnersNext" json:"learners_next,omitempty"`
-	AutoLeave        bool     `protobuf:"varint,5,opt,name=auto_leave,json=autoLeave" json:"auto_leave"`
-	XXX_unrecognized []byte   `json:"-"`
+	// The voters in the incoming config. (If the configuration is not joint,
+	// then the outgoing config is empty).
+	Voters []uint64 `protobuf:"varint,1,rep,name=voters" json:"voters,omitempty"`
+	// The learners in the incoming config.
+	Learners []uint64 `protobuf:"varint,2,rep,name=learners" json:"learners,omitempty"`
+	// The voters in the outgoing config.
+	VotersOutgoing []uint64 `protobuf:"varint,3,rep,name=voters_outgoing,json=votersOutgoing" json:"voters_outgoing,omitempty"`
+	// The nodes that will become learners when the outgoing config is removed.
+	// These nodes are necessarily currently in nodes_joint (or they would have
+	// been added to the incoming config right away).
+	LearnersNext []uint64 `protobuf:"varint,4,rep,name=learners_next,json=learnersNext" json:"learners_next,omitempty"`
+	// If set, the config is joint and Raft will automatically transition into
+	// the final config (i.e. remove the outgoing config) when this is safe.
+	AutoLeave        bool   `protobuf:"varint,5,opt,name=auto_leave,json=autoLeave" json:"auto_leave"`
+	XXX_unrecognized []byte `json:"-"`
 }
 
 func (m *ConfState) Reset()                    { *m = ConfState{} }
@@ -651,8 +666,8 @@ func (m *ConfState) MarshalTo(dAtA []byte) (int, error) {
 	_ = i
 	var l int
 	_ = l
-	if len(m.Nodes) > 0 {
-		for _, num := range m.Nodes {
+	if len(m.Voters) > 0 {
+		for _, num := range m.Voters {
 			dAtA[i] = 0x8
 			i++
 			i = encodeVarintRaft(dAtA, i, uint64(num))
@@ -665,8 +680,8 @@ func (m *ConfState) MarshalTo(dAtA []byte) (int, error) {
 			i = encodeVarintRaft(dAtA, i, uint64(num))
 		}
 	}
-	if len(m.NodesJoint) > 0 {
-		for _, num := range m.NodesJoint {
+	if len(m.VotersOutgoing) > 0 {
+		for _, num := range m.VotersOutgoing {
 			dAtA[i] = 0x18
 			i++
 			i = encodeVarintRaft(dAtA, i, uint64(num))
@@ -896,8 +911,8 @@ func (m *HardState) Size() (n int) {
 func (m *ConfState) Size() (n int) {
 	var l int
 	_ = l
-	if len(m.Nodes) > 0 {
-		for _, e := range m.Nodes {
+	if len(m.Voters) > 0 {
+		for _, e := range m.Voters {
 			n += 1 + sovRaft(uint64(e))
 		}
 	}
@@ -906,8 +921,8 @@ func (m *ConfState) Size() (n int) {
 			n += 1 + sovRaft(uint64(e))
 		}
 	}
-	if len(m.NodesJoint) > 0 {
-		for _, e := range m.NodesJoint {
+	if len(m.VotersOutgoing) > 0 {
+		for _, e := range m.VotersOutgoing {
 			n += 1 + sovRaft(uint64(e))
 		}
 	}
@@ -1822,7 +1837,7 @@ func (m *ConfState) Unmarshal(dAtA []byte) error {
 						break
 					}
 				}
-				m.Nodes = append(m.Nodes, v)
+				m.Voters = append(m.Voters, v)
 			} else if wireType == 2 {
 				var packedLen int
 				for shift := uint(0); ; shift += 7 {
@@ -1862,10 +1877,10 @@ func (m *ConfState) Unmarshal(dAtA []byte) error {
 							break
 						}
 					}
-					m.Nodes = append(m.Nodes, v)
+					m.Voters = append(m.Voters, v)
 				}
 			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field Nodes", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Voters", wireType)
 			}
 		case 2:
 			if wireType == 0 {
@@ -1946,7 +1961,7 @@ func (m *ConfState) Unmarshal(dAtA []byte) error {
 						break
 					}
 				}
-				m.NodesJoint = append(m.NodesJoint, v)
+				m.VotersOutgoing = append(m.VotersOutgoing, v)
 			} else if wireType == 2 {
 				var packedLen int
 				for shift := uint(0); ; shift += 7 {
@@ -1986,10 +2001,10 @@ func (m *ConfState) Unmarshal(dAtA []byte) error {
 							break
 						}
 					}
-					m.NodesJoint = append(m.NodesJoint, v)
+					m.VotersOutgoing = append(m.VotersOutgoing, v)
 				}
 			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field NodesJoint", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field VotersOutgoing", wireType)
 			}
 		case 4:
 			if wireType == 0 {
@@ -2563,68 +2578,69 @@ var (
 func init() { proto.RegisterFile("raft.proto", fileDescriptorRaft) }
 
 var fileDescriptorRaft = []byte{
-	// 999 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcf, 0x6f, 0x23, 0x35,
-	0x14, 0x8e, 0x27, 0x93, 0x5f, 0x2f, 0x69, 0xea, 0x7a, 0x03, 0xb2, 0xaa, 0x2a, 0x0d, 0x01, 0x44,
-	0x54, 0xb4, 0x05, 0xe5, 0x80, 0x10, 0xb7, 0xfe, 0x58, 0xa9, 0x41, 0x4d, 0xb5, 0xa4, 0xdd, 0x1e,
-	0x90, 0x50, 0xe4, 0x66, 0x9c, 0xe9, 0x2c, 0x99, 0xf1, 0xc8, 0xe3, 0x94, 0xf6, 0x82, 0x10, 0x7f,
-	0x05, 0x97, 0xbd, 0x22, 0xfe, 0x94, 0x1e, 0x57, 0xe2, 0xbe, 0x62, 0xcb, 0x3f, 0x82, 0xec, 0xf1,
-	0x24, 0x93, 0xb4, 0xda, 0x03, 0x37, 0xfb, 0xfb, 0x3e, 0xbf, 0xf7, 0xbd, 0xe7, 0xe7, 0x19, 0x00,
-	0xc9, 0xa6, 0x6a, 0x3f, 0x96, 0x42, 0x09, 0x52, 0xd6, 0xeb, 0xf8, 0x6a, 0xbb, 0xe5, 0x0b, 0x5f,
-	0x18, 0xe8, 0x2b, 0xbd, 0x4a, 0xd9, 0xee, 0xaf, 0x50, 0x7a, 0x11, 0x29, 0x79, 0x47, 0xbe, 0x04,
-	0xf7, 0xe2, 0x2e, 0xe6, 0x14, 0x75, 0x50, 0xaf, 0xd9, 0xdf, 0xda, 0x4f, 0x4f, 0xed, 0x1b, 0x52,
-	0x13, 0x87, 0xee, 0xfd, 0xbb, 0xdd, 0xc2, 0xc8, 0x88, 0x08, 0x05, 0xf7, 0x82, 0xcb, 0x90, 0x3a,
-	0x1d, 0xd4, 0x73, 0x17, 0x0c, 0x97, 0x21, 0xd9, 0x86, 0xd2, 0x20, 0xf2, 0xf8, 0x2d, 0x2d, 0xe6,
-	0xa8, 0x14, 0x22, 0x04, 0xdc, 0x63, 0xa6, 0x18, 0x75, 0x3b, 0xa8, 0xd7, 0x18, 0x99, 0x75, 0xf7,
-	0x37, 0x04, 0xf8, 0x3c, 0x62, 0x71, 0x72, 0x2d, 0xd4, 0x90, 0x2b, 0xe6, 0x31, 0xc5, 0xc8, 0x37,
-	0x00, 0x13, 0x11, 0x4d, 0xc7, 0x89, 0x62, 0x2a, 0x75, 0x54, 0x5f, 0x3a, 0x3a, 0x12, 0xd1, 0xf4,
-	0x5c, 0x13, 0x36, 0x78, 0x6d, 0x92, 0x01, 0x3a, 0x79, 0x60, 0x92, 0xe7, 0x7d, 0xa5, 0x90, 0xb6,
-	0xac, 0xb4, 0xe5, 0xbc, 0x2f, 0x83, 0x74, 0x7f, 0x84, 0x6a, 0xe6, 0x40, 0x5b, 0xd4, 0x0e, 0x4c,
-	0xce, 0xc6, 0xc8, 0xac, 0xc9, 0x77, 0x50, 0x0d, 0xad, 0x33, 0x13, 0xb8, 0xde, 0xa7, 0x99, 0x97,
-	0x75, 0xe7, 0x36, 0xee, 0x42, 0xdf, 0x7d, 0x53, 0x84, 0xca, 0x90, 0x27, 0x09, 0xf3, 0x39, 0x79,
-	0x0e, 0xae, 0x5a, 0x76, 0xf8, 0x59, 0x16, 0xc3, 0xd2, 0xf9, 0x1e, 0x6b, 0x19, 0x69, 0x81, 0xa3,
-	0xc4, 0x4a, 0x25, 0x8e, 0x12, 0xba, 0x8c, 0xa9, 0x14, 0x6b, 0x65, 0x68, 0x64, 0x51, 0xa0, 0xbb,
-	0x5e, 0x20, 0x69, 0x43, 0x65, 0x26, 0x7c, 0x73, 0x61, 0xa5, 0x1c, 0x99, 0x81, 0xcb, 0xb6, 0x95,
-	0x1f, 0xb7, 0xed, 0x39, 0x54, 0x78, 0xa4, 0x64, 0xc0, 0x13, 0x5a, 0xe9, 0x14, 0x7b, 0xf5, 0xfe,
-	0xc6, 0xca, 0x64, 0x64, 0xa1, 0xac, 0x86, 0xec, 0x40, 0x79, 0x22, 0xc2, 0x30, 0x50, 0xb4, 0x9a,
-	0x8b, 0x65, 0x31, 0xd2, 0x87, 0x6a, 0x62, 0x3b, 0x46, 0x6b, 0xa6, 0x93, 0x78, 0xbd, 0x93, 0x59,
-	0x07, 0x33, 0x9d, 0x8e, 0x28, 0xf9, 0x6b, 0x3e, 0x51, 0x14, 0x3a, 0xa8, 0x57, 0xcd, 0x22, 0xa6,
-	0x18, 0xf9, 0x0c, 0x20, 0x5d, 0x9d, 0x04, 0x91, 0xa2, 0xf5, 0x5c, 0xce, 0x1c, 0x4e, 0x28, 0x54,
-	0x26, 0x22, 0x52, 0xfc, 0x56, 0xd1, 0x86, 0xb9, 0xd8, 0x6c, 0xdb, 0xfd, 0x09, 0x6a, 0x27, 0x4c,
-	0x7a, 0xe9, 0xf8, 0x64, 0x1d, 0x44, 0x8f, 0x3a, 0x48, 0xc1, 0xbd, 0x11, 0x8a, 0xaf, 0xce, 0xbb,
-	0x46, 0x72, 0x05, 0x17, 0x1f, 0x17, 0xdc, 0xfd, 0x0b, 0x41, 0x6d, 0x31, 0xaf, 0xa4, 0x05, 0xa5,
-	0x48, 0x78, 0x3c, 0xa1, 0xa8, 0x53, 0xec, 0xb9, 0xa3, 0x74, 0x43, 0xb6, 0xa1, 0x3a, 0xe3, 0x4c,
-	0x46, 0x5c, 0x26, 0xd4, 0x31, 0xc4, 0x62, 0x4f, 0x76, 0xa1, 0x6e, 0x44, 0xe3, 0xd7, 0x42, 0xd7,
-	0x57, 0x34, 0x34, 0x18, 0xe8, 0x7b, 0x8d, 0x90, 0x4f, 0x61, 0x23, 0x13, 0x8f, 0x23, 0x5d, 0x9f,
-	0x6b, 0x24, 0x8d, 0x0c, 0x3c, 0xe3, 0xb7, 0x5a, 0x04, 0x6c, 0xae, 0xc4, 0x78, 0xc6, 0xd9, 0x0d,
-	0x37, 0x23, 0x90, 0xb5, 0xb1, 0xa6, 0xf1, 0x53, 0x0d, 0x77, 0xdf, 0x20, 0x00, 0x6d, 0xf5, 0xe8,
-	0x9a, 0x45, 0xbe, 0x7e, 0x4a, 0x4e, 0xe0, 0xd9, 0x4e, 0x80, 0xd6, 0x3e, 0xbc, 0xdb, 0x75, 0x06,
-	0xc7, 0x23, 0x27, 0xf0, 0xc8, 0xd7, 0x76, 0x90, 0x1d, 0x33, 0xc8, 0x1f, 0xe7, 0x1f, 0x66, 0x7a,
-	0xfa, 0xd1, 0x2c, 0x7f, 0x01, 0x15, 0x6d, 0x7a, 0x1c, 0x78, 0xb6, 0x4d, 0x4d, 0x1b, 0xb2, 0x7c,
-	0x26, 0x3c, 0x3e, 0x38, 0x1e, 0x95, 0x35, 0x3d, 0xf0, 0xf2, 0x37, 0xe5, 0xae, 0xde, 0x54, 0x08,
-	0x78, 0x99, 0xe0, 0x3c, 0x88, 0xfc, 0x19, 0x5f, 0x18, 0x41, 0xff, 0xc7, 0x88, 0xf3, 0x21, 0x23,
-	0xdd, 0x3f, 0x11, 0x34, 0x96, 0x71, 0x2e, 0xfb, 0xe4, 0x10, 0x40, 0x49, 0x16, 0x25, 0x81, 0x0a,
-	0x44, 0x64, 0x33, 0xee, 0x3c, 0x91, 0x71, 0xa1, 0xc9, 0xe6, 0x70, 0x79, 0x8a, 0x7c, 0x0b, 0x95,
-	0x89, 0x51, 0xa5, 0x37, 0x9d, 0xfb, 0x90, 0xac, 0x97, 0x96, 0xbd, 0x2b, 0x2b, 0xcf, 0xf7, 0xa5,
-	0xb8, 0xd2, 0x97, 0xbd, 0x13, 0xa8, 0x2d, 0xbe, 0xd1, 0x64, 0x13, 0xea, 0x66, 0x73, 0x26, 0x64,
-	0xc8, 0x66, 0xb8, 0x40, 0x9e, 0xc1, 0xa6, 0x01, 0x96, 0xf1, 0x31, 0x22, 0x1f, 0xc1, 0xd6, 0x1a,
-	0x78, 0xd9, 0xc7, 0xce, 0xde, 0xdf, 0x0e, 0xd4, 0x73, 0x1f, 0x23, 0x02, 0x50, 0x1e, 0x26, 0xfe,
-	0xc9, 0x3c, 0xc6, 0x05, 0x52, 0x87, 0xca, 0x30, 0xf1, 0x0f, 0x39, 0x53, 0x18, 0xd9, 0xcd, 0x4b,
-	0x29, 0x62, 0xec, 0x58, 0xd5, 0x41, 0x1c, 0xe3, 0x22, 0x69, 0x02, 0xa4, 0xeb, 0x11, 0x4f, 0x62,
-	0xec, 0x5a, 0xe1, 0xa5, 0x50, 0x1c, 0x97, 0xb4, 0x37, 0xbb, 0x31, 0x6c, 0xd9, 0xb2, 0xfa, 0xe1,
-	0xe3, 0x0a, 0xc1, 0xd0, 0xd0, 0xc9, 0x38, 0x93, 0xea, 0x4a, 0x67, 0xa9, 0x92, 0x16, 0xe0, 0x3c,
-	0x62, 0x0e, 0xd5, 0x08, 0x81, 0xe6, 0x30, 0xf1, 0x5f, 0x45, 0x92, 0xb3, 0xc9, 0x35, 0xbb, 0x9a,
-	0x71, 0x0c, 0x64, 0x0b, 0x36, 0x6c, 0x20, 0xfd, 0xce, 0xe6, 0x09, 0xae, 0x5b, 0xd9, 0xd1, 0x35,
-	0x9f, 0xfc, 0xfc, 0xc3, 0x5c, 0xc8, 0x79, 0x88, 0x1b, 0xba, 0xec, 0x61, 0xe2, 0x9b, 0x0b, 0x9a,
-	0x72, 0x79, 0xca, 0x99, 0xc7, 0x25, 0xde, 0xb0, 0xa7, 0x2f, 0x82, 0x90, 0x8b, 0xb9, 0x3a, 0x13,
-	0xbf, 0xe0, 0xa6, 0x35, 0x33, 0xe2, 0xcc, 0x33, 0x3f, 0x2e, 0xbc, 0x69, 0xcd, 0x2c, 0x10, 0x63,
-	0x06, 0xdb, 0x7a, 0x5f, 0x4a, 0x6e, 0x4a, 0xdc, 0xb2, 0x59, 0xed, 0xde, 0x68, 0xc8, 0xde, 0xef,
-	0x08, 0x5a, 0x4f, 0x8d, 0x07, 0xd9, 0x01, 0xfa, 0x14, 0x7e, 0x30, 0x57, 0x02, 0x17, 0xc8, 0xe7,
-	0xf0, 0xc9, 0x53, 0xac, 0x79, 0xf5, 0x83, 0x30, 0x9e, 0x05, 0x93, 0x40, 0x5f, 0xc5, 0x87, 0x64,
-	0x2f, 0x6e, 0xad, 0xcc, 0xd9, 0xbb, 0x83, 0xe6, 0xea, 0xa3, 0xd0, 0xcd, 0x58, 0x22, 0x07, 0x9e,
-	0xa7, 0xc7, 0x1f, 0x17, 0x08, 0xcd, 0x9b, 0x1d, 0xf1, 0x50, 0xdc, 0x70, 0xc3, 0xa0, 0x55, 0xe6,
-	0x55, 0xec, 0x31, 0x95, 0x32, 0xce, 0x6a, 0x21, 0x07, 0x9e, 0x77, 0x9a, 0x7e, 0x7b, 0x0c, 0x5b,
-	0x3c, 0xa4, 0xf7, 0xef, 0xdb, 0x85, 0xb7, 0xef, 0xdb, 0x85, 0xfb, 0x87, 0x36, 0x7a, 0xfb, 0xd0,
-	0x46, 0xff, 0x3c, 0xb4, 0xd1, 0x1f, 0xff, 0xb6, 0x0b, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x39,
-	0x56, 0x89, 0xed, 0xa5, 0x08, 0x00, 0x00,
+	// 1009 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcd, 0x6e, 0xe3, 0x36,
+	0x17, 0xb5, 0x64, 0xc5, 0x3f, 0xd7, 0x8e, 0xc3, 0xdc, 0xc9, 0x37, 0x20, 0x82, 0xc0, 0xe3, 0xcf,
+	0xd3, 0x62, 0x8c, 0x14, 0x93, 0x16, 0x5e, 0x14, 0x45, 0x77, 0xf9, 0x19, 0x20, 0x29, 0xe2, 0x74,
+	0xea, 0x64, 0xb2, 0x28, 0x50, 0x04, 0x8c, 0x45, 0x2b, 0x6a, 0x2d, 0x51, 0xa0, 0xe8, 0x34, 0xd9,
+	0x14, 0x45, 0x9f, 0xa2, 0x9b, 0xd9, 0xf6, 0x01, 0xfa, 0x14, 0x59, 0x0e, 0xd0, 0xfd, 0xa0, 0x93,
+	0xbe, 0x48, 0x41, 0x8a, 0xb2, 0x65, 0x27, 0x98, 0x45, 0x77, 0xe4, 0x39, 0x87, 0xf7, 0x9e, 0x7b,
+	0x79, 0x45, 0x01, 0x48, 0x36, 0x56, 0x3b, 0x89, 0x14, 0x4a, 0x60, 0x45, 0xaf, 0x93, 0xcb, 0xcd,
+	0x8d, 0x40, 0x04, 0xc2, 0x40, 0x9f, 0xeb, 0x55, 0xc6, 0x76, 0x7f, 0x81, 0x95, 0x57, 0xb1, 0x92,
+	0xb7, 0xf8, 0x19, 0x78, 0x67, 0xb7, 0x09, 0xa7, 0x4e, 0xc7, 0xe9, 0xb5, 0xfa, 0xeb, 0x3b, 0xd9,
+	0xa9, 0x1d, 0x43, 0x6a, 0x62, 0xcf, 0xbb, 0x7b, 0xff, 0xac, 0x34, 0x34, 0x22, 0xa4, 0xe0, 0x9d,
+	0x71, 0x19, 0x51, 0xb7, 0xe3, 0xf4, 0xbc, 0x19, 0xc3, 0x65, 0x84, 0x9b, 0xb0, 0x72, 0x14, 0xfb,
+	0xfc, 0x86, 0x96, 0x0b, 0x54, 0x06, 0x21, 0x82, 0x77, 0xc0, 0x14, 0xa3, 0x5e, 0xc7, 0xe9, 0x35,
+	0x87, 0x66, 0xdd, 0xfd, 0xd5, 0x01, 0x72, 0x1a, 0xb3, 0x24, 0xbd, 0x12, 0x6a, 0xc0, 0x15, 0xf3,
+	0x99, 0x62, 0xf8, 0x25, 0xc0, 0x48, 0xc4, 0xe3, 0x8b, 0x54, 0x31, 0x95, 0x39, 0x6a, 0xcc, 0x1d,
+	0xed, 0x8b, 0x78, 0x7c, 0xaa, 0x09, 0x1b, 0xbc, 0x3e, 0xca, 0x01, 0x9d, 0x3c, 0x34, 0xc9, 0x8b,
+	0xbe, 0x32, 0x48, 0x5b, 0x56, 0xda, 0x72, 0xd1, 0x97, 0x41, 0xba, 0xdf, 0x43, 0x2d, 0x77, 0xa0,
+	0x2d, 0x6a, 0x07, 0x26, 0x67, 0x73, 0x68, 0xd6, 0xf8, 0x35, 0xd4, 0x22, 0xeb, 0xcc, 0x04, 0x6e,
+	0xf4, 0x69, 0xee, 0x65, 0xd9, 0xb9, 0x8d, 0x3b, 0xd3, 0x77, 0xdf, 0x96, 0xa1, 0x3a, 0xe0, 0x69,
+	0xca, 0x02, 0x8e, 0x2f, 0xc1, 0x53, 0xf3, 0x0e, 0x3f, 0xc9, 0x63, 0x58, 0xba, 0xd8, 0x63, 0x2d,
+	0xc3, 0x0d, 0x70, 0x95, 0x58, 0xa8, 0xc4, 0x55, 0x42, 0x97, 0x31, 0x96, 0x62, 0xa9, 0x0c, 0x8d,
+	0xcc, 0x0a, 0xf4, 0x96, 0x0b, 0xc4, 0x36, 0x54, 0x27, 0x22, 0x30, 0x17, 0xb6, 0x52, 0x20, 0x73,
+	0x70, 0xde, 0xb6, 0xca, 0xc3, 0xb6, 0xbd, 0x84, 0x2a, 0x8f, 0x95, 0x0c, 0x79, 0x4a, 0xab, 0x9d,
+	0x72, 0xaf, 0xd1, 0x5f, 0x5d, 0x98, 0x8c, 0x3c, 0x94, 0xd5, 0xe0, 0x16, 0x54, 0x46, 0x22, 0x8a,
+	0x42, 0x45, 0x6b, 0x85, 0x58, 0x16, 0xc3, 0x3e, 0xd4, 0x52, 0xdb, 0x31, 0x5a, 0x37, 0x9d, 0x24,
+	0xcb, 0x9d, 0xcc, 0x3b, 0x98, 0xeb, 0x74, 0x44, 0xc9, 0x7f, 0xe4, 0x23, 0x45, 0xa1, 0xe3, 0xf4,
+	0x6a, 0x79, 0xc4, 0x0c, 0xc3, 0x4f, 0x00, 0xb2, 0xd5, 0x61, 0x18, 0x2b, 0xda, 0x28, 0xe4, 0x2c,
+	0xe0, 0x48, 0xa1, 0x3a, 0x12, 0xb1, 0xe2, 0x37, 0x8a, 0x36, 0xcd, 0xc5, 0xe6, 0xdb, 0xee, 0x0f,
+	0x50, 0x3f, 0x64, 0xd2, 0xcf, 0xc6, 0x27, 0xef, 0xa0, 0xf3, 0xa0, 0x83, 0x14, 0xbc, 0x6b, 0xa1,
+	0xf8, 0xe2, 0xbc, 0x6b, 0xa4, 0x50, 0x70, 0xf9, 0x61, 0xc1, 0xdd, 0x3f, 0x1d, 0xa8, 0xcf, 0xe6,
+	0x15, 0x9f, 0x42, 0x45, 0x9f, 0x91, 0x29, 0x75, 0x3a, 0xe5, 0x9e, 0x37, 0xb4, 0x3b, 0xdc, 0x84,
+	0xda, 0x84, 0x33, 0x19, 0x6b, 0xc6, 0x35, 0xcc, 0x6c, 0x8f, 0x2f, 0x60, 0x2d, 0x53, 0x5d, 0x88,
+	0xa9, 0x0a, 0x44, 0x18, 0x07, 0xb4, 0x6c, 0x24, 0xad, 0x0c, 0xfe, 0xd6, 0xa2, 0xf8, 0x1c, 0x56,
+	0xf3, 0x43, 0x17, 0xb1, 0xae, 0xd4, 0x33, 0xb2, 0x66, 0x0e, 0x9e, 0xf0, 0x1b, 0x85, 0xcf, 0x01,
+	0xd8, 0x54, 0x89, 0x8b, 0x09, 0x67, 0xd7, 0xdc, 0x0c, 0x43, 0xde, 0xd0, 0xba, 0xc6, 0x8f, 0x35,
+	0xdc, 0x7d, 0xeb, 0x00, 0x68, 0xd3, 0xfb, 0x57, 0x2c, 0x0e, 0xf4, 0x47, 0xe5, 0x86, 0xbe, 0xed,
+	0x09, 0x68, 0xed, 0xfd, 0xfb, 0x67, 0xee, 0xd1, 0xc1, 0xd0, 0x0d, 0x7d, 0xfc, 0xc2, 0x8e, 0xb4,
+	0x6b, 0x46, 0xfa, 0x69, 0xf1, 0x13, 0xcd, 0x4e, 0x3f, 0x98, 0xea, 0x17, 0x50, 0x8d, 0x85, 0xcf,
+	0x2f, 0x42, 0xdf, 0x36, 0xac, 0x65, 0x43, 0x56, 0x4e, 0x84, 0xcf, 0x8f, 0x0e, 0x86, 0x15, 0x4d,
+	0x1f, 0xf9, 0xc5, 0x3b, 0xf3, 0x16, 0xef, 0x2c, 0x02, 0x32, 0x4f, 0x70, 0x1a, 0xc6, 0xc1, 0x84,
+	0xcf, 0x8c, 0x38, 0xff, 0xc5, 0x88, 0xfb, 0x31, 0x23, 0xdd, 0x3f, 0x1c, 0x68, 0xce, 0xe3, 0x9c,
+	0xf7, 0x71, 0x0f, 0x40, 0x49, 0x16, 0xa7, 0xa1, 0x0a, 0x45, 0x6c, 0x33, 0x6e, 0x3d, 0x92, 0x71,
+	0xa6, 0xc9, 0x27, 0x72, 0x7e, 0x0a, 0xbf, 0x82, 0xea, 0xc8, 0xa8, 0xb2, 0x1b, 0x2f, 0x3c, 0x29,
+	0xcb, 0xa5, 0xe5, 0x5f, 0x98, 0x95, 0x17, 0xfb, 0x52, 0x5e, 0xe8, 0xcb, 0xf6, 0x21, 0xd4, 0x67,
+	0xaf, 0x35, 0xae, 0x41, 0xc3, 0x6c, 0x4e, 0x84, 0x8c, 0xd8, 0x84, 0x94, 0xf0, 0x09, 0xac, 0x19,
+	0x60, 0x1e, 0x9f, 0x38, 0xf8, 0x3f, 0x58, 0x5f, 0x02, 0xcf, 0xfb, 0xc4, 0xdd, 0xfe, 0xcb, 0x85,
+	0x46, 0xe1, 0x59, 0x42, 0x80, 0xca, 0x20, 0x0d, 0x0e, 0xa7, 0x09, 0x29, 0x61, 0x03, 0xaa, 0x83,
+	0x34, 0xd8, 0xe3, 0x4c, 0x11, 0xc7, 0x6e, 0x5e, 0x4b, 0x91, 0x10, 0xd7, 0xaa, 0x76, 0x93, 0x84,
+	0x94, 0xb1, 0x05, 0x90, 0xad, 0x87, 0x3c, 0x4d, 0x88, 0x67, 0x85, 0xe7, 0x42, 0x71, 0xb2, 0xa2,
+	0xbd, 0xd9, 0x8d, 0x61, 0x2b, 0x96, 0xd5, 0x4f, 0x00, 0xa9, 0x22, 0x81, 0xa6, 0x4e, 0xc6, 0x99,
+	0x54, 0x97, 0x3a, 0x4b, 0x0d, 0x37, 0x80, 0x14, 0x11, 0x73, 0xa8, 0x8e, 0x08, 0xad, 0x41, 0x1a,
+	0xbc, 0x89, 0x25, 0x67, 0xa3, 0x2b, 0x76, 0x39, 0xe1, 0x04, 0x70, 0x1d, 0x56, 0x6d, 0x20, 0xfd,
+	0xc5, 0x4d, 0x53, 0xd2, 0xb0, 0xb2, 0xfd, 0x2b, 0x3e, 0xfa, 0xe9, 0xbb, 0xa9, 0x90, 0xd3, 0x88,
+	0x34, 0x75, 0xd9, 0x83, 0x34, 0x30, 0x17, 0x34, 0xe6, 0xf2, 0x98, 0x33, 0x9f, 0x4b, 0xb2, 0x6a,
+	0x4f, 0x9f, 0x85, 0x11, 0x17, 0x53, 0x75, 0x22, 0x7e, 0x26, 0x2d, 0x6b, 0x66, 0xc8, 0x99, 0x6f,
+	0x7e, 0x61, 0x64, 0xcd, 0x9a, 0x99, 0x21, 0xc6, 0x0c, 0xb1, 0xf5, 0xbe, 0x96, 0xdc, 0x94, 0xb8,
+	0x6e, 0xb3, 0xda, 0xbd, 0xd1, 0xe0, 0xf6, 0x6f, 0x0e, 0x6c, 0x3c, 0x36, 0x1e, 0xb8, 0x05, 0xf4,
+	0x31, 0x7c, 0x77, 0xaa, 0x04, 0x29, 0xe1, 0xa7, 0xf0, 0xff, 0xc7, 0xd8, 0x6f, 0x44, 0x18, 0xab,
+	0xa3, 0x28, 0x99, 0x84, 0xa3, 0x50, 0x5f, 0xc5, 0xc7, 0x64, 0xaf, 0x6e, 0xac, 0xcc, 0xdd, 0xbe,
+	0x85, 0xd6, 0xe2, 0x47, 0xa1, 0x9b, 0x31, 0x47, 0x76, 0x7d, 0x5f, 0x8f, 0x3f, 0x29, 0x21, 0x2d,
+	0x9a, 0x1d, 0xf2, 0x48, 0x5c, 0x73, 0xc3, 0x38, 0x8b, 0xcc, 0x9b, 0xc4, 0x67, 0x2a, 0x63, 0xdc,
+	0xc5, 0x42, 0x76, 0x7d, 0xff, 0x38, 0x7b, 0x7b, 0x0c, 0x5b, 0xde, 0xa3, 0x77, 0x1f, 0xda, 0xa5,
+	0x77, 0x1f, 0xda, 0xa5, 0xbb, 0xfb, 0xb6, 0xf3, 0xee, 0xbe, 0xed, 0xfc, 0x7d, 0xdf, 0x76, 0x7e,
+	0xff, 0xa7, 0x5d, 0xfa, 0x37, 0x00, 0x00, 0xff, 0xff, 0x87, 0x11, 0x6d, 0xd6, 0xaf, 0x08, 0x00,
+	0x00,
 }

+ 6 - 6
raft/raftpb/raft.proto

@@ -99,18 +99,18 @@ enum ConfChangeTransition {
 message ConfState {
 	// The voters in the incoming config. (If the configuration is not joint,
 	// then the outgoing config is empty).
-	repeated uint64 nodes          = 1;
-	// The voters in the outgoing config.
-	repeated uint64 nodes_joint    = 3;
+	repeated uint64 voters = 1;
 	// The learners in the incoming config.
-	repeated uint64 learners       = 2;
+	repeated uint64 learners          = 2;
+	// The voters in the outgoing config.
+	repeated uint64 voters_outgoing   = 3;
 	// The nodes that will become learners when the outgoing config is removed.
 	// These nodes are necessarily currently in nodes_joint (or they would have
 	// been added to the incoming config right away).
-	repeated uint64 learners_next  = 4;
+	repeated uint64 learners_next     = 4;
 	// If set, the config is joint and Raft will automatically transition into
 	// the final config (i.e. remove the outgoing config) when this is safe.
-	optional bool   auto_leave     = 5 [(gogoproto.nullable) = false];
+	optional bool   auto_leave        = 5 [(gogoproto.nullable) = false];
 }
 
 enum ConfChangeType {

+ 29 - 29
raft/rawnode_test.go

@@ -77,7 +77,7 @@ func TestRawNodeStep(t *testing.T) {
 			s.Append([]pb.Entry{{Term: 1, Index: 1}})
 			if err := s.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{
 				ConfState: pb.ConfState{
-					Nodes: []uint64{1},
+					Voters: []uint64{1},
 				},
 				Index: 1,
 				Term:  1,
@@ -118,7 +118,7 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
 		// V1 config change.
 		{
 			pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: 2},
-			pb.ConfState{Nodes: []uint64{1, 2}},
+			pb.ConfState{Voters: []uint64{1, 2}},
 			nil,
 		},
 		// Proposing the same as a V2 change works just the same, without entering
@@ -128,7 +128,7 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
 				{Type: pb.ConfChangeAddNode, NodeID: 2},
 			},
 			},
-			pb.ConfState{Nodes: []uint64{1, 2}},
+			pb.ConfState{Voters: []uint64{1, 2}},
 			nil,
 		},
 		// Ditto if we add it as a learner instead.
@@ -137,7 +137,7 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
 				{Type: pb.ConfChangeAddLearnerNode, NodeID: 2},
 			},
 			},
-			pb.ConfState{Nodes: []uint64{1}, Learners: []uint64{2}},
+			pb.ConfState{Voters: []uint64{1}, Learners: []uint64{2}},
 			nil,
 		},
 		// We can ask explicitly for joint consensus if we want it.
@@ -147,8 +147,8 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
 			},
 				Transition: pb.ConfChangeTransitionJointExplicit,
 			},
-			pb.ConfState{Nodes: []uint64{1}, NodesJoint: []uint64{1}, Learners: []uint64{2}},
-			&pb.ConfState{Nodes: []uint64{1}, Learners: []uint64{2}},
+			pb.ConfState{Voters: []uint64{1}, VotersOutgoing: []uint64{1}, Learners: []uint64{2}},
+			&pb.ConfState{Voters: []uint64{1}, Learners: []uint64{2}},
 		},
 		// Ditto, but with implicit transition (the harness checks this).
 		{
@@ -158,10 +158,10 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
 				Transition: pb.ConfChangeTransitionJointImplicit,
 			},
 			pb.ConfState{
-				Nodes: []uint64{1}, NodesJoint: []uint64{1}, Learners: []uint64{2},
+				Voters: []uint64{1}, VotersOutgoing: []uint64{1}, Learners: []uint64{2},
 				AutoLeave: true,
 			},
-			&pb.ConfState{Nodes: []uint64{1}, Learners: []uint64{2}},
+			&pb.ConfState{Voters: []uint64{1}, Learners: []uint64{2}},
 		},
 		// Add a new node and demote n1. This exercises the interesting case in
 		// which we really need joint config changes and also need LearnersNext.
@@ -173,13 +173,13 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
 			},
 			},
 			pb.ConfState{
-				Nodes:        []uint64{2},
-				NodesJoint:   []uint64{1},
-				Learners:     []uint64{3},
-				LearnersNext: []uint64{1},
-				AutoLeave:    true,
+				Voters:         []uint64{2},
+				VotersOutgoing: []uint64{1},
+				Learners:       []uint64{3},
+				LearnersNext:   []uint64{1},
+				AutoLeave:      true,
 			},
-			&pb.ConfState{Nodes: []uint64{2}, Learners: []uint64{1, 3}},
+			&pb.ConfState{Voters: []uint64{2}, Learners: []uint64{1, 3}},
 		},
 		// Ditto explicit.
 		{
@@ -191,12 +191,12 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
 				Transition: pb.ConfChangeTransitionJointExplicit,
 			},
 			pb.ConfState{
-				Nodes:        []uint64{2},
-				NodesJoint:   []uint64{1},
-				Learners:     []uint64{3},
-				LearnersNext: []uint64{1},
+				Voters:         []uint64{2},
+				VotersOutgoing: []uint64{1},
+				Learners:       []uint64{3},
+				LearnersNext:   []uint64{1},
 			},
-			&pb.ConfState{Nodes: []uint64{2}, Learners: []uint64{1, 3}},
+			&pb.ConfState{Voters: []uint64{2}, Learners: []uint64{1, 3}},
 		},
 		// Ditto implicit.
 		{
@@ -208,13 +208,13 @@ func TestRawNodeProposeAndConfChange(t *testing.T) {
 				Transition: pb.ConfChangeTransitionJointImplicit,
 			},
 			pb.ConfState{
-				Nodes:        []uint64{2},
-				NodesJoint:   []uint64{1},
-				Learners:     []uint64{3},
-				LearnersNext: []uint64{1},
-				AutoLeave:    true,
+				Voters:         []uint64{2},
+				VotersOutgoing: []uint64{1},
+				Learners:       []uint64{3},
+				LearnersNext:   []uint64{1},
+				AutoLeave:      true,
 			},
-			&pb.ConfState{Nodes: []uint64{2}, Learners: []uint64{1, 3}},
+			&pb.ConfState{Voters: []uint64{2}, Learners: []uint64{1, 3}},
 		},
 	}
 
@@ -542,7 +542,7 @@ func TestRawNodeStart(t *testing.T) {
 		ApplySnapshot(pb.Snapshot) error
 	}
 	bootstrap := func(storage appenderStorage, cs pb.ConfState) error {
-		if len(cs.Nodes) == 0 {
+		if len(cs.Voters) == 0 {
 			return fmt.Errorf("no voters specified")
 		}
 		fi, err := storage.FirstIndex()
@@ -570,7 +570,7 @@ func TestRawNodeStart(t *testing.T) {
 		if !IsEmptyHardState(hs) {
 			return fmt.Errorf("HardState not empty")
 		}
-		if len(ics.Nodes) != 0 {
+		if len(ics.Voters) != 0 {
 			return fmt.Errorf("ConfState not empty")
 		}
 
@@ -583,7 +583,7 @@ func TestRawNodeStart(t *testing.T) {
 		return storage.ApplySnapshot(snap)
 	}
 
-	if err := bootstrap(storage, pb.ConfState{Nodes: []uint64{1}}); err != nil {
+	if err := bootstrap(storage, pb.ConfState{Voters: []uint64{1}}); err != nil {
 		t.Fatal(err)
 	}
 
@@ -648,7 +648,7 @@ func TestRawNodeRestart(t *testing.T) {
 func TestRawNodeRestartFromSnapshot(t *testing.T) {
 	snap := pb.Snapshot{
 		Metadata: pb.SnapshotMetadata{
-			ConfState: pb.ConfState{Nodes: []uint64{1, 2}},
+			ConfState: pb.ConfState{Voters: []uint64{1, 2}},
 			Index:     2,
 			Term:      1,
 		},

+ 2 - 2
raft/storage_test.go

@@ -177,7 +177,7 @@ func TestStorageCompact(t *testing.T) {
 
 func TestStorageCreateSnapshot(t *testing.T) {
 	ents := []pb.Entry{{Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 5}}
-	cs := &pb.ConfState{Nodes: []uint64{1, 2, 3}}
+	cs := &pb.ConfState{Voters: []uint64{1, 2, 3}}
 	data := []byte("data")
 
 	tests := []struct {
@@ -263,7 +263,7 @@ func TestStorageAppend(t *testing.T) {
 }
 
 func TestStorageApplySnapshot(t *testing.T) {
-	cs := &pb.ConfState{Nodes: []uint64{1, 2, 3}}
+	cs := &pb.ConfState{Voters: []uint64{1, 2, 3}}
 	data := []byte("data")
 
 	tests := []pb.Snapshot{{Data: data, Metadata: pb.SnapshotMetadata{Index: 4, Term: 4, ConfState: *cs}},

+ 1 - 2
tools/etcd-dump-logs/main.go

@@ -35,7 +35,6 @@ import (
 	"go.etcd.io/etcd/raft/raftpb"
 	"go.etcd.io/etcd/wal"
 	"go.etcd.io/etcd/wal/walpb"
-
 	"go.uber.org/zap"
 )
 
@@ -83,7 +82,7 @@ func main() {
 		switch err {
 		case nil:
 			walsnap.Index, walsnap.Term = snapshot.Metadata.Index, snapshot.Metadata.Term
-			nodes := genIDSlice(snapshot.Metadata.ConfState.Nodes)
+			nodes := genIDSlice(snapshot.Metadata.ConfState.Voters)
 			fmt.Printf("Snapshot:\nterm=%d index=%d nodes=%s\n",
 				walsnap.Term, walsnap.Index, nodes)
 		case snap.ErrNoSnapshot: