From 9cff5fd2b94e53b2e5ac2b0bebcc1bea0e746549 Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Mon, 26 Aug 2019 13:20:48 -0700 Subject: [PATCH 01/23] cli: Add default locality settings for multi node demo clusters Addresses part of #39938. Release note (cli change): Default cluster locality topologies for multi-node cockroach demo clusters. --- pkg/cli/demo.go | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/pkg/cli/demo.go b/pkg/cli/demo.go index 8ee424830ab9..db9d08d6a2b2 100644 --- a/pkg/cli/demo.go +++ b/pkg/cli/demo.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cli/cliflags" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -51,6 +52,20 @@ to avoid pre-loading a dataset.`, const defaultGeneratorName = "movr" var defaultGenerator workload.Generator +var defaultLocalities = []roachpb.Locality{ + // Default localities for a 3 node cluster + {Tiers: []roachpb.Tier{{Key: "region", Value: "us-east1"}, {Key: "az", Value: "b"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "us-east1"}, {Key: "az", Value: "c"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "us-east1"}, {Key: "az", Value: "d"}}}, + // Default localities for a 6 node cluster + {Tiers: []roachpb.Tier{{Key: "region", Value: "us-west1"}, {Key: "az", Value: "a"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "us-west1"}, {Key: "az", Value: "b"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "us-west1"}, {Key: "az", Value: "c"}}}, + // Default localities for a 9 node cluster + {Tiers: []roachpb.Tier{{Key: "region", Value: "europe-west1"}, {Key: "az", Value: "b"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "europe-west1"}, {Key: "az", Value: "c"}}}, + {Tiers: []roachpb.Tier{{Key: "region", Value: "europe-west1"}, {Key: "az", Value: "d"}}}, +} func init() { for _, meta := range workload.Registered() { @@ -90,12 +105,18 @@ func setupTransientServers( return "", "", cleanup, errors.Errorf("must have a positive number of nodes") } + // The user specified some localities for their nodes. if len(demoCtx.localities) != 0 { // Error out of localities don't line up with requested node // count before doing any sort of setup. if len(demoCtx.localities) != demoCtx.nodes { return "", "", cleanup, errors.Errorf("number of localities specified must equal number of nodes") } + } else { + demoCtx.localities = make([]roachpb.Locality, demoCtx.nodes) + for i := 0; i < demoCtx.nodes; i++ { + demoCtx.localities[i] = defaultLocalities[i%len(defaultLocalities)] + } } // Set up logging. For demo/transient server we use non-standard From 24a17e16c3cf7a641fa6b21bec8313a233a68097 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 16 Aug 2019 17:34:59 +0200 Subject: [PATCH 02/23] vendor: bump go.etcd.io/etcd/raft This picks up improvements related to joint consensus. We're pinning a SHA that is close to master but not quite there to avoid tickling to a bug in CRDB: https://github.com/cockroachdb/cockroach/issues/40207 Release note: None --- Gopkg.lock | 7 ++++--- Gopkg.toml | 8 +++++++- vendor | 2 +- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 4701d218e640..5073392697b0 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -1508,8 +1508,7 @@ revision = "b5bfa59ec0adc420475f97f89b58045c721d761c" [[projects]] - branch = "master" - digest = "1:7b2765d91bece066a1cb4928d8b39cce152a6a08602e13b1169648c2f200cfff" + digest = "1:a893ff8345b4fea61f22af749d8bbe6c546e9ed36b52bb80da25e991a884cc8e" name = "go.etcd.io/etcd" packages = [ "raft", @@ -1519,7 +1518,7 @@ "raft/tracker", ] pruneopts = "UT" - revision = "4a4629fd9f01d4e2669c4f537a56a0019d95617e" + revision = "9b29151d3072511f574e7272a5348504086013fa" [[projects]] digest = "1:3b5a3bc35810830ded5e26ef9516e933083a2380d8e57371fdfde3c70d7c6952" @@ -2051,6 +2050,8 @@ "github.com/stretchr/testify/require", "github.com/wadey/gocovmerge", "go.etcd.io/etcd/raft", + "go.etcd.io/etcd/raft/confchange", + "go.etcd.io/etcd/raft/quorum", "go.etcd.io/etcd/raft/raftpb", "go.etcd.io/etcd/raft/tracker", "golang.org/x/crypto/bcrypt", diff --git a/Gopkg.toml b/Gopkg.toml index 9143acf49e61..29d30086606b 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -38,7 +38,13 @@ ignored = [ [[constraint]] name = "go.etcd.io/etcd" - branch = "master" + # We're stopping just shy of 4a2b4c8f7e0a3754fdd5a3341a27c2431c2a5385 + # which picks up a fix to an inefficiency that at the time of writing + # triggers a bug: + # https://github.com/cockroachdb/cockroach/issues/40207 + # + # branch = "master" + revision = "9b29151d3072511f574e7272a5348504086013fa" # Used for the API client; we want the latest. [[constraint]] diff --git a/vendor b/vendor index 32b7d49bdc5e..ce3756cc233f 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit 32b7d49bdc5ec4e25b11c096254fa5dbf9025738 +Subproject commit ce3756cc233fdf3810025c7cce6d5d137f13918e From 073ffd8725fec0785b1ed0d0f8e5a5b2b0ee07e3 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 13:55:13 +0200 Subject: [PATCH 03/23] roachpb: generalize ReplicaDescriptor.String() The new code will generalize to new replica types. Release note: None --- pkg/roachpb/metadata.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index eef2bea4ce31..81bbec1a635c 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -278,8 +278,8 @@ func (r ReplicaDescriptor) String() string { } else { fmt.Fprintf(&buf, "%d", r.ReplicaID) } - if r.GetType() == ReplicaType_LEARNER { - buf.WriteString("LEARNER") + if typ := r.GetType(); typ != ReplicaType_VOTER { + buf.WriteString(strings.ToUpper(typ.String())) } return buf.String() } From add4b5c992a556a698137a718016e2e745671b0e Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Thu, 22 Aug 2019 15:24:18 +0200 Subject: [PATCH 04/23] roachpb: improve RangeDescriptor.Validate Make sure there isn't more than one replica per store. Release note: None --- pkg/roachpb/metadata.go | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index 81bbec1a635c..9606d29b4048 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -220,18 +220,25 @@ func (r *RangeDescriptor) Validate() error { return errors.Errorf("NextReplicaID must be non-zero") } seen := map[ReplicaID]struct{}{} + stores := map[StoreID]struct{}{} for i, rep := range r.Replicas().All() { if err := rep.Validate(); err != nil { return errors.Errorf("replica %d is invalid: %s", i, err) } + if rep.ReplicaID >= r.NextReplicaID { + return errors.Errorf("ReplicaID %d must be less than NextReplicaID %d", + rep.ReplicaID, r.NextReplicaID) + } + if _, ok := seen[rep.ReplicaID]; ok { return errors.Errorf("ReplicaID %d was reused", rep.ReplicaID) } seen[rep.ReplicaID] = struct{}{} - if rep.ReplicaID >= r.NextReplicaID { - return errors.Errorf("ReplicaID %d must be less than NextReplicaID %d", - rep.ReplicaID, r.NextReplicaID) + + if _, ok := stores[rep.StoreID]; ok { + return errors.Errorf("StoreID %d was reused", rep.StoreID) } + stores[rep.StoreID] = struct{}{} } return nil } From 9915f0d0f104aa918c94340e9e47129b90421999 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 17:10:26 +0200 Subject: [PATCH 05/23] batcheval: generalize checkNotLearnerReplica This now errors out whenever the replica is not a voter, which is more robust as new replica types are introduced (which generally should not automatically become eligible to receive leases). Release note: None --- pkg/storage/batcheval/cmd_lease.go | 4 ++-- pkg/storage/batcheval/cmd_lease_request.go | 2 +- pkg/storage/batcheval/cmd_lease_transfer.go | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/storage/batcheval/cmd_lease.go b/pkg/storage/batcheval/cmd_lease.go index 5bfaddaf4b1f..dbe5f557570d 100644 --- a/pkg/storage/batcheval/cmd_lease.go +++ b/pkg/storage/batcheval/cmd_lease.go @@ -42,12 +42,12 @@ func newFailedLeaseTrigger(isTransfer bool) result.Result { return trigger } -func checkNotLearnerReplica(rec EvalContext) error { +func checkCanReceiveLease(rec EvalContext) error { repDesc, ok := rec.Desc().GetReplicaDescriptor(rec.StoreID()) if !ok { return errors.AssertionFailedf( `could not find replica for store %s in %s`, rec.StoreID(), rec.Desc()) - } else if t := repDesc.GetType(); t == roachpb.ReplicaType_LEARNER { + } else if t := repDesc.GetType(); t != roachpb.ReplicaType_VOTER { return errors.Errorf(`cannot transfer lease to replica of type %s`, t) } return nil diff --git a/pkg/storage/batcheval/cmd_lease_request.go b/pkg/storage/batcheval/cmd_lease_request.go index f45caf1d3176..af23d4b111a5 100644 --- a/pkg/storage/batcheval/cmd_lease_request.go +++ b/pkg/storage/batcheval/cmd_lease_request.go @@ -47,7 +47,7 @@ func RequestLease( // // If this check is removed at some point, the filtering of learners on the // sending side would have to be removed as well. - if err := checkNotLearnerReplica(cArgs.EvalCtx); err != nil { + if err := checkCanReceiveLease(cArgs.EvalCtx); err != nil { return newFailedLeaseTrigger(false /* isTransfer */), err } diff --git a/pkg/storage/batcheval/cmd_lease_transfer.go b/pkg/storage/batcheval/cmd_lease_transfer.go index 91012c9e08ab..946def98b443 100644 --- a/pkg/storage/batcheval/cmd_lease_transfer.go +++ b/pkg/storage/batcheval/cmd_lease_transfer.go @@ -46,7 +46,7 @@ func TransferLease( // // If this check is removed at some point, the filtering of learners on the // sending side would have to be removed as well. - if err := checkNotLearnerReplica(cArgs.EvalCtx); err != nil { + if err := checkCanReceiveLease(cArgs.EvalCtx); err != nil { return newFailedLeaseTrigger(true /* isTransfer */), err } From ad4f3dd61600e60ba714a4885f7a9cb163b3d04d Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 13:48:38 +0200 Subject: [PATCH 06/23] roachpb: rename ReplicaType variants The current naming is idiomatic for proto enums, but atypical for its usage in Go code. There is no `(gogoproto.customname)` that can fix this, and we're about to add more replica types that would require awkward names such as `roachpb.ReplicaType_VOTER_OUTGOING`. Switch to a Go-friendly naming scheme instead. Release note: None --- c-deps/libroach/protos/roachpb/metadata.pb.h | 8 +- pkg/roachpb/data_test.go | 4 +- pkg/roachpb/metadata.go | 4 +- pkg/roachpb/metadata.pb.go | 224 +++++++++---------- pkg/roachpb/metadata.proto | 8 +- pkg/roachpb/metadata_replicas.go | 16 +- pkg/roachpb/metadata_replicas_test.go | 8 +- pkg/storage/allocator_test.go | 2 +- pkg/storage/batcheval/cmd_lease.go | 2 +- pkg/storage/batcheval/cmd_lease_test.go | 4 +- pkg/storage/client_test.go | 2 +- pkg/storage/merge_queue.go | 4 +- pkg/storage/raft_snapshot_queue.go | 2 +- pkg/storage/replica_command.go | 6 +- pkg/storage/replica_follower_read.go | 2 +- pkg/storage/replica_learner_test.go | 4 +- pkg/storage/replica_proposal_buf.go | 4 +- pkg/storage/replica_range_lease.go | 2 +- 18 files changed, 153 insertions(+), 153 deletions(-) diff --git a/c-deps/libroach/protos/roachpb/metadata.pb.h b/c-deps/libroach/protos/roachpb/metadata.pb.h index 99b287622382..2524c3f336df 100644 --- a/c-deps/libroach/protos/roachpb/metadata.pb.h +++ b/c-deps/libroach/protos/roachpb/metadata.pb.h @@ -114,12 +114,12 @@ namespace cockroach { namespace roachpb { enum ReplicaType { - VOTER = 0, - LEARNER = 1 + Voter = 0, + Learner = 1 }; bool ReplicaType_IsValid(int value); -const ReplicaType ReplicaType_MIN = VOTER; -const ReplicaType ReplicaType_MAX = LEARNER; +const ReplicaType ReplicaType_MIN = Voter; +const ReplicaType ReplicaType_MAX = Learner; const int ReplicaType_ARRAYSIZE = ReplicaType_MAX + 1; // =================================================================== diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 7b91cdce61e9..11d550db0494 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1635,8 +1635,8 @@ func TestUpdateObservedTimestamps(t *testing.T) { func TestChangeReplicasTrigger_String(t *testing.T) { defer leaktest.AfterTest(t)() - l := ReplicaType_LEARNER - v := ReplicaType_VOTER + l := ReplicaType_Learner + v := ReplicaType_Voter repl1 := ReplicaDescriptor{NodeID: 1, StoreID: 2, ReplicaID: 3, Type: &l} repl2 := ReplicaDescriptor{NodeID: 4, StoreID: 5, ReplicaID: 6, Type: &v} crt := ChangeReplicasTrigger{ diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index 9606d29b4048..6eceb5613149 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -285,7 +285,7 @@ func (r ReplicaDescriptor) String() string { } else { fmt.Fprintf(&buf, "%d", r.ReplicaID) } - if typ := r.GetType(); typ != ReplicaType_VOTER { + if typ := r.GetType(); typ != ReplicaType_Voter { buf.WriteString(strings.ToUpper(typ.String())) } return buf.String() @@ -308,7 +308,7 @@ func (r ReplicaDescriptor) Validate() error { // GetType returns the type of this ReplicaDescriptor. func (r ReplicaDescriptor) GetType() ReplicaType { if r.Type == nil { - return ReplicaType_VOTER + return ReplicaType_Voter } return *r.Type } diff --git a/pkg/roachpb/metadata.pb.go b/pkg/roachpb/metadata.pb.go index 636137b3e940..c4974dfc544c 100644 --- a/pkg/roachpb/metadata.pb.go +++ b/pkg/roachpb/metadata.pb.go @@ -30,26 +30,26 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package type ReplicaType int32 const ( - // ReplicaType_VOTER indicates a replica that participates in all raft + // ReplicaType_Voter indicates a replica that participates in all raft // activities, including voting for leadership and committing entries. // Notably, voters are considered for quorum size of the raft group. - ReplicaType_VOTER ReplicaType = 0 - // ReplicaType_LEARNER indicates a replica that applies committed entries, but + ReplicaType_Voter ReplicaType = 0 + // ReplicaType_Learner indicates a replica that applies committed entries, but // does not count towards the quorum. Learners do not vote for leadership nor // do their acknowledged log entries get taken into account for determining // the committed index. At the time of writing, learners in CockroachDB are a // short-term transient state: a replica being added and on its way to being a // VOTER. - ReplicaType_LEARNER ReplicaType = 1 + ReplicaType_Learner ReplicaType = 1 ) var ReplicaType_name = map[int32]string{ - 0: "VOTER", - 1: "LEARNER", + 0: "Voter", + 1: "Learner", } var ReplicaType_value = map[string]int32{ - "VOTER": 0, - "LEARNER": 1, + "Voter": 0, + "Learner": 1, } func (x ReplicaType) Enum() *ReplicaType { @@ -69,7 +69,7 @@ func (x *ReplicaType) UnmarshalJSON(data []byte) error { return nil } func (ReplicaType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{0} + return fileDescriptor_metadata_ebc18063440435c8, []int{0} } // Attributes specifies a list of arbitrary strings describing @@ -81,7 +81,7 @@ type Attributes struct { func (m *Attributes) Reset() { *m = Attributes{} } func (*Attributes) ProtoMessage() {} func (*Attributes) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{0} + return fileDescriptor_metadata_ebc18063440435c8, []int{0} } func (m *Attributes) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -115,7 +115,7 @@ type ReplicationTarget struct { func (m *ReplicationTarget) Reset() { *m = ReplicationTarget{} } func (*ReplicationTarget) ProtoMessage() {} func (*ReplicationTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{1} + return fileDescriptor_metadata_ebc18063440435c8, []int{1} } func (m *ReplicationTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -160,7 +160,7 @@ type ReplicaDescriptor struct { func (m *ReplicaDescriptor) Reset() { *m = ReplicaDescriptor{} } func (*ReplicaDescriptor) ProtoMessage() {} func (*ReplicaDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{2} + return fileDescriptor_metadata_ebc18063440435c8, []int{2} } func (m *ReplicaDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -195,7 +195,7 @@ func (m *ReplicaIdent) Reset() { *m = ReplicaIdent{} } func (m *ReplicaIdent) String() string { return proto.CompactTextString(m) } func (*ReplicaIdent) ProtoMessage() {} func (*ReplicaIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{3} + return fileDescriptor_metadata_ebc18063440435c8, []int{3} } func (m *ReplicaIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -352,7 +352,7 @@ type RangeDescriptor struct { func (m *RangeDescriptor) Reset() { *m = RangeDescriptor{} } func (*RangeDescriptor) ProtoMessage() {} func (*RangeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{4} + return fileDescriptor_metadata_ebc18063440435c8, []int{4} } func (m *RangeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -391,7 +391,7 @@ type Percentiles struct { func (m *Percentiles) Reset() { *m = Percentiles{} } func (*Percentiles) ProtoMessage() {} func (*Percentiles) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{5} + return fileDescriptor_metadata_ebc18063440435c8, []int{5} } func (m *Percentiles) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -453,7 +453,7 @@ type StoreCapacity struct { func (m *StoreCapacity) Reset() { *m = StoreCapacity{} } func (*StoreCapacity) ProtoMessage() {} func (*StoreCapacity) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{6} + return fileDescriptor_metadata_ebc18063440435c8, []int{6} } func (m *StoreCapacity) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -498,7 +498,7 @@ func (m *NodeDescriptor) Reset() { *m = NodeDescriptor{} } func (m *NodeDescriptor) String() string { return proto.CompactTextString(m) } func (*NodeDescriptor) ProtoMessage() {} func (*NodeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{7} + return fileDescriptor_metadata_ebc18063440435c8, []int{7} } func (m *NodeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -534,7 +534,7 @@ func (m *LocalityAddress) Reset() { *m = LocalityAddress{} } func (m *LocalityAddress) String() string { return proto.CompactTextString(m) } func (*LocalityAddress) ProtoMessage() {} func (*LocalityAddress) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{8} + return fileDescriptor_metadata_ebc18063440435c8, []int{8} } func (m *LocalityAddress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -572,7 +572,7 @@ func (m *StoreDescriptor) Reset() { *m = StoreDescriptor{} } func (m *StoreDescriptor) String() string { return proto.CompactTextString(m) } func (*StoreDescriptor) ProtoMessage() {} func (*StoreDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{9} + return fileDescriptor_metadata_ebc18063440435c8, []int{9} } func (m *StoreDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -609,7 +609,7 @@ func (m *StoreDeadReplicas) Reset() { *m = StoreDeadReplicas{} } func (m *StoreDeadReplicas) String() string { return proto.CompactTextString(m) } func (*StoreDeadReplicas) ProtoMessage() {} func (*StoreDeadReplicas) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{10} + return fileDescriptor_metadata_ebc18063440435c8, []int{10} } func (m *StoreDeadReplicas) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -643,7 +643,7 @@ type Locality struct { func (m *Locality) Reset() { *m = Locality{} } func (*Locality) ProtoMessage() {} func (*Locality) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{11} + return fileDescriptor_metadata_ebc18063440435c8, []int{11} } func (m *Locality) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -679,7 +679,7 @@ type Tier struct { func (m *Tier) Reset() { *m = Tier{} } func (*Tier) ProtoMessage() {} func (*Tier) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{12} + return fileDescriptor_metadata_ebc18063440435c8, []int{12} } func (m *Tier) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +720,7 @@ type Version struct { func (m *Version) Reset() { *m = Version{} } func (*Version) ProtoMessage() {} func (*Version) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_512b13eb8d4291cc, []int{13} + return fileDescriptor_metadata_ebc18063440435c8, []int{13} } func (m *Version) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4000,93 +4000,93 @@ var ( ErrIntOverflowMetadata = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_512b13eb8d4291cc) } - -var fileDescriptor_metadata_512b13eb8d4291cc = []byte{ - // 1359 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0x41, 0x6f, 0xdb, 0xc6, - 0x12, 0x16, 0x2d, 0xc9, 0xa2, 0x46, 0x76, 0x6c, 0x2d, 0x92, 0x3c, 0x41, 0x0f, 0x4f, 0x52, 0xf8, - 0x5e, 0xf0, 0x9c, 0xb4, 0xb0, 0x1d, 0x07, 0x46, 0x10, 0xb7, 0x69, 0x6b, 0x25, 0x2e, 0xe0, 0x26, - 0x71, 0x5d, 0xda, 0x4d, 0x81, 0x5e, 0x88, 0x35, 0xb9, 0x95, 0xd9, 0x50, 0x24, 0xb3, 0x5c, 0x39, - 0xd1, 0xbd, 0x40, 0x0f, 0x45, 0x81, 0x5e, 0x0a, 0xf4, 0x98, 0x4b, 0xfe, 0x43, 0x7f, 0x42, 0x8e, - 0x39, 0xe6, 0x64, 0xa4, 0xce, 0xa5, 0xe7, 0x1e, 0x73, 0x28, 0x8a, 0x1d, 0xee, 0x92, 0x94, 0xe3, - 0xb4, 0x49, 0x0a, 0xf4, 0x46, 0xcd, 0x7c, 0xdf, 0x68, 0x76, 0xf6, 0x9b, 0x99, 0x85, 0xb3, 0x3c, - 0xa2, 0xee, 0x7e, 0xbc, 0xb7, 0x34, 0x64, 0x82, 0x7a, 0x54, 0xd0, 0xc5, 0x98, 0x47, 0x22, 0x22, - 0x4d, 0x37, 0x72, 0xef, 0xa2, 0x6f, 0x51, 0x21, 0xda, 0xed, 0x91, 0xf0, 0x83, 0xa5, 0x51, 0xc8, - 0x59, 0x12, 0x05, 0x07, 0xcc, 0x73, 0xa8, 0xe7, 0xf1, 0x14, 0xde, 0x6e, 0xa1, 0x6f, 0x3f, 0x70, - 0x97, 0x84, 0x3f, 0x64, 0x89, 0xa0, 0xc3, 0x58, 0x79, 0x4e, 0x0f, 0xa2, 0x41, 0x84, 0x9f, 0x4b, - 0xf2, 0x2b, 0xb5, 0x5a, 0x1f, 0x02, 0xac, 0x0b, 0xc1, 0xfd, 0xbd, 0x91, 0x60, 0x09, 0x79, 0x07, - 0xaa, 0x54, 0x08, 0x9e, 0xb4, 0x8c, 0x5e, 0x79, 0xa1, 0xde, 0x3f, 0xf3, 0xdb, 0x61, 0xb7, 0x39, - 0xa6, 0xc3, 0x60, 0xcd, 0x42, 0xf3, 0xbb, 0x5f, 0x05, 0xd1, 0x7d, 0xcb, 0x4e, 0x31, 0x6b, 0x95, - 0x9f, 0x1e, 0x76, 0x4b, 0xd6, 0xb7, 0x06, 0x34, 0x6d, 0x16, 0x07, 0xbe, 0x4b, 0x85, 0x1f, 0x85, - 0xbb, 0x94, 0x0f, 0x98, 0x20, 0x97, 0xa0, 0x16, 0x46, 0x1e, 0x73, 0x7c, 0xaf, 0x65, 0xf4, 0x8c, - 0x85, 0x6a, 0xbf, 0xf5, 0xf8, 0xb0, 0x5b, 0x3a, 0x3a, 0xec, 0x4e, 0x6f, 0x45, 0x1e, 0xdb, 0xbc, - 0xf1, 0x22, 0xfb, 0xb2, 0xa7, 0x25, 0x70, 0xd3, 0x23, 0xab, 0x60, 0x26, 0x22, 0xe2, 0xc8, 0x99, - 0x42, 0x4e, 0x5b, 0x71, 0x6a, 0x3b, 0xd2, 0x8e, 0x24, 0xfd, 0x69, 0xd7, 0x10, 0xbb, 0xe9, 0xad, - 0x99, 0x32, 0x8b, 0x5f, 0x1f, 0x76, 0x0d, 0xeb, 0xf7, 0x3c, 0x93, 0x1b, 0x2c, 0x71, 0xb9, 0x1f, - 0x8b, 0x88, 0xff, 0x73, 0x99, 0x90, 0x6b, 0x00, 0x3c, 0xfd, 0x7b, 0x49, 0x2c, 0x23, 0xb1, 0xa3, - 0x88, 0x75, 0x95, 0x18, 0x52, 0xf3, 0x1f, 0x76, 0x5d, 0x31, 0x36, 0x3d, 0xb2, 0x02, 0x15, 0x31, - 0x8e, 0x59, 0xab, 0xd2, 0x33, 0x16, 0x4e, 0xad, 0x74, 0x16, 0x5f, 0xba, 0xf7, 0x45, 0x45, 0xdb, - 0x1d, 0xc7, 0xcc, 0x46, 0xec, 0xda, 0x8c, 0x3c, 0xfc, 0xcf, 0x0f, 0xbb, 0x06, 0x16, 0xe0, 0x3b, - 0x03, 0x66, 0x74, 0x68, 0x8f, 0x85, 0x42, 0x1e, 0x84, 0xd3, 0x70, 0x90, 0x1d, 0xbe, 0x9c, 0x1f, - 0xc4, 0x96, 0xf6, 0xf4, 0x20, 0xea, 0xd3, 0xae, 0x21, 0x76, 0xd3, 0x23, 0x37, 0xa0, 0xa6, 0xd2, - 0xc2, 0xe3, 0x37, 0x56, 0xfe, 0xf7, 0xea, 0x64, 0xf2, 0x4a, 0xf7, 0x2b, 0x32, 0xb6, 0xad, 0xa9, - 0xd6, 0xd3, 0x32, 0xcc, 0x61, 0xe8, 0xc2, 0x65, 0xbc, 0x65, 0x42, 0xe7, 0xa1, 0x9e, 0x08, 0xca, - 0x85, 0x73, 0x97, 0x8d, 0x31, 0xa5, 0x99, 0xbe, 0xf9, 0xe2, 0xb0, 0x5b, 0xb1, 0x6f, 0xb2, 0xb1, - 0x6d, 0xa2, 0xeb, 0x26, 0x1b, 0x93, 0x73, 0x50, 0x63, 0xa1, 0x87, 0xa0, 0xf2, 0x31, 0xd0, 0x34, - 0x0b, 0x3d, 0x09, 0xf9, 0x02, 0x9a, 0x7e, 0x28, 0x18, 0x0f, 0x69, 0xe0, 0xa8, 0x44, 0x93, 0x56, - 0xa5, 0x57, 0x7e, 0xc3, 0x43, 0xce, 0xeb, 0x20, 0x0a, 0x90, 0x90, 0x4f, 0x60, 0x2e, 0x64, 0x0f, - 0x84, 0x53, 0x50, 0x40, 0x15, 0x15, 0x60, 0xa9, 0x03, 0xce, 0x6e, 0xb1, 0x07, 0xe2, 0x15, 0x2a, - 0x98, 0x0d, 0x0b, 0x3e, 0x8f, 0x74, 0x00, 0x06, 0x2c, 0x64, 0x1c, 0x1b, 0xaa, 0x35, 0x2d, 0xeb, - 0x64, 0x17, 0x2c, 0xe4, 0x7d, 0x80, 0x44, 0xf8, 0xee, 0xdd, 0xb1, 0xb3, 0xe7, 0x8b, 0x56, 0x0d, - 0xaf, 0xe8, 0x3f, 0x85, 0xec, 0xe5, 0x08, 0x58, 0xdc, 0x0f, 0xdc, 0xc5, 0x5d, 0x3d, 0x02, 0xec, - 0x7a, 0x4a, 0xe8, 0xfb, 0x82, 0x5c, 0x86, 0x33, 0x79, 0x2c, 0xc7, 0x8d, 0x86, 0x31, 0xe5, 0x74, - 0x2f, 0x60, 0x2d, 0xb3, 0x67, 0x2c, 0x98, 0xf6, 0xe9, 0xdc, 0x79, 0x3d, 0xf3, 0x1d, 0x13, 0xda, - 0x23, 0x03, 0x1a, 0xdb, 0x8c, 0xbb, 0x2c, 0x14, 0x7e, 0xc0, 0x12, 0x72, 0x16, 0xca, 0xf1, 0xa5, - 0x65, 0xbc, 0x51, 0x43, 0x55, 0x48, 0x1a, 0xd0, 0xbe, 0xb2, 0x8a, 0x37, 0x96, 0xdb, 0x57, 0x56, - 0xd1, 0xbe, 0xba, 0x8c, 0x97, 0x94, 0xdb, 0x57, 0x53, 0xfc, 0x95, 0x55, 0xec, 0x80, 0xdc, 0x7e, - 0x25, 0xc5, 0x5f, 0x5d, 0xc6, 0x82, 0xe6, 0xf6, 0xab, 0xcb, 0xa4, 0x05, 0x95, 0xf8, 0x36, 0x7d, - 0x80, 0x25, 0xd2, 0x0e, 0xb4, 0xa8, 0xd9, 0xf4, 0xa2, 0x0c, 0xb3, 0xd8, 0xa6, 0xd7, 0x69, 0x4c, - 0x5d, 0x5f, 0x8c, 0x49, 0x0f, 0x4c, 0x57, 0x7d, 0x2b, 0x01, 0xa6, 0xac, 0xcc, 0x4a, 0x2c, 0xa8, - 0xd3, 0x03, 0xea, 0x07, 0x58, 0x92, 0xa9, 0x02, 0x24, 0x37, 0x93, 0xf3, 0xd0, 0x48, 0x65, 0xec, - 0x46, 0xa3, 0x50, 0xa8, 0x56, 0x4f, 0x51, 0x80, 0x8e, 0xeb, 0xd2, 0x2e, 0x61, 0x01, 0xa3, 0x89, - 0x86, 0x55, 0x8a, 0x30, 0x74, 0xa4, 0xb0, 0x65, 0x68, 0xde, 0xe7, 0xbe, 0x60, 0x89, 0x13, 0x33, - 0xee, 0x24, 0xcc, 0x8d, 0x42, 0x6f, 0xe2, 0xac, 0x73, 0xa9, 0x7b, 0x9b, 0xf1, 0x1d, 0x74, 0x92, - 0x6d, 0x68, 0xee, 0x8d, 0x35, 0x41, 0xb7, 0xea, 0x34, 0xea, 0xe0, 0xa4, 0xb9, 0x51, 0xb8, 0x2a, - 0x1d, 0x11, 0xe9, 0xdb, 0x8c, 0x2b, 0xd1, 0x11, 0x1b, 0x48, 0x21, 0x07, 0x1d, 0xb2, 0xf6, 0x06, - 0x21, 0xe7, 0xb3, 0x24, 0x75, 0xcc, 0x16, 0x54, 0x46, 0x09, 0xf3, 0x50, 0x57, 0xba, 0x88, 0x68, - 0x21, 0x17, 0x60, 0x36, 0x88, 0x06, 0xbe, 0x4b, 0x03, 0x07, 0x13, 0x69, 0xd5, 0x0b, 0x90, 0x19, - 0xe5, 0xea, 0x4b, 0x0f, 0x59, 0x01, 0x72, 0x6f, 0xc4, 0xb8, 0x3f, 0x59, 0x1d, 0x28, 0x54, 0x67, - 0x5e, 0xf9, 0xb3, 0xf2, 0xa8, 0xcb, 0x7f, 0x56, 0x81, 0x53, 0x72, 0xb0, 0xff, 0xbd, 0x5d, 0xf0, - 0x01, 0xd4, 0xe4, 0x76, 0x65, 0x49, 0xa2, 0x66, 0x61, 0xe7, 0x78, 0xa3, 0x7d, 0x9e, 0xed, 0xe1, - 0x75, 0xcf, 0xcb, 0xa6, 0xa0, 0x22, 0x91, 0xab, 0x7a, 0xa3, 0x96, 0x5f, 0x6a, 0x53, 0x5d, 0xcb, - 0x7c, 0xff, 0x2a, 0x72, 0xca, 0x20, 0xd7, 0xc0, 0x0c, 0x22, 0x97, 0x06, 0x52, 0xab, 0x15, 0x64, - 0xff, 0xfb, 0x04, 0xf6, 0x2d, 0x05, 0xd1, 0x42, 0xd6, 0x14, 0xf2, 0x31, 0xcc, 0xee, 0x30, 0x7e, - 0xc0, 0xf8, 0x1d, 0xc6, 0x13, 0x39, 0x48, 0xaa, 0x18, 0xa3, 0x7d, 0x42, 0x0c, 0x85, 0x50, 0x21, - 0x26, 0x69, 0xe4, 0x1c, 0xd4, 0xf7, 0x46, 0x7e, 0xe0, 0x39, 0x82, 0x0e, 0x50, 0x64, 0x75, 0xfd, - 0x57, 0x68, 0xde, 0xa5, 0x03, 0xf2, 0x5f, 0x39, 0x90, 0x28, 0x17, 0xf2, 0x29, 0x92, 0x0e, 0xa4, - 0xac, 0x69, 0x94, 0x7d, 0x5d, 0x90, 0x1d, 0x98, 0xd7, 0xb9, 0x39, 0xba, 0xa4, 0x26, 0x4e, 0x5e, - 0xeb, 0x4f, 0x8e, 0xb5, 0x9e, 0x22, 0xb5, 0x6e, 0x83, 0x49, 0x33, 0xf9, 0x3f, 0xcc, 0xb8, 0xc1, - 0x28, 0x11, 0x8c, 0x3b, 0x21, 0x1d, 0x32, 0x14, 0x92, 0xce, 0xaf, 0xa1, 0x3c, 0x5b, 0x74, 0xc8, - 0xc8, 0x0e, 0x34, 0x92, 0x7b, 0x41, 0xf6, 0xc7, 0xf0, 0x5a, 0x77, 0x49, 0x94, 0x3c, 0x60, 0xe7, - 0xb3, 0x5b, 0xea, 0x1f, 0x6d, 0x48, 0xee, 0x05, 0xea, 0xdb, 0xfa, 0xd1, 0x80, 0xb9, 0x63, 0x89, - 0x16, 0x05, 0x63, 0xbc, 0x8d, 0x60, 0xfa, 0xb2, 0x37, 0x54, 0x99, 0x84, 0xcf, 0xb8, 0x92, 0xdd, - 0xbf, 0x4e, 0xa8, 0xd1, 0xae, 0xcf, 0x78, 0xde, 0x34, 0x29, 0x47, 0xda, 0xac, 0x6f, 0xa6, 0x60, - 0x0e, 0xe7, 0xde, 0xe4, 0xea, 0xcd, 0x1e, 0x35, 0xc6, 0xeb, 0x3f, 0x6a, 0x32, 0xfd, 0x4e, 0xbd, - 0xb1, 0x7e, 0xdf, 0x83, 0x8a, 0x6c, 0x22, 0xa5, 0xfc, 0x73, 0x27, 0x30, 0x27, 0xdb, 0x53, 0x8f, - 0x08, 0x49, 0x22, 0xfd, 0xc2, 0xa0, 0x4e, 0xc5, 0xdf, 0x3b, 0x21, 0xc0, 0xc4, 0x70, 0x3f, 0x3e, - 0xca, 0xad, 0xef, 0x0d, 0x68, 0xaa, 0x32, 0x50, 0x2f, 0xdb, 0xd4, 0x6f, 0x59, 0x88, 0x75, 0x30, - 0xb3, 0x07, 0xc3, 0x14, 0xca, 0xb6, 0xfb, 0xea, 0x07, 0x03, 0x3e, 0xbf, 0x74, 0x3e, 0x9a, 0x66, - 0x6d, 0x80, 0xa9, 0xd5, 0x42, 0x2e, 0x43, 0x55, 0xde, 0x6e, 0xfa, 0xd2, 0xfe, 0xcb, 0xeb, 0x4d, - 0xb1, 0x6a, 0xb0, 0x7d, 0x04, 0x15, 0xe9, 0x92, 0x5b, 0x51, 0x3e, 0x75, 0x8c, 0x82, 0xe4, 0xa5, - 0x81, 0xb4, 0xa1, 0x7a, 0x40, 0x83, 0x51, 0xba, 0xbd, 0xb4, 0x27, 0x35, 0xa9, 0x08, 0x8f, 0x0c, - 0xa8, 0xe9, 0xf6, 0xbe, 0x08, 0xf5, 0x21, 0xfd, 0x3a, 0xe2, 0xce, 0x01, 0x0d, 0x54, 0x3d, 0x66, - 0x55, 0x3d, 0xaa, 0xb7, 0xa5, 0xc3, 0x36, 0xd1, 0x7f, 0x87, 0x06, 0x88, 0xf5, 0x43, 0x85, 0x9d, - 0x3a, 0x86, 0x95, 0x0e, 0xdb, 0x44, 0xbf, 0xc4, 0xb6, 0xa1, 0x1a, 0x53, 0xe1, 0xee, 0x4f, 0x6c, - 0xc7, 0xd4, 0x24, 0xb7, 0xf0, 0x28, 0x4c, 0x04, 0xae, 0xd8, 0xe2, 0x56, 0xcc, 0xac, 0x69, 0x9e, - 0x17, 0xcf, 0x43, 0xa3, 0xf0, 0xe6, 0x25, 0x75, 0xa8, 0xde, 0xf9, 0x74, 0x77, 0xc3, 0x9e, 0x2f, - 0x91, 0x06, 0xd4, 0x6e, 0x6d, 0xac, 0xdb, 0x5b, 0x1b, 0xf6, 0xbc, 0xd1, 0xbf, 0xf0, 0xf8, 0x97, - 0x4e, 0xe9, 0xf1, 0x51, 0xc7, 0x78, 0x72, 0xd4, 0x31, 0x9e, 0x1e, 0x75, 0x8c, 0x67, 0x47, 0x1d, - 0xe3, 0x87, 0xe7, 0x9d, 0xd2, 0x93, 0xe7, 0x9d, 0xd2, 0xd3, 0xe7, 0x9d, 0xd2, 0x97, 0x35, 0x55, - 0xd3, 0x3f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x29, 0x01, 0x66, 0x7b, 0x66, 0x0d, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_ebc18063440435c8) } + +var fileDescriptor_metadata_ebc18063440435c8 = []byte{ + // 1354 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xcf, 0x6f, 0x1b, 0xc5, + 0x17, 0xcf, 0xc6, 0xeb, 0x78, 0xfd, 0x9c, 0x34, 0xf1, 0xa8, 0xed, 0xd7, 0xf2, 0x57, 0xd8, 0xee, + 0x42, 0x45, 0x5a, 0x50, 0x92, 0xa6, 0x8a, 0xaa, 0x06, 0x0a, 0xc4, 0x2d, 0x48, 0xa1, 0x3f, 0x14, + 0x36, 0xa1, 0x48, 0x5c, 0x56, 0x93, 0xdd, 0xc1, 0x59, 0xba, 0xde, 0xdd, 0xce, 0x8e, 0xd3, 0xfa, + 0x8e, 0xc4, 0x01, 0x21, 0x71, 0x41, 0xe2, 0xd8, 0x4b, 0xff, 0x07, 0xfe, 0x84, 0x1e, 0x7b, 0xec, + 0x29, 0x2a, 0xe9, 0x85, 0x33, 0xc7, 0x1e, 0x10, 0x9a, 0xb7, 0x33, 0xbb, 0xeb, 0x34, 0x85, 0xb6, + 0x48, 0xdc, 0xd6, 0xef, 0x7d, 0x3e, 0xcf, 0x6f, 0xde, 0x7c, 0xde, 0x7b, 0x03, 0xa7, 0x79, 0x4c, + 0xbd, 0xbd, 0x64, 0x77, 0x79, 0xc8, 0x04, 0xf5, 0xa9, 0xa0, 0x4b, 0x09, 0x8f, 0x45, 0x4c, 0x9a, + 0x5e, 0xec, 0xdd, 0x41, 0xdf, 0x92, 0x42, 0xb4, 0xdb, 0x23, 0x11, 0x84, 0xcb, 0xa3, 0x88, 0xb3, + 0x34, 0x0e, 0xf7, 0x99, 0xef, 0x52, 0xdf, 0xe7, 0x19, 0xbc, 0xdd, 0x42, 0xdf, 0x5e, 0xe8, 0x2d, + 0x8b, 0x60, 0xc8, 0x52, 0x41, 0x87, 0x89, 0xf2, 0x9c, 0x1c, 0xc4, 0x83, 0x18, 0x3f, 0x97, 0xe5, + 0x57, 0x66, 0xb5, 0x3f, 0x06, 0xd8, 0x10, 0x82, 0x07, 0xbb, 0x23, 0xc1, 0x52, 0xf2, 0x1e, 0x54, + 0xa9, 0x10, 0x3c, 0x6d, 0x19, 0xbd, 0xca, 0x62, 0xbd, 0x7f, 0xea, 0x8f, 0x83, 0x6e, 0x73, 0x4c, + 0x87, 0xe1, 0xba, 0x8d, 0xe6, 0xf7, 0xbf, 0x09, 0xe3, 0x7b, 0xb6, 0x93, 0x61, 0xd6, 0xcd, 0x5f, + 0x1e, 0x74, 0xa7, 0xec, 0xef, 0x0d, 0x68, 0x3a, 0x2c, 0x09, 0x03, 0x8f, 0x8a, 0x20, 0x8e, 0x76, + 0x28, 0x1f, 0x30, 0x41, 0x2e, 0x40, 0x2d, 0x8a, 0x7d, 0xe6, 0x06, 0x7e, 0xcb, 0xe8, 0x19, 0x8b, + 0xd5, 0x7e, 0xeb, 0xd1, 0x41, 0x77, 0xea, 0xf0, 0xa0, 0x3b, 0x73, 0x2b, 0xf6, 0xd9, 0xe6, 0xb5, + 0xe7, 0xf9, 0x97, 0x33, 0x23, 0x81, 0x9b, 0x3e, 0x59, 0x03, 0x2b, 0x15, 0x31, 0x47, 0xce, 0x34, + 0x72, 0xda, 0x8a, 0x53, 0xdb, 0x96, 0x76, 0x24, 0xe9, 0x4f, 0xa7, 0x86, 0xd8, 0x4d, 0x7f, 0xdd, + 0x92, 0x59, 0xfc, 0xfe, 0xa0, 0x6b, 0xd8, 0x7f, 0x16, 0x99, 0x5c, 0x63, 0xa9, 0xc7, 0x83, 0x44, + 0xc4, 0xfc, 0xbf, 0xcb, 0x84, 0x5c, 0x01, 0xe0, 0xd9, 0xdf, 0x4b, 0x62, 0x05, 0x89, 0x1d, 0x45, + 0xac, 0xab, 0xc4, 0x90, 0x5a, 0xfc, 0x70, 0xea, 0x8a, 0xb1, 0xe9, 0x93, 0x55, 0x30, 0xc5, 0x38, + 0x61, 0x2d, 0xb3, 0x67, 0x2c, 0x9e, 0x58, 0xed, 0x2c, 0xbd, 0x70, 0xef, 0x4b, 0x8a, 0xb6, 0x33, + 0x4e, 0x98, 0x83, 0xd8, 0xf5, 0x59, 0x79, 0xf8, 0x5f, 0x1f, 0x74, 0x0d, 0x2c, 0xc0, 0x0f, 0x06, + 0xcc, 0xea, 0xd0, 0x3e, 0x8b, 0x84, 0x3c, 0x08, 0xa7, 0xd1, 0x20, 0x3f, 0x7c, 0xa5, 0x38, 0x88, + 0x23, 0xed, 0xd9, 0x41, 0xd4, 0xa7, 0x53, 0x43, 0xec, 0xa6, 0x4f, 0xae, 0x41, 0x4d, 0xa5, 0x85, + 0xc7, 0x6f, 0xac, 0xbe, 0xf3, 0xf2, 0x64, 0x8a, 0x4a, 0xf7, 0x4d, 0x19, 0xdb, 0xd1, 0x54, 0xfb, + 0x49, 0x05, 0xe6, 0x31, 0x74, 0xe9, 0x32, 0xde, 0x30, 0xa1, 0xb3, 0x50, 0x4f, 0x05, 0xe5, 0xc2, + 0xbd, 0xc3, 0xc6, 0x98, 0xd2, 0x6c, 0xdf, 0x7a, 0x7e, 0xd0, 0x35, 0x9d, 0xeb, 0x6c, 0xec, 0x58, + 0xe8, 0xba, 0xce, 0xc6, 0xe4, 0x0c, 0xd4, 0x58, 0xe4, 0x23, 0xa8, 0x72, 0x04, 0x34, 0xc3, 0x22, + 0x5f, 0x42, 0xbe, 0x82, 0x66, 0x10, 0x09, 0xc6, 0x23, 0x1a, 0xba, 0x2a, 0xd1, 0xb4, 0x65, 0xf6, + 0x2a, 0xaf, 0x79, 0xc8, 0x05, 0x1d, 0x44, 0x01, 0x52, 0xf2, 0x39, 0xcc, 0x47, 0xec, 0xbe, 0x70, + 0x4b, 0x0a, 0xa8, 0xa2, 0x02, 0x6c, 0x75, 0xc0, 0xb9, 0x5b, 0xec, 0xbe, 0x78, 0x89, 0x0a, 0xe6, + 0xa2, 0x92, 0xcf, 0x27, 0x1d, 0x80, 0x01, 0x8b, 0x18, 0xc7, 0x86, 0x6a, 0xcd, 0xc8, 0x3a, 0x39, + 0x25, 0x0b, 0xf9, 0x10, 0x20, 0x15, 0x81, 0x77, 0x67, 0xec, 0xee, 0x06, 0xa2, 0x55, 0xc3, 0x2b, + 0x7a, 0xab, 0x94, 0xbd, 0x1c, 0x01, 0x4b, 0x7b, 0xa1, 0xb7, 0xb4, 0xa3, 0x47, 0x80, 0x53, 0xcf, + 0x08, 0xfd, 0x40, 0x90, 0x8b, 0x70, 0xaa, 0x88, 0xe5, 0x7a, 0xf1, 0x30, 0xa1, 0x9c, 0xee, 0x86, + 0xac, 0x65, 0xf5, 0x8c, 0x45, 0xcb, 0x39, 0x59, 0x38, 0xaf, 0xe6, 0xbe, 0x23, 0x42, 0x7b, 0x68, + 0x40, 0x63, 0x8b, 0x71, 0x8f, 0x45, 0x22, 0x08, 0x59, 0x4a, 0x4e, 0x43, 0x25, 0xb9, 0xb0, 0x82, + 0x37, 0x6a, 0xa8, 0x0a, 0x49, 0x03, 0xda, 0x57, 0xd7, 0xf0, 0xc6, 0x0a, 0xfb, 0xea, 0x1a, 0xda, + 0xd7, 0x56, 0xf0, 0x92, 0x0a, 0xfb, 0x5a, 0x86, 0xbf, 0xb4, 0x86, 0x1d, 0x50, 0xd8, 0x2f, 0x65, + 0xf8, 0xcb, 0x2b, 0x58, 0xd0, 0xc2, 0x7e, 0x79, 0x85, 0xb4, 0xc0, 0x4c, 0x6e, 0xd2, 0xfb, 0x58, + 0x22, 0xed, 0x40, 0x8b, 0x9a, 0x4d, 0xcf, 0x2b, 0x30, 0x87, 0x6d, 0x7a, 0x95, 0x26, 0xd4, 0x0b, + 0xc4, 0x98, 0xf4, 0xc0, 0xf2, 0xd4, 0xb7, 0x12, 0x60, 0xc6, 0xca, 0xad, 0xc4, 0x86, 0x3a, 0xdd, + 0xa7, 0x41, 0x88, 0x25, 0x99, 0x2e, 0x41, 0x0a, 0x33, 0x39, 0x0b, 0x8d, 0x4c, 0xc6, 0x5e, 0x3c, + 0x8a, 0x84, 0x6a, 0xf5, 0x0c, 0x05, 0xe8, 0xb8, 0x2a, 0xed, 0x12, 0x16, 0x32, 0x9a, 0x6a, 0x98, + 0x59, 0x86, 0xa1, 0x23, 0x83, 0xad, 0x40, 0xf3, 0x1e, 0x0f, 0x04, 0x4b, 0xdd, 0x84, 0x71, 0x37, + 0x65, 0x5e, 0x1c, 0xf9, 0x13, 0x67, 0x9d, 0xcf, 0xdc, 0x5b, 0x8c, 0x6f, 0xa3, 0x93, 0x6c, 0x41, + 0x73, 0x77, 0xac, 0x09, 0xba, 0x55, 0x67, 0x50, 0x07, 0xc7, 0xcd, 0x8d, 0xd2, 0x55, 0xe9, 0x88, + 0x48, 0xdf, 0x62, 0x5c, 0x89, 0x8e, 0x38, 0x40, 0x4a, 0x39, 0xe8, 0x90, 0xb5, 0xd7, 0x08, 0xb9, + 0x90, 0x27, 0xa9, 0x63, 0xb6, 0xc0, 0x1c, 0xa5, 0xcc, 0x47, 0x5d, 0xe9, 0x22, 0xa2, 0x85, 0x9c, + 0x83, 0xb9, 0x30, 0x1e, 0x04, 0x1e, 0x0d, 0x5d, 0x4c, 0xa4, 0x55, 0x2f, 0x41, 0x66, 0x95, 0xab, + 0x2f, 0x3d, 0x64, 0x15, 0xc8, 0xdd, 0x11, 0xe3, 0xc1, 0x64, 0x75, 0xa0, 0x54, 0x9d, 0x05, 0xe5, + 0xcf, 0xcb, 0xa3, 0x2e, 0xff, 0xa9, 0x09, 0x27, 0xe4, 0x60, 0xff, 0x77, 0xbb, 0xe0, 0x23, 0xa8, + 0xc9, 0xed, 0xca, 0xd2, 0x54, 0xcd, 0xc2, 0xce, 0xd1, 0x46, 0xfb, 0x32, 0xdf, 0xc3, 0x1b, 0xbe, + 0x9f, 0x4f, 0x41, 0x45, 0x22, 0x97, 0xf5, 0x46, 0xad, 0xbc, 0xd0, 0xa6, 0xba, 0x96, 0xc5, 0xfe, + 0x55, 0xe4, 0x8c, 0x41, 0xae, 0x80, 0x15, 0xc6, 0x1e, 0x0d, 0xa5, 0x56, 0x4d, 0x64, 0xff, 0xff, + 0x18, 0xf6, 0x0d, 0x05, 0xd1, 0x42, 0xd6, 0x14, 0xf2, 0x19, 0xcc, 0x6d, 0x33, 0xbe, 0xcf, 0xf8, + 0x6d, 0xc6, 0x53, 0x39, 0x48, 0xaa, 0x18, 0xa3, 0x7d, 0x4c, 0x0c, 0x85, 0x50, 0x21, 0x26, 0x69, + 0xe4, 0x0c, 0xd4, 0x77, 0x47, 0x41, 0xe8, 0xbb, 0x82, 0x0e, 0x50, 0x64, 0x75, 0xfd, 0x57, 0x68, + 0xde, 0xa1, 0x03, 0xf2, 0xb6, 0x1c, 0x48, 0x94, 0x0b, 0xf9, 0x14, 0xc9, 0x06, 0x52, 0xde, 0x34, + 0xca, 0xbe, 0x21, 0xc8, 0x36, 0x2c, 0xe8, 0xdc, 0x5c, 0x5d, 0x52, 0x0b, 0x27, 0xaf, 0xfd, 0x37, + 0xc7, 0xda, 0xc8, 0x90, 0x5a, 0xb7, 0xe1, 0xa4, 0x99, 0xbc, 0x0b, 0xb3, 0x5e, 0x38, 0x4a, 0x05, + 0xe3, 0x6e, 0x44, 0x87, 0x0c, 0x85, 0xa4, 0xf3, 0x6b, 0x28, 0xcf, 0x2d, 0x3a, 0x64, 0x64, 0x1b, + 0x1a, 0xe9, 0xdd, 0x30, 0xff, 0x63, 0x78, 0xa5, 0xbb, 0x24, 0x4a, 0x1e, 0xb0, 0xfd, 0xc5, 0x0d, + 0xf5, 0x8f, 0x0e, 0xa4, 0x77, 0x43, 0xf5, 0x6d, 0xff, 0x6c, 0xc0, 0xfc, 0x91, 0x44, 0xcb, 0x82, + 0x31, 0xde, 0x44, 0x30, 0x7d, 0xd9, 0x1b, 0xaa, 0x4c, 0x22, 0x60, 0x5c, 0xc9, 0xee, 0x7f, 0xc7, + 0xd4, 0x68, 0x27, 0x60, 0xbc, 0x68, 0x9a, 0x8c, 0x23, 0x6d, 0xf6, 0x77, 0xd3, 0x30, 0x8f, 0x73, + 0x6f, 0x72, 0xf5, 0xe6, 0x8f, 0x1a, 0xe3, 0xd5, 0x1f, 0x35, 0xb9, 0x7e, 0xa7, 0x5f, 0x5b, 0xbf, + 0x1f, 0x80, 0x29, 0x9b, 0x48, 0x29, 0xff, 0xcc, 0x31, 0xcc, 0xc9, 0xf6, 0xd4, 0x23, 0x42, 0x92, + 0x48, 0xbf, 0x34, 0xa8, 0x33, 0xf1, 0xf7, 0x8e, 0x09, 0x30, 0x31, 0xdc, 0x8f, 0x8e, 0x72, 0xfb, + 0x47, 0x03, 0x9a, 0xaa, 0x0c, 0xd4, 0xcf, 0x37, 0xf5, 0x1b, 0x16, 0x62, 0x03, 0xac, 0xfc, 0xc1, + 0x30, 0x8d, 0xb2, 0xed, 0xbe, 0xfc, 0xc1, 0x80, 0xcf, 0x2f, 0x9d, 0x8f, 0xa6, 0xd9, 0x9f, 0x82, + 0xa5, 0xd5, 0x42, 0x2e, 0x42, 0x55, 0xde, 0x6e, 0xf6, 0xd2, 0xfe, 0xc7, 0xeb, 0xcd, 0xb0, 0x6a, + 0xb0, 0x7d, 0x02, 0xa6, 0x74, 0xc9, 0xad, 0x28, 0x9f, 0x3a, 0x46, 0x49, 0xf2, 0xd2, 0x40, 0xda, + 0x50, 0xdd, 0xa7, 0xe1, 0x28, 0xdb, 0x5e, 0xda, 0x93, 0x99, 0x54, 0x84, 0x87, 0x06, 0xd4, 0x74, + 0x7b, 0x9f, 0x87, 0xfa, 0x90, 0x7e, 0x1b, 0x73, 0x77, 0x9f, 0x86, 0xaa, 0x1e, 0x73, 0xaa, 0x1e, + 0xd5, 0x9b, 0xd2, 0xe1, 0x58, 0xe8, 0xbf, 0x4d, 0x43, 0xc4, 0x06, 0x91, 0xc2, 0x4e, 0x1f, 0xc1, + 0x4a, 0x87, 0x63, 0xa1, 0x5f, 0x62, 0xdb, 0x50, 0x4d, 0xa8, 0xf0, 0xf6, 0x26, 0xb6, 0x63, 0x66, + 0x92, 0x5b, 0x78, 0x14, 0xa5, 0x02, 0x57, 0x6c, 0x79, 0x2b, 0xe6, 0xd6, 0x2c, 0xcf, 0xf3, 0x67, + 0xa1, 0x51, 0x7a, 0xf3, 0x92, 0x3a, 0x54, 0x6f, 0xc7, 0x82, 0xf1, 0x85, 0x29, 0xd2, 0x80, 0xda, + 0x0d, 0x46, 0x79, 0xc4, 0xf8, 0x82, 0xd1, 0x3f, 0xf7, 0xe8, 0xb7, 0xce, 0xd4, 0xa3, 0xc3, 0x8e, + 0xf1, 0xf8, 0xb0, 0x63, 0x3c, 0x39, 0xec, 0x18, 0x4f, 0x0f, 0x3b, 0xc6, 0x4f, 0xcf, 0x3a, 0x53, + 0x8f, 0x9f, 0x75, 0xa6, 0x9e, 0x3c, 0xeb, 0x4c, 0x7d, 0x5d, 0x53, 0x35, 0xfd, 0x2b, 0x00, 0x00, + 0xff, 0xff, 0xf3, 0xb6, 0xef, 0xe2, 0x66, 0x0d, 0x00, 0x00, } diff --git a/pkg/roachpb/metadata.proto b/pkg/roachpb/metadata.proto index c55162e80bcd..cf5f8d2dad00 100644 --- a/pkg/roachpb/metadata.proto +++ b/pkg/roachpb/metadata.proto @@ -37,17 +37,17 @@ message ReplicationTarget { // ReplicaType identifies which raft activities a replica participates in. enum ReplicaType { - // ReplicaType_VOTER indicates a replica that participates in all raft + // ReplicaType_Voter indicates a replica that participates in all raft // activities, including voting for leadership and committing entries. // Notably, voters are considered for quorum size of the raft group. - VOTER = 0; - // ReplicaType_LEARNER indicates a replica that applies committed entries, but + Voter = 0; + // ReplicaType_Learner indicates a replica that applies committed entries, but // does not count towards the quorum. Learners do not vote for leadership nor // do their acknowledged log entries get taken into account for determining // the committed index. At the time of writing, learners in CockroachDB are a // short-term transient state: a replica being added and on its way to being a // VOTER. - LEARNER = 1; + Learner = 1; } // ReplicaDescriptor describes a replica location by node ID diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index bdcfb22798b0..e7924970010e 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -15,17 +15,17 @@ import ( "strings" ) -// ReplicaTypeVoter returns a ReplicaType_VOTER pointer suitable for use in a +// ReplicaTypeVoter returns a ReplicaType_Voter pointer suitable for use in a // nullable proto field. func ReplicaTypeVoter() *ReplicaType { - t := ReplicaType_VOTER + t := ReplicaType_Voter return &t } -// ReplicaTypeLearner returns a ReplicaType_LEARNER pointer suitable for use in +// ReplicaTypeLearner returns a ReplicaType_Learner pointer suitable for use in // a nullable proto field. func ReplicaTypeLearner() *ReplicaType { - t := ReplicaType_LEARNER + t := ReplicaType_Learner return &t } @@ -77,7 +77,7 @@ func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { // save the alloc. fastpath := true for i := range d.wrapped { - if d.wrapped[i].GetType() != ReplicaType_VOTER { + if d.wrapped[i].GetType() != ReplicaType_Voter { fastpath = false break } @@ -87,7 +87,7 @@ func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { } voters := make([]ReplicaDescriptor, 0, len(d.wrapped)) for i := range d.wrapped { - if d.wrapped[i].GetType() == ReplicaType_VOTER { + if d.wrapped[i].GetType() == ReplicaType_Voter { voters = append(voters, d.wrapped[i]) } } @@ -185,7 +185,7 @@ func (d ReplicaDescriptors) Learners() []ReplicaDescriptor { // save the alloc. var learners []ReplicaDescriptor for i := range d.wrapped { - if d.wrapped[i].GetType() == ReplicaType_LEARNER { + if d.wrapped[i].GetType() == ReplicaType_Learner { if learners == nil { learners = make([]ReplicaDescriptor, 0, len(d.wrapped)-i) } @@ -243,7 +243,7 @@ func (d *ReplicaDescriptors) RemoveReplica( func (d ReplicaDescriptors) QuorumSize() int { var numVoters int for i := range d.wrapped { - if d.wrapped[i].GetType() == ReplicaType_VOTER { + if d.wrapped[i].GetType() == ReplicaType_Voter { numVoters++ } } diff --git a/pkg/roachpb/metadata_replicas_test.go b/pkg/roachpb/metadata_replicas_test.go index dd72323b4e0b..d298c13143e4 100644 --- a/pkg/roachpb/metadata_replicas_test.go +++ b/pkg/roachpb/metadata_replicas_test.go @@ -32,10 +32,10 @@ func TestVotersLearnersAll(t *testing.T) { for i, test := range tests { r := MakeReplicaDescriptors(test) for _, voter := range r.Voters() { - assert.Equal(t, ReplicaType_VOTER, voter.GetType(), "testcase %d", i) + assert.Equal(t, ReplicaType_Voter, voter.GetType(), "testcase %d", i) } for _, learner := range r.Learners() { - assert.Equal(t, ReplicaType_LEARNER, learner.GetType(), "testcase %d", i) + assert.Equal(t, ReplicaType_Learner, learner.GetType(), "testcase %d", i) } assert.Equal(t, len(test), len(r.All()), "testcase %d", i) } @@ -86,10 +86,10 @@ func TestReplicaDescriptorsRemove(t *testing.T) { assert.Equal(t, lenBefore, len(r.All()), "testcase %d", i) } for _, voter := range r.Voters() { - assert.Equal(t, ReplicaType_VOTER, voter.GetType(), "testcase %d", i) + assert.Equal(t, ReplicaType_Voter, voter.GetType(), "testcase %d", i) } for _, learner := range r.Learners() { - assert.Equal(t, ReplicaType_LEARNER, learner.GetType(), "testcase %d", i) + assert.Equal(t, ReplicaType_Learner, learner.GetType(), "testcase %d", i) } } } diff --git a/pkg/storage/allocator_test.go b/pkg/storage/allocator_test.go index eb1efd9392f5..388b63b08e5f 100644 --- a/pkg/storage/allocator_test.go +++ b/pkg/storage/allocator_test.go @@ -4784,7 +4784,7 @@ func TestAllocatorRemoveLearner(t *testing.T) { zone := config.ZoneConfig{ NumReplicas: proto.Int32(3), } - learnerType := roachpb.ReplicaType_LEARNER + learnerType := roachpb.ReplicaType_Learner rangeWithLearnerDesc := roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { diff --git a/pkg/storage/batcheval/cmd_lease.go b/pkg/storage/batcheval/cmd_lease.go index dbe5f557570d..8c02d4c6d46f 100644 --- a/pkg/storage/batcheval/cmd_lease.go +++ b/pkg/storage/batcheval/cmd_lease.go @@ -47,7 +47,7 @@ func checkCanReceiveLease(rec EvalContext) error { if !ok { return errors.AssertionFailedf( `could not find replica for store %s in %s`, rec.StoreID(), rec.Desc()) - } else if t := repDesc.GetType(); t != roachpb.ReplicaType_VOTER { + } else if t := repDesc.GetType(); t != roachpb.ReplicaType_Voter { return errors.Errorf(`cannot transfer lease to replica of type %s`, t) } return nil diff --git a/pkg/storage/batcheval/cmd_lease_test.go b/pkg/storage/batcheval/cmd_lease_test.go index 7fe36eb2ae98..e1a0f274f354 100644 --- a/pkg/storage/batcheval/cmd_lease_test.go +++ b/pkg/storage/batcheval/cmd_lease_test.go @@ -129,9 +129,9 @@ func TestLeaseCommandLearnerReplica(t *testing.T) { // Learners are not allowed to become leaseholders for now, see the comments // in TransferLease and RequestLease. _, err := TransferLease(ctx, nil, cArgs, nil) - require.EqualError(t, err, `cannot transfer lease to replica of type LEARNER`) + require.EqualError(t, err, `cannot transfer lease to replica of type Learner`) cArgs.Args = &roachpb.RequestLeaseRequest{} _, err = RequestLease(ctx, nil, cArgs, nil) - require.EqualError(t, err, `cannot transfer lease to replica of type LEARNER`) + require.EqualError(t, err, `cannot transfer lease to replica of type Learner`) } diff --git a/pkg/storage/client_test.go b/pkg/storage/client_test.go index 52e173d77a56..e4782757d887 100644 --- a/pkg/storage/client_test.go +++ b/pkg/storage/client_test.go @@ -1204,7 +1204,7 @@ func (m *multiTestContext) replicateRangeNonFatal(rangeID roachpb.RangeID, dests if e := expectedReplicaIDs[i]; repDesc.ReplicaID != e { return errors.Errorf("expected replica %s to have ID %d", repl, e) } - if t := repDesc.GetType(); t != roachpb.ReplicaType_VOTER { + if t := repDesc.GetType(); t != roachpb.ReplicaType_Voter { return errors.Errorf("expected replica %s to be a voter was %s", repl, t) } if !repl.Desc().ContainsKey(startKey) { diff --git a/pkg/storage/merge_queue.go b/pkg/storage/merge_queue.go index c9b0c5e7d3fd..8a55aeb57749 100644 --- a/pkg/storage/merge_queue.go +++ b/pkg/storage/merge_queue.go @@ -295,12 +295,12 @@ func (mq *mergeQueue) process( // Defensive sanity check that everything is now a voter. for i := range lhsReplicas { - if lhsReplicas[i].GetType() != roachpb.ReplicaType_VOTER { + if lhsReplicas[i].GetType() != roachpb.ReplicaType_Voter { return errors.Errorf(`cannot merge non-voter replicas on lhs: %v`, lhsReplicas) } } for i := range rhsReplicas { - if rhsReplicas[i].GetType() != roachpb.ReplicaType_VOTER { + if rhsReplicas[i].GetType() != roachpb.ReplicaType_Voter { return errors.Errorf(`cannot merge non-voter replicas on rhs: %v`, rhsReplicas) } } diff --git a/pkg/storage/raft_snapshot_queue.go b/pkg/storage/raft_snapshot_queue.go index f4dc213962a2..79d9b6facef7 100644 --- a/pkg/storage/raft_snapshot_queue.go +++ b/pkg/storage/raft_snapshot_queue.go @@ -111,7 +111,7 @@ func (rq *raftSnapshotQueue) processRaftSnapshot( // that's adding it or it's been orphaned and it's about to be cleaned up by // the replicate queue. Either way, no point in also sending it a snapshot of // type RAFT. - if repDesc.GetType() == roachpb.ReplicaType_LEARNER { + if repDesc.GetType() == roachpb.ReplicaType_Learner { if index := repl.getAndGCSnapshotLogTruncationConstraints(timeutil.Now()); index > 0 { // There is a snapshot being transferred. It's probably a LEARNER snap, so // bail for now and try again later. diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index 98f50fae9ae0..a4ff71639378 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -988,7 +988,7 @@ func validateReplicationChanges( // trying to add it with the learner+snapshot+voter cycle and got // interrupted or else we hit a race between the replicate queue and // AdminChangeReplicas. - if rDesc.GetType() == roachpb.ReplicaType_LEARNER { + if rDesc.GetType() == roachpb.ReplicaType_Learner { return errors.Errorf( "unable to add replica %v which is already present as a learner in %s", chg.Target, desc) } @@ -1068,7 +1068,7 @@ func (r *Replica) finalizeChangeReplicas( return nil, errors.Errorf("programming error: replica %v not found in %v", target, updatedDesc) } - if rDesc.GetType() != roachpb.ReplicaType_LEARNER { + if rDesc.GetType() != roachpb.ReplicaType_Learner { return nil, errors.Errorf("programming error: cannot promote replica of type %s", rDesc.Type) } @@ -1132,7 +1132,7 @@ func (r *Replica) tryRollBackLearnerReplica( newDesc := *desc newDesc.SetReplicas(desc.Replicas().DeepCopy()) replDesc, ok := newDesc.RemoveReplica(target.NodeID, target.StoreID) - if !ok || replDesc.GetType() != roachpb.ReplicaType_LEARNER { + if !ok || replDesc.GetType() != roachpb.ReplicaType_Learner { // There's no learner to roll back. log.Event(ctx, "learner to roll back not found; skipping") return diff --git a/pkg/storage/replica_follower_read.go b/pkg/storage/replica_follower_read.go index 1a2faa867edc..650952e0d247 100644 --- a/pkg/storage/replica_follower_read.go +++ b/pkg/storage/replica_follower_read.go @@ -45,7 +45,7 @@ func (r *Replica) canServeFollowerRead( if err != nil { return roachpb.NewError(err) } - if repDesc.GetType() == roachpb.ReplicaType_LEARNER { + if repDesc.GetType() == roachpb.ReplicaType_Learner { log.Event(ctx, "learner replicas cannot serve follower reads") return pErr } diff --git a/pkg/storage/replica_learner_test.go b/pkg/storage/replica_learner_test.go index 7a19a5bb97c5..d3b5d4eea588 100644 --- a/pkg/storage/replica_learner_test.go +++ b/pkg/storage/replica_learner_test.go @@ -592,8 +592,8 @@ func TestLearnerNoAcceptLease(t *testing.T) { desc := tc.LookupRangeOrFatal(t, scratchStartKey) err := tc.TransferRangeLease(desc, tc.Target(1)) - if !testutils.IsError(err, `cannot transfer lease to replica of type LEARNER`) { - t.Fatalf(`expected "cannot transfer lease to replica of type LEARNER" error got: %+v`, err) + if !testutils.IsError(err, `cannot transfer lease to replica of type Learner`) { + t.Fatalf(`expected "cannot transfer lease to replica of type Learner" error got: %+v`, err) } } diff --git a/pkg/storage/replica_proposal_buf.go b/pkg/storage/replica_proposal_buf.go index 6b82c166bfb3..f14e7746e676 100644 --- a/pkg/storage/replica_proposal_buf.go +++ b/pkg/storage/replica_proposal_buf.go @@ -456,9 +456,9 @@ func (b *propBuf) FlushLockedWithRaftGroup(raftGroup *raft.RawNode) error { replicaID = added[0].ReplicaID typ := added[0].GetType() switch typ { - case roachpb.ReplicaType_VOTER: + case roachpb.ReplicaType_Voter: changeType = raftpb.ConfChangeAddNode - case roachpb.ReplicaType_LEARNER: + case roachpb.ReplicaType_Learner: changeType = raftpb.ConfChangeAddLearnerNode default: panic(errors.Errorf("unknown replica type %v", typ)) diff --git a/pkg/storage/replica_range_lease.go b/pkg/storage/replica_range_lease.go index 64f4954c666c..a7a6c23712a4 100644 --- a/pkg/storage/replica_range_lease.go +++ b/pkg/storage/replica_range_lease.go @@ -683,7 +683,7 @@ func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID // `r.mu.minLeaseProposedTS = status.Timestamp` line below will likely play // badly with that. This would be an issue even without learners, but // omitting this check would make it worse. Fixme. - if t := nextLeaseHolder.GetType(); t != roachpb.ReplicaType_VOTER { + if t := nextLeaseHolder.GetType(); t != roachpb.ReplicaType_Voter { return nil, nil, errors.Errorf(`cannot transfer lease to replica of type %s`, t) } From 8a485c7f07f919a1aaf8d5a0e7f9a1628ad71c74 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 14:22:14 +0200 Subject: [PATCH 07/23] roachpb: add ReplicaType_Voter{Incoming,Outgoing} These are required for atomic replication changes to describe joint configurations, i.e. configurations consisting of two sets of replica which both need to reach quorum to make replication decisions. An audit of existing consumers of this enum will follow. Release note: None --- c-deps/libroach/protos/roachpb/metadata.pb.cc | 2 + c-deps/libroach/protos/roachpb/metadata.pb.h | 8 +- pkg/roachpb/data_test.go | 4 +- pkg/roachpb/metadata.go | 4 +- pkg/roachpb/metadata.pb.go | 266 ++++++++++-------- pkg/roachpb/metadata.proto | 45 ++- pkg/roachpb/metadata_replicas.go | 12 +- pkg/roachpb/metadata_replicas_test.go | 6 +- pkg/storage/batcheval/cmd_lease.go | 2 +- pkg/storage/batcheval/cmd_lease_test.go | 2 +- pkg/storage/below_raft_protos_test.go | 2 +- pkg/storage/client_test.go | 2 +- pkg/storage/merge_queue.go | 4 +- pkg/storage/replica_command.go | 2 +- pkg/storage/replica_proposal_buf.go | 2 +- pkg/storage/replica_range_lease.go | 2 +- 16 files changed, 214 insertions(+), 151 deletions(-) diff --git a/c-deps/libroach/protos/roachpb/metadata.pb.cc b/c-deps/libroach/protos/roachpb/metadata.pb.cc index 526e976865dc..dba2911821c7 100644 --- a/c-deps/libroach/protos/roachpb/metadata.pb.cc +++ b/c-deps/libroach/protos/roachpb/metadata.pb.cc @@ -345,6 +345,8 @@ bool ReplicaType_IsValid(int value) { switch (value) { case 0: case 1: + case 2: + case 3: return true; default: return false; diff --git a/c-deps/libroach/protos/roachpb/metadata.pb.h b/c-deps/libroach/protos/roachpb/metadata.pb.h index 2524c3f336df..9ce1ecffefae 100644 --- a/c-deps/libroach/protos/roachpb/metadata.pb.h +++ b/c-deps/libroach/protos/roachpb/metadata.pb.h @@ -114,12 +114,14 @@ namespace cockroach { namespace roachpb { enum ReplicaType { - Voter = 0, + VoterFull = 0, + VoterIncoming = 2, + VoterOutgoing = 3, Learner = 1 }; bool ReplicaType_IsValid(int value); -const ReplicaType ReplicaType_MIN = Voter; -const ReplicaType ReplicaType_MAX = Learner; +const ReplicaType ReplicaType_MIN = VoterFull; +const ReplicaType ReplicaType_MAX = VoterOutgoing; const int ReplicaType_ARRAYSIZE = ReplicaType_MAX + 1; // =================================================================== diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 11d550db0494..9e251d32f772 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -900,7 +900,7 @@ func TestLeaseEquivalence(t *testing.T) { stasis2 := Lease{Replica: r1, Start: ts1, Epoch: 1, DeprecatedStartStasis: ts2.Clone()} r1Voter, r1Learner := r1, r1 - r1Voter.Type = ReplicaTypeVoter() + r1Voter.Type = ReplicaTypeVoterFull() r1Learner.Type = ReplicaTypeLearner() epoch1Voter := Lease{Replica: r1Voter, Start: ts1, Epoch: 1} epoch1Learner := Lease{Replica: r1Learner, Start: ts1, Epoch: 1} @@ -1636,7 +1636,7 @@ func TestChangeReplicasTrigger_String(t *testing.T) { defer leaktest.AfterTest(t)() l := ReplicaType_Learner - v := ReplicaType_Voter + v := ReplicaType_VoterFull repl1 := ReplicaDescriptor{NodeID: 1, StoreID: 2, ReplicaID: 3, Type: &l} repl2 := ReplicaDescriptor{NodeID: 4, StoreID: 5, ReplicaID: 6, Type: &v} crt := ChangeReplicasTrigger{ diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index 6eceb5613149..fc6248d62ed7 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -285,7 +285,7 @@ func (r ReplicaDescriptor) String() string { } else { fmt.Fprintf(&buf, "%d", r.ReplicaID) } - if typ := r.GetType(); typ != ReplicaType_Voter { + if typ := r.GetType(); typ != ReplicaType_VoterFull { buf.WriteString(strings.ToUpper(typ.String())) } return buf.String() @@ -308,7 +308,7 @@ func (r ReplicaDescriptor) Validate() error { // GetType returns the type of this ReplicaDescriptor. func (r ReplicaDescriptor) GetType() ReplicaType { if r.Type == nil { - return ReplicaType_Voter + return ReplicaType_VoterFull } return *r.Type } diff --git a/pkg/roachpb/metadata.pb.go b/pkg/roachpb/metadata.pb.go index c4974dfc544c..4758722931ed 100644 --- a/pkg/roachpb/metadata.pb.go +++ b/pkg/roachpb/metadata.pb.go @@ -27,29 +27,60 @@ var _ = math.Inf const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // ReplicaType identifies which raft activities a replica participates in. +// In normal operation, Voter and Learner are the only used states. However, +// atomic replication changes require a transition through a "joint config"; +// in this joint config, the VoterOutgoing and VoterIncoming types are used +// as well to denote voters which are being removed and newly added by the +// change, respectively. +// +// All voter types indicate a replica that participates in all raft activities, +// including voting for leadership and committing entries. Typically, this +// requires a majority of voters to reach a decision. In the joint config, +// two separate majorities are required: one from the set of replicas that +// have either type Voter or VoterOutgoing, as well as that of the set of +// types Voter and VoterIncoming. For example, when type Voter is assigned +// to replicas 1 and 2, while 3 is VoterOutgoing and 4 is VoterIncoming, then +// the two sets over which quorums need to be achieved are {1,2,3} and {1,2,4}. +// Thus, {1,2} is a quorum of both, {1,3} is a quorum of the first but not the +// second, {1,4} is a quorum of the second but not the first, and {3,4} is a +// quorum of neither. type ReplicaType int32 const ( - // ReplicaType_Voter indicates a replica that participates in all raft - // activities, including voting for leadership and committing entries. - // Notably, voters are considered for quorum size of the raft group. - ReplicaType_Voter ReplicaType = 0 + // ReplicaType_VoterFull indicates a replica that is a voter both in the + // incoming and outgoing set. + ReplicaType_VoterFull ReplicaType = 0 + // ReplicaType_VoterIncoming indicates a voting replica that will be a + // VoterFull once the ongoing atomic replication change is finalized; that is, + // it is in the process of being added. In practice, this replica type should + // be treated like a VoterFull. + ReplicaType_VoterIncoming ReplicaType = 2 + // ReplicaType_VoterOutgoing indicates a voting replica that will not be part + // of the descriptor once the ongoing atomic replication change is finalized; + // that is, it is in the process of being removed. In practice, a replica of + // this type should be treated accordingly and no work should be assigned to + // it. + ReplicaType_VoterOutgoing ReplicaType = 3 // ReplicaType_Learner indicates a replica that applies committed entries, but - // does not count towards the quorum. Learners do not vote for leadership nor - // do their acknowledged log entries get taken into account for determining - // the committed index. At the time of writing, learners in CockroachDB are a - // short-term transient state: a replica being added and on its way to being a - // VOTER. + // does not count towards the quorum(s). Learners do not vote for leadership + // nor do their acknowledged log entries get taken into account for + // determining the committed index. At the time of writing, learners in + // CockroachDB are a short-term transient state: a replica being added and on + // its way to being a VOTER. ReplicaType_Learner ReplicaType = 1 ) var ReplicaType_name = map[int32]string{ - 0: "Voter", + 0: "VoterFull", + 2: "VoterIncoming", + 3: "VoterOutgoing", 1: "Learner", } var ReplicaType_value = map[string]int32{ - "Voter": 0, - "Learner": 1, + "VoterFull": 0, + "VoterIncoming": 2, + "VoterOutgoing": 3, + "Learner": 1, } func (x ReplicaType) Enum() *ReplicaType { @@ -69,7 +100,7 @@ func (x *ReplicaType) UnmarshalJSON(data []byte) error { return nil } func (ReplicaType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{0} + return fileDescriptor_metadata_e31f96670cc94504, []int{0} } // Attributes specifies a list of arbitrary strings describing @@ -81,7 +112,7 @@ type Attributes struct { func (m *Attributes) Reset() { *m = Attributes{} } func (*Attributes) ProtoMessage() {} func (*Attributes) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{0} + return fileDescriptor_metadata_e31f96670cc94504, []int{0} } func (m *Attributes) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -115,7 +146,7 @@ type ReplicationTarget struct { func (m *ReplicationTarget) Reset() { *m = ReplicationTarget{} } func (*ReplicationTarget) ProtoMessage() {} func (*ReplicationTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{1} + return fileDescriptor_metadata_e31f96670cc94504, []int{1} } func (m *ReplicationTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -160,7 +191,7 @@ type ReplicaDescriptor struct { func (m *ReplicaDescriptor) Reset() { *m = ReplicaDescriptor{} } func (*ReplicaDescriptor) ProtoMessage() {} func (*ReplicaDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{2} + return fileDescriptor_metadata_e31f96670cc94504, []int{2} } func (m *ReplicaDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -195,7 +226,7 @@ func (m *ReplicaIdent) Reset() { *m = ReplicaIdent{} } func (m *ReplicaIdent) String() string { return proto.CompactTextString(m) } func (*ReplicaIdent) ProtoMessage() {} func (*ReplicaIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{3} + return fileDescriptor_metadata_e31f96670cc94504, []int{3} } func (m *ReplicaIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -352,7 +383,7 @@ type RangeDescriptor struct { func (m *RangeDescriptor) Reset() { *m = RangeDescriptor{} } func (*RangeDescriptor) ProtoMessage() {} func (*RangeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{4} + return fileDescriptor_metadata_e31f96670cc94504, []int{4} } func (m *RangeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -391,7 +422,7 @@ type Percentiles struct { func (m *Percentiles) Reset() { *m = Percentiles{} } func (*Percentiles) ProtoMessage() {} func (*Percentiles) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{5} + return fileDescriptor_metadata_e31f96670cc94504, []int{5} } func (m *Percentiles) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -453,7 +484,7 @@ type StoreCapacity struct { func (m *StoreCapacity) Reset() { *m = StoreCapacity{} } func (*StoreCapacity) ProtoMessage() {} func (*StoreCapacity) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{6} + return fileDescriptor_metadata_e31f96670cc94504, []int{6} } func (m *StoreCapacity) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -498,7 +529,7 @@ func (m *NodeDescriptor) Reset() { *m = NodeDescriptor{} } func (m *NodeDescriptor) String() string { return proto.CompactTextString(m) } func (*NodeDescriptor) ProtoMessage() {} func (*NodeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{7} + return fileDescriptor_metadata_e31f96670cc94504, []int{7} } func (m *NodeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -534,7 +565,7 @@ func (m *LocalityAddress) Reset() { *m = LocalityAddress{} } func (m *LocalityAddress) String() string { return proto.CompactTextString(m) } func (*LocalityAddress) ProtoMessage() {} func (*LocalityAddress) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{8} + return fileDescriptor_metadata_e31f96670cc94504, []int{8} } func (m *LocalityAddress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -572,7 +603,7 @@ func (m *StoreDescriptor) Reset() { *m = StoreDescriptor{} } func (m *StoreDescriptor) String() string { return proto.CompactTextString(m) } func (*StoreDescriptor) ProtoMessage() {} func (*StoreDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{9} + return fileDescriptor_metadata_e31f96670cc94504, []int{9} } func (m *StoreDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -609,7 +640,7 @@ func (m *StoreDeadReplicas) Reset() { *m = StoreDeadReplicas{} } func (m *StoreDeadReplicas) String() string { return proto.CompactTextString(m) } func (*StoreDeadReplicas) ProtoMessage() {} func (*StoreDeadReplicas) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{10} + return fileDescriptor_metadata_e31f96670cc94504, []int{10} } func (m *StoreDeadReplicas) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -643,7 +674,7 @@ type Locality struct { func (m *Locality) Reset() { *m = Locality{} } func (*Locality) ProtoMessage() {} func (*Locality) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{11} + return fileDescriptor_metadata_e31f96670cc94504, []int{11} } func (m *Locality) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -679,7 +710,7 @@ type Tier struct { func (m *Tier) Reset() { *m = Tier{} } func (*Tier) ProtoMessage() {} func (*Tier) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{12} + return fileDescriptor_metadata_e31f96670cc94504, []int{12} } func (m *Tier) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +751,7 @@ type Version struct { func (m *Version) Reset() { *m = Version{} } func (*Version) ProtoMessage() {} func (*Version) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_ebc18063440435c8, []int{13} + return fileDescriptor_metadata_e31f96670cc94504, []int{13} } func (m *Version) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1489,7 +1520,7 @@ func NewPopulatedReplicaDescriptor(r randyMetadata, easy bool) *ReplicaDescripto this.ReplicaID *= -1 } if r.Intn(10) != 0 { - v1 := ReplicaType([]int32{0, 1}[r.Intn(2)]) + v1 := ReplicaType([]int32{0, 2, 3, 1}[r.Intn(4)]) this.Type = &v1 } if !easy && r.Intn(10) != 0 { @@ -4000,93 +4031,94 @@ var ( ErrIntOverflowMetadata = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_ebc18063440435c8) } - -var fileDescriptor_metadata_ebc18063440435c8 = []byte{ - // 1354 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xcf, 0x6f, 0x1b, 0xc5, - 0x17, 0xcf, 0xc6, 0xeb, 0x78, 0xfd, 0x9c, 0x34, 0xf1, 0xa8, 0xed, 0xd7, 0xf2, 0x57, 0xd8, 0xee, - 0x42, 0x45, 0x5a, 0x50, 0x92, 0xa6, 0x8a, 0xaa, 0x06, 0x0a, 0xc4, 0x2d, 0x48, 0xa1, 0x3f, 0x14, - 0x36, 0xa1, 0x48, 0x5c, 0x56, 0x93, 0xdd, 0xc1, 0x59, 0xba, 0xde, 0xdd, 0xce, 0x8e, 0xd3, 0xfa, - 0x8e, 0xc4, 0x01, 0x21, 0x71, 0x41, 0xe2, 0xd8, 0x4b, 0xff, 0x07, 0xfe, 0x84, 0x1e, 0x7b, 0xec, - 0x29, 0x2a, 0xe9, 0x85, 0x33, 0xc7, 0x1e, 0x10, 0x9a, 0xb7, 0x33, 0xbb, 0xeb, 0x34, 0x85, 0xb6, - 0x48, 0xdc, 0xd6, 0xef, 0x7d, 0x3e, 0xcf, 0x6f, 0xde, 0x7c, 0xde, 0x7b, 0x03, 0xa7, 0x79, 0x4c, - 0xbd, 0xbd, 0x64, 0x77, 0x79, 0xc8, 0x04, 0xf5, 0xa9, 0xa0, 0x4b, 0x09, 0x8f, 0x45, 0x4c, 0x9a, - 0x5e, 0xec, 0xdd, 0x41, 0xdf, 0x92, 0x42, 0xb4, 0xdb, 0x23, 0x11, 0x84, 0xcb, 0xa3, 0x88, 0xb3, - 0x34, 0x0e, 0xf7, 0x99, 0xef, 0x52, 0xdf, 0xe7, 0x19, 0xbc, 0xdd, 0x42, 0xdf, 0x5e, 0xe8, 0x2d, - 0x8b, 0x60, 0xc8, 0x52, 0x41, 0x87, 0x89, 0xf2, 0x9c, 0x1c, 0xc4, 0x83, 0x18, 0x3f, 0x97, 0xe5, - 0x57, 0x66, 0xb5, 0x3f, 0x06, 0xd8, 0x10, 0x82, 0x07, 0xbb, 0x23, 0xc1, 0x52, 0xf2, 0x1e, 0x54, - 0xa9, 0x10, 0x3c, 0x6d, 0x19, 0xbd, 0xca, 0x62, 0xbd, 0x7f, 0xea, 0x8f, 0x83, 0x6e, 0x73, 0x4c, - 0x87, 0xe1, 0xba, 0x8d, 0xe6, 0xf7, 0xbf, 0x09, 0xe3, 0x7b, 0xb6, 0x93, 0x61, 0xd6, 0xcd, 0x5f, - 0x1e, 0x74, 0xa7, 0xec, 0xef, 0x0d, 0x68, 0x3a, 0x2c, 0x09, 0x03, 0x8f, 0x8a, 0x20, 0x8e, 0x76, - 0x28, 0x1f, 0x30, 0x41, 0x2e, 0x40, 0x2d, 0x8a, 0x7d, 0xe6, 0x06, 0x7e, 0xcb, 0xe8, 0x19, 0x8b, - 0xd5, 0x7e, 0xeb, 0xd1, 0x41, 0x77, 0xea, 0xf0, 0xa0, 0x3b, 0x73, 0x2b, 0xf6, 0xd9, 0xe6, 0xb5, - 0xe7, 0xf9, 0x97, 0x33, 0x23, 0x81, 0x9b, 0x3e, 0x59, 0x03, 0x2b, 0x15, 0x31, 0x47, 0xce, 0x34, - 0x72, 0xda, 0x8a, 0x53, 0xdb, 0x96, 0x76, 0x24, 0xe9, 0x4f, 0xa7, 0x86, 0xd8, 0x4d, 0x7f, 0xdd, - 0x92, 0x59, 0xfc, 0xfe, 0xa0, 0x6b, 0xd8, 0x7f, 0x16, 0x99, 0x5c, 0x63, 0xa9, 0xc7, 0x83, 0x44, - 0xc4, 0xfc, 0xbf, 0xcb, 0x84, 0x5c, 0x01, 0xe0, 0xd9, 0xdf, 0x4b, 0x62, 0x05, 0x89, 0x1d, 0x45, - 0xac, 0xab, 0xc4, 0x90, 0x5a, 0xfc, 0x70, 0xea, 0x8a, 0xb1, 0xe9, 0x93, 0x55, 0x30, 0xc5, 0x38, - 0x61, 0x2d, 0xb3, 0x67, 0x2c, 0x9e, 0x58, 0xed, 0x2c, 0xbd, 0x70, 0xef, 0x4b, 0x8a, 0xb6, 0x33, - 0x4e, 0x98, 0x83, 0xd8, 0xf5, 0x59, 0x79, 0xf8, 0x5f, 0x1f, 0x74, 0x0d, 0x2c, 0xc0, 0x0f, 0x06, - 0xcc, 0xea, 0xd0, 0x3e, 0x8b, 0x84, 0x3c, 0x08, 0xa7, 0xd1, 0x20, 0x3f, 0x7c, 0xa5, 0x38, 0x88, - 0x23, 0xed, 0xd9, 0x41, 0xd4, 0xa7, 0x53, 0x43, 0xec, 0xa6, 0x4f, 0xae, 0x41, 0x4d, 0xa5, 0x85, - 0xc7, 0x6f, 0xac, 0xbe, 0xf3, 0xf2, 0x64, 0x8a, 0x4a, 0xf7, 0x4d, 0x19, 0xdb, 0xd1, 0x54, 0xfb, - 0x49, 0x05, 0xe6, 0x31, 0x74, 0xe9, 0x32, 0xde, 0x30, 0xa1, 0xb3, 0x50, 0x4f, 0x05, 0xe5, 0xc2, - 0xbd, 0xc3, 0xc6, 0x98, 0xd2, 0x6c, 0xdf, 0x7a, 0x7e, 0xd0, 0x35, 0x9d, 0xeb, 0x6c, 0xec, 0x58, - 0xe8, 0xba, 0xce, 0xc6, 0xe4, 0x0c, 0xd4, 0x58, 0xe4, 0x23, 0xa8, 0x72, 0x04, 0x34, 0xc3, 0x22, - 0x5f, 0x42, 0xbe, 0x82, 0x66, 0x10, 0x09, 0xc6, 0x23, 0x1a, 0xba, 0x2a, 0xd1, 0xb4, 0x65, 0xf6, - 0x2a, 0xaf, 0x79, 0xc8, 0x05, 0x1d, 0x44, 0x01, 0x52, 0xf2, 0x39, 0xcc, 0x47, 0xec, 0xbe, 0x70, - 0x4b, 0x0a, 0xa8, 0xa2, 0x02, 0x6c, 0x75, 0xc0, 0xb9, 0x5b, 0xec, 0xbe, 0x78, 0x89, 0x0a, 0xe6, - 0xa2, 0x92, 0xcf, 0x27, 0x1d, 0x80, 0x01, 0x8b, 0x18, 0xc7, 0x86, 0x6a, 0xcd, 0xc8, 0x3a, 0x39, - 0x25, 0x0b, 0xf9, 0x10, 0x20, 0x15, 0x81, 0x77, 0x67, 0xec, 0xee, 0x06, 0xa2, 0x55, 0xc3, 0x2b, - 0x7a, 0xab, 0x94, 0xbd, 0x1c, 0x01, 0x4b, 0x7b, 0xa1, 0xb7, 0xb4, 0xa3, 0x47, 0x80, 0x53, 0xcf, - 0x08, 0xfd, 0x40, 0x90, 0x8b, 0x70, 0xaa, 0x88, 0xe5, 0x7a, 0xf1, 0x30, 0xa1, 0x9c, 0xee, 0x86, - 0xac, 0x65, 0xf5, 0x8c, 0x45, 0xcb, 0x39, 0x59, 0x38, 0xaf, 0xe6, 0xbe, 0x23, 0x42, 0x7b, 0x68, - 0x40, 0x63, 0x8b, 0x71, 0x8f, 0x45, 0x22, 0x08, 0x59, 0x4a, 0x4e, 0x43, 0x25, 0xb9, 0xb0, 0x82, - 0x37, 0x6a, 0xa8, 0x0a, 0x49, 0x03, 0xda, 0x57, 0xd7, 0xf0, 0xc6, 0x0a, 0xfb, 0xea, 0x1a, 0xda, - 0xd7, 0x56, 0xf0, 0x92, 0x0a, 0xfb, 0x5a, 0x86, 0xbf, 0xb4, 0x86, 0x1d, 0x50, 0xd8, 0x2f, 0x65, - 0xf8, 0xcb, 0x2b, 0x58, 0xd0, 0xc2, 0x7e, 0x79, 0x85, 0xb4, 0xc0, 0x4c, 0x6e, 0xd2, 0xfb, 0x58, - 0x22, 0xed, 0x40, 0x8b, 0x9a, 0x4d, 0xcf, 0x2b, 0x30, 0x87, 0x6d, 0x7a, 0x95, 0x26, 0xd4, 0x0b, - 0xc4, 0x98, 0xf4, 0xc0, 0xf2, 0xd4, 0xb7, 0x12, 0x60, 0xc6, 0xca, 0xad, 0xc4, 0x86, 0x3a, 0xdd, - 0xa7, 0x41, 0x88, 0x25, 0x99, 0x2e, 0x41, 0x0a, 0x33, 0x39, 0x0b, 0x8d, 0x4c, 0xc6, 0x5e, 0x3c, - 0x8a, 0x84, 0x6a, 0xf5, 0x0c, 0x05, 0xe8, 0xb8, 0x2a, 0xed, 0x12, 0x16, 0x32, 0x9a, 0x6a, 0x98, - 0x59, 0x86, 0xa1, 0x23, 0x83, 0xad, 0x40, 0xf3, 0x1e, 0x0f, 0x04, 0x4b, 0xdd, 0x84, 0x71, 0x37, - 0x65, 0x5e, 0x1c, 0xf9, 0x13, 0x67, 0x9d, 0xcf, 0xdc, 0x5b, 0x8c, 0x6f, 0xa3, 0x93, 0x6c, 0x41, - 0x73, 0x77, 0xac, 0x09, 0xba, 0x55, 0x67, 0x50, 0x07, 0xc7, 0xcd, 0x8d, 0xd2, 0x55, 0xe9, 0x88, - 0x48, 0xdf, 0x62, 0x5c, 0x89, 0x8e, 0x38, 0x40, 0x4a, 0x39, 0xe8, 0x90, 0xb5, 0xd7, 0x08, 0xb9, - 0x90, 0x27, 0xa9, 0x63, 0xb6, 0xc0, 0x1c, 0xa5, 0xcc, 0x47, 0x5d, 0xe9, 0x22, 0xa2, 0x85, 0x9c, - 0x83, 0xb9, 0x30, 0x1e, 0x04, 0x1e, 0x0d, 0x5d, 0x4c, 0xa4, 0x55, 0x2f, 0x41, 0x66, 0x95, 0xab, - 0x2f, 0x3d, 0x64, 0x15, 0xc8, 0xdd, 0x11, 0xe3, 0xc1, 0x64, 0x75, 0xa0, 0x54, 0x9d, 0x05, 0xe5, - 0xcf, 0xcb, 0xa3, 0x2e, 0xff, 0xa9, 0x09, 0x27, 0xe4, 0x60, 0xff, 0x77, 0xbb, 0xe0, 0x23, 0xa8, - 0xc9, 0xed, 0xca, 0xd2, 0x54, 0xcd, 0xc2, 0xce, 0xd1, 0x46, 0xfb, 0x32, 0xdf, 0xc3, 0x1b, 0xbe, - 0x9f, 0x4f, 0x41, 0x45, 0x22, 0x97, 0xf5, 0x46, 0xad, 0xbc, 0xd0, 0xa6, 0xba, 0x96, 0xc5, 0xfe, - 0x55, 0xe4, 0x8c, 0x41, 0xae, 0x80, 0x15, 0xc6, 0x1e, 0x0d, 0xa5, 0x56, 0x4d, 0x64, 0xff, 0xff, - 0x18, 0xf6, 0x0d, 0x05, 0xd1, 0x42, 0xd6, 0x14, 0xf2, 0x19, 0xcc, 0x6d, 0x33, 0xbe, 0xcf, 0xf8, - 0x6d, 0xc6, 0x53, 0x39, 0x48, 0xaa, 0x18, 0xa3, 0x7d, 0x4c, 0x0c, 0x85, 0x50, 0x21, 0x26, 0x69, - 0xe4, 0x0c, 0xd4, 0x77, 0x47, 0x41, 0xe8, 0xbb, 0x82, 0x0e, 0x50, 0x64, 0x75, 0xfd, 0x57, 0x68, - 0xde, 0xa1, 0x03, 0xf2, 0xb6, 0x1c, 0x48, 0x94, 0x0b, 0xf9, 0x14, 0xc9, 0x06, 0x52, 0xde, 0x34, - 0xca, 0xbe, 0x21, 0xc8, 0x36, 0x2c, 0xe8, 0xdc, 0x5c, 0x5d, 0x52, 0x0b, 0x27, 0xaf, 0xfd, 0x37, - 0xc7, 0xda, 0xc8, 0x90, 0x5a, 0xb7, 0xe1, 0xa4, 0x99, 0xbc, 0x0b, 0xb3, 0x5e, 0x38, 0x4a, 0x05, - 0xe3, 0x6e, 0x44, 0x87, 0x0c, 0x85, 0xa4, 0xf3, 0x6b, 0x28, 0xcf, 0x2d, 0x3a, 0x64, 0x64, 0x1b, - 0x1a, 0xe9, 0xdd, 0x30, 0xff, 0x63, 0x78, 0xa5, 0xbb, 0x24, 0x4a, 0x1e, 0xb0, 0xfd, 0xc5, 0x0d, - 0xf5, 0x8f, 0x0e, 0xa4, 0x77, 0x43, 0xf5, 0x6d, 0xff, 0x6c, 0xc0, 0xfc, 0x91, 0x44, 0xcb, 0x82, - 0x31, 0xde, 0x44, 0x30, 0x7d, 0xd9, 0x1b, 0xaa, 0x4c, 0x22, 0x60, 0x5c, 0xc9, 0xee, 0x7f, 0xc7, - 0xd4, 0x68, 0x27, 0x60, 0xbc, 0x68, 0x9a, 0x8c, 0x23, 0x6d, 0xf6, 0x77, 0xd3, 0x30, 0x8f, 0x73, - 0x6f, 0x72, 0xf5, 0xe6, 0x8f, 0x1a, 0xe3, 0xd5, 0x1f, 0x35, 0xb9, 0x7e, 0xa7, 0x5f, 0x5b, 0xbf, - 0x1f, 0x80, 0x29, 0x9b, 0x48, 0x29, 0xff, 0xcc, 0x31, 0xcc, 0xc9, 0xf6, 0xd4, 0x23, 0x42, 0x92, - 0x48, 0xbf, 0x34, 0xa8, 0x33, 0xf1, 0xf7, 0x8e, 0x09, 0x30, 0x31, 0xdc, 0x8f, 0x8e, 0x72, 0xfb, - 0x47, 0x03, 0x9a, 0xaa, 0x0c, 0xd4, 0xcf, 0x37, 0xf5, 0x1b, 0x16, 0x62, 0x03, 0xac, 0xfc, 0xc1, - 0x30, 0x8d, 0xb2, 0xed, 0xbe, 0xfc, 0xc1, 0x80, 0xcf, 0x2f, 0x9d, 0x8f, 0xa6, 0xd9, 0x9f, 0x82, - 0xa5, 0xd5, 0x42, 0x2e, 0x42, 0x55, 0xde, 0x6e, 0xf6, 0xd2, 0xfe, 0xc7, 0xeb, 0xcd, 0xb0, 0x6a, - 0xb0, 0x7d, 0x02, 0xa6, 0x74, 0xc9, 0xad, 0x28, 0x9f, 0x3a, 0x46, 0x49, 0xf2, 0xd2, 0x40, 0xda, - 0x50, 0xdd, 0xa7, 0xe1, 0x28, 0xdb, 0x5e, 0xda, 0x93, 0x99, 0x54, 0x84, 0x87, 0x06, 0xd4, 0x74, - 0x7b, 0x9f, 0x87, 0xfa, 0x90, 0x7e, 0x1b, 0x73, 0x77, 0x9f, 0x86, 0xaa, 0x1e, 0x73, 0xaa, 0x1e, - 0xd5, 0x9b, 0xd2, 0xe1, 0x58, 0xe8, 0xbf, 0x4d, 0x43, 0xc4, 0x06, 0x91, 0xc2, 0x4e, 0x1f, 0xc1, - 0x4a, 0x87, 0x63, 0xa1, 0x5f, 0x62, 0xdb, 0x50, 0x4d, 0xa8, 0xf0, 0xf6, 0x26, 0xb6, 0x63, 0x66, - 0x92, 0x5b, 0x78, 0x14, 0xa5, 0x02, 0x57, 0x6c, 0x79, 0x2b, 0xe6, 0xd6, 0x2c, 0xcf, 0xf3, 0x67, - 0xa1, 0x51, 0x7a, 0xf3, 0x92, 0x3a, 0x54, 0x6f, 0xc7, 0x82, 0xf1, 0x85, 0x29, 0xd2, 0x80, 0xda, - 0x0d, 0x46, 0x79, 0xc4, 0xf8, 0x82, 0xd1, 0x3f, 0xf7, 0xe8, 0xb7, 0xce, 0xd4, 0xa3, 0xc3, 0x8e, - 0xf1, 0xf8, 0xb0, 0x63, 0x3c, 0x39, 0xec, 0x18, 0x4f, 0x0f, 0x3b, 0xc6, 0x4f, 0xcf, 0x3a, 0x53, - 0x8f, 0x9f, 0x75, 0xa6, 0x9e, 0x3c, 0xeb, 0x4c, 0x7d, 0x5d, 0x53, 0x35, 0xfd, 0x2b, 0x00, 0x00, - 0xff, 0xff, 0xf3, 0xb6, 0xef, 0xe2, 0x66, 0x0d, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_e31f96670cc94504) } + +var fileDescriptor_metadata_e31f96670cc94504 = []byte{ + // 1376 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xcd, 0x6f, 0x1b, 0x45, + 0x14, 0xcf, 0xfa, 0x23, 0xb6, 0x9f, 0xe3, 0xc6, 0x1e, 0xb5, 0xc5, 0x32, 0xc2, 0x76, 0x0d, 0x15, + 0x69, 0x41, 0x49, 0x9a, 0x2a, 0xaa, 0x1a, 0x28, 0x10, 0xb7, 0x54, 0x0a, 0xfd, 0x0a, 0x9b, 0x50, + 0x24, 0x2e, 0xab, 0xc9, 0xee, 0xe0, 0x2c, 0x1d, 0xef, 0xba, 0xb3, 0xe3, 0xb4, 0xbe, 0x23, 0x71, + 0x40, 0x48, 0x5c, 0x90, 0x38, 0xf6, 0xd2, 0xff, 0x81, 0x3f, 0xa1, 0xc7, 0x1e, 0x7b, 0x8a, 0x4a, + 0x7a, 0xe1, 0xcc, 0xb1, 0x07, 0x84, 0xe6, 0xed, 0xcc, 0xee, 0x3a, 0x4d, 0xa1, 0x2d, 0x12, 0xb7, + 0xf5, 0xef, 0xfd, 0x7e, 0xcf, 0x6f, 0xde, 0xbc, 0x8f, 0x81, 0x93, 0x22, 0xa4, 0xee, 0xee, 0x68, + 0x67, 0x69, 0xc8, 0x24, 0xf5, 0xa8, 0xa4, 0x8b, 0x23, 0x11, 0xca, 0x90, 0x34, 0xdc, 0xd0, 0xbd, + 0x83, 0xb6, 0x45, 0xcd, 0x68, 0xb5, 0xc6, 0xd2, 0xe7, 0x4b, 0xe3, 0x40, 0xb0, 0x28, 0xe4, 0x7b, + 0xcc, 0x73, 0xa8, 0xe7, 0x89, 0x98, 0xde, 0x6a, 0xa2, 0x6d, 0x97, 0xbb, 0x4b, 0xd2, 0x1f, 0xb2, + 0x48, 0xd2, 0xe1, 0x48, 0x5b, 0x8e, 0x0f, 0xc2, 0x41, 0x88, 0x9f, 0x4b, 0xea, 0x2b, 0x46, 0x7b, + 0x9f, 0x02, 0xac, 0x4b, 0x29, 0xfc, 0x9d, 0xb1, 0x64, 0x11, 0xf9, 0x00, 0x8a, 0x54, 0x4a, 0x11, + 0x35, 0xad, 0x6e, 0x7e, 0xa1, 0xd2, 0x3f, 0xf1, 0xe7, 0x7e, 0xa7, 0x31, 0xa1, 0x43, 0xbe, 0xd6, + 0x43, 0xf8, 0xc3, 0x6f, 0x79, 0x78, 0xaf, 0x67, 0xc7, 0x9c, 0xb5, 0xc2, 0xaf, 0x0f, 0x3a, 0x33, + 0xbd, 0x1f, 0x2c, 0x68, 0xd8, 0x6c, 0xc4, 0x7d, 0x97, 0x4a, 0x3f, 0x0c, 0xb6, 0xa9, 0x18, 0x30, + 0x49, 0xce, 0x41, 0x29, 0x08, 0x3d, 0xe6, 0xf8, 0x5e, 0xd3, 0xea, 0x5a, 0x0b, 0xc5, 0x7e, 0xf3, + 0xd1, 0x7e, 0x67, 0xe6, 0x60, 0xbf, 0x33, 0x7b, 0x33, 0xf4, 0xd8, 0xc6, 0x95, 0xe7, 0xc9, 0x97, + 0x3d, 0xab, 0x88, 0x1b, 0x1e, 0x59, 0x85, 0x72, 0x24, 0x43, 0x81, 0x9a, 0x1c, 0x6a, 0x5a, 0x5a, + 0x53, 0xda, 0x52, 0x38, 0x8a, 0xcc, 0xa7, 0x5d, 0x42, 0xee, 0x86, 0xb7, 0x56, 0x56, 0x51, 0xfc, + 0xf1, 0xa0, 0x63, 0xf5, 0xfe, 0x4a, 0x23, 0xb9, 0xc2, 0x22, 0x57, 0xf8, 0x23, 0x19, 0x8a, 0xff, + 0x2f, 0x12, 0x72, 0x09, 0x40, 0xc4, 0x7f, 0xaf, 0x84, 0x79, 0x14, 0xb6, 0xb5, 0xb0, 0xa2, 0x03, + 0x43, 0x69, 0xfa, 0xc3, 0xae, 0x68, 0xc5, 0x86, 0x47, 0x56, 0xa0, 0x20, 0x27, 0x23, 0xd6, 0x2c, + 0x74, 0xad, 0x85, 0x63, 0x2b, 0xed, 0xc5, 0x17, 0xee, 0x7d, 0x51, 0xcb, 0xb6, 0x27, 0x23, 0x66, + 0x23, 0x77, 0x6d, 0x4e, 0x1d, 0xfe, 0xb7, 0x07, 0x1d, 0x0b, 0x13, 0xf0, 0xa3, 0x05, 0x73, 0xc6, + 0xb5, 0xc7, 0x02, 0xa9, 0x0e, 0x22, 0x68, 0x30, 0x48, 0x0e, 0x9f, 0x4f, 0x0f, 0x62, 0x2b, 0x3c, + 0x3e, 0x88, 0xfe, 0xb4, 0x4b, 0xc8, 0xdd, 0xf0, 0xc8, 0x15, 0x28, 0xe9, 0xb0, 0xf0, 0xf8, 0xd5, + 0x95, 0xf7, 0x5e, 0x1e, 0x4c, 0x9a, 0xe9, 0x7e, 0x41, 0xf9, 0xb6, 0x8d, 0xb4, 0xf7, 0x24, 0x0f, + 0xf3, 0xe8, 0x3a, 0x73, 0x19, 0x6f, 0x18, 0xd0, 0x69, 0xa8, 0x44, 0x92, 0x0a, 0xe9, 0xdc, 0x61, + 0x13, 0x0c, 0x69, 0xae, 0x5f, 0x7e, 0xbe, 0xdf, 0x29, 0xd8, 0xd7, 0xd8, 0xc4, 0x2e, 0xa3, 0xe9, + 0x1a, 0x9b, 0x90, 0x53, 0x50, 0x62, 0x81, 0x87, 0xa4, 0xfc, 0x21, 0xd2, 0x2c, 0x0b, 0x3c, 0x45, + 0xf9, 0x1a, 0x1a, 0x7e, 0x20, 0x99, 0x08, 0x28, 0x77, 0x74, 0xa0, 0x51, 0xb3, 0xd0, 0xcd, 0xbf, + 0xe6, 0x21, 0xeb, 0xc6, 0x89, 0x26, 0x44, 0xe4, 0x0b, 0x98, 0x0f, 0xd8, 0x7d, 0xe9, 0x64, 0x2a, + 0xa0, 0x88, 0x15, 0xd0, 0xd3, 0x07, 0xac, 0xdd, 0x64, 0xf7, 0xe5, 0x4b, 0xaa, 0xa0, 0x16, 0x64, + 0x6c, 0x1e, 0x69, 0x03, 0x0c, 0x58, 0xc0, 0x04, 0x36, 0x54, 0x73, 0x56, 0xe5, 0xc9, 0xce, 0x20, + 0xe4, 0x63, 0x80, 0x48, 0xfa, 0xee, 0x9d, 0x89, 0xb3, 0xe3, 0xcb, 0x66, 0x09, 0xaf, 0xe8, 0x9d, + 0x4c, 0xf4, 0x6a, 0x04, 0x2c, 0xee, 0x72, 0x77, 0x71, 0xdb, 0x8c, 0x00, 0xbb, 0x12, 0x0b, 0xfa, + 0xbe, 0x24, 0xe7, 0xe1, 0x44, 0xea, 0xcb, 0x71, 0xc3, 0xe1, 0x88, 0x0a, 0xba, 0xc3, 0x59, 0xb3, + 0xdc, 0xb5, 0x16, 0xca, 0xf6, 0xf1, 0xd4, 0x78, 0x39, 0xb1, 0x1d, 0x2a, 0xb4, 0x87, 0x16, 0x54, + 0x37, 0x99, 0x70, 0x59, 0x20, 0x7d, 0xce, 0x22, 0x72, 0x12, 0xf2, 0xa3, 0x73, 0xcb, 0x78, 0xa3, + 0x96, 0xce, 0x90, 0x02, 0x10, 0x5f, 0x59, 0xc5, 0x1b, 0x4b, 0xf1, 0x95, 0x55, 0xc4, 0x57, 0x97, + 0xf1, 0x92, 0x52, 0x7c, 0x35, 0xe6, 0x5f, 0x58, 0xc5, 0x0e, 0x48, 0xf1, 0x0b, 0x31, 0xff, 0xe2, + 0x32, 0x26, 0x34, 0xc5, 0x2f, 0x2e, 0x93, 0x26, 0x14, 0x46, 0x37, 0xe8, 0x7d, 0x4c, 0x91, 0x31, + 0x20, 0xa2, 0x67, 0xd3, 0xf3, 0x3c, 0xd4, 0xb0, 0x4d, 0x2f, 0xd3, 0x11, 0x75, 0x7d, 0x39, 0x21, + 0x5d, 0x28, 0xbb, 0xfa, 0x5b, 0x17, 0x60, 0xac, 0x4a, 0x50, 0xd2, 0x83, 0x0a, 0xdd, 0xa3, 0x3e, + 0xc7, 0x94, 0xe4, 0x32, 0x94, 0x14, 0x26, 0xa7, 0xa1, 0x1a, 0x97, 0xb1, 0x1b, 0x8e, 0x03, 0xa9, + 0x5b, 0x3d, 0x66, 0x01, 0x1a, 0x2e, 0x2b, 0x5c, 0xd1, 0x38, 0xa3, 0x91, 0xa1, 0x15, 0xb2, 0x34, + 0x34, 0xc4, 0xb4, 0x65, 0x68, 0xdc, 0x13, 0xbe, 0x64, 0x91, 0x33, 0x62, 0xc2, 0x89, 0x98, 0x1b, + 0x06, 0xde, 0xd4, 0x59, 0xe7, 0x63, 0xf3, 0x26, 0x13, 0x5b, 0x68, 0x24, 0x9b, 0xd0, 0xd8, 0x99, + 0x18, 0x81, 0x69, 0xd5, 0x59, 0xac, 0x83, 0xa3, 0xe6, 0x46, 0xe6, 0xaa, 0x8c, 0x47, 0x94, 0x6f, + 0x32, 0xa1, 0x8b, 0x8e, 0xd8, 0x40, 0x32, 0x31, 0x18, 0x97, 0xa5, 0xd7, 0x70, 0x59, 0x4f, 0x82, + 0x34, 0x3e, 0x9b, 0x50, 0x18, 0x47, 0xcc, 0xc3, 0xba, 0x32, 0x49, 0x44, 0x84, 0x9c, 0x81, 0x1a, + 0x0f, 0x07, 0xbe, 0x4b, 0xb9, 0x83, 0x81, 0x34, 0x2b, 0x19, 0xca, 0x9c, 0x36, 0xf5, 0x95, 0x85, + 0xac, 0x00, 0xb9, 0x3b, 0x66, 0xc2, 0x9f, 0xce, 0x0e, 0x64, 0xb2, 0x53, 0xd7, 0xf6, 0x24, 0x3d, + 0xfa, 0xf2, 0x9f, 0x16, 0xe0, 0x98, 0x1a, 0xec, 0xff, 0x6d, 0x17, 0x7c, 0x02, 0x25, 0xb5, 0x5d, + 0x59, 0x14, 0xe9, 0x59, 0xd8, 0x3e, 0xdc, 0x68, 0x5f, 0x25, 0x7b, 0x78, 0xdd, 0xf3, 0x92, 0x29, + 0xa8, 0x45, 0xe4, 0xa2, 0xd9, 0xa8, 0xf9, 0x17, 0xda, 0xd4, 0xe4, 0x32, 0xdd, 0xbf, 0x5a, 0x1c, + 0x2b, 0xc8, 0x25, 0x28, 0xf3, 0xd0, 0xa5, 0x5c, 0xd5, 0x6a, 0x01, 0xd5, 0x6f, 0x1f, 0xa1, 0xbe, + 0xae, 0x29, 0xa6, 0x90, 0x8d, 0x84, 0x5c, 0x85, 0xda, 0x16, 0x13, 0x7b, 0x4c, 0xdc, 0x66, 0x22, + 0x52, 0x83, 0xa4, 0x88, 0x3e, 0x5a, 0x47, 0xf8, 0xd0, 0x0c, 0xed, 0x62, 0x5a, 0x46, 0x4e, 0x41, + 0x65, 0x67, 0xec, 0x73, 0xcf, 0x91, 0x74, 0x80, 0x45, 0x56, 0x31, 0x7f, 0x85, 0xf0, 0x36, 0x1d, + 0x90, 0x77, 0xd5, 0x40, 0xa2, 0x42, 0xaa, 0xa7, 0x48, 0x3c, 0x90, 0x92, 0xa6, 0xd1, 0xf8, 0xba, + 0x24, 0x5b, 0x50, 0x37, 0xb1, 0x39, 0x26, 0xa5, 0x65, 0x9c, 0xbc, 0xbd, 0x7f, 0x38, 0xd6, 0x7a, + 0xcc, 0x34, 0x75, 0xcb, 0xa7, 0x61, 0xf2, 0x3e, 0xcc, 0xb9, 0x7c, 0x1c, 0x49, 0x26, 0x9c, 0x80, + 0x0e, 0x19, 0x16, 0x92, 0x89, 0xaf, 0xaa, 0x2d, 0x37, 0xe9, 0x90, 0x91, 0x2d, 0xa8, 0x46, 0x77, + 0x79, 0xf2, 0xc7, 0xf0, 0x4a, 0x77, 0x49, 0x74, 0x79, 0xc0, 0xd6, 0x97, 0xd7, 0xf5, 0x3f, 0xda, + 0x10, 0xdd, 0xe5, 0xfa, 0xbb, 0xf7, 0x8b, 0x05, 0xf3, 0x87, 0x02, 0xcd, 0x16, 0x8c, 0xf5, 0x26, + 0x05, 0xd3, 0x57, 0xbd, 0xa1, 0xd3, 0x24, 0x7d, 0x26, 0x74, 0xd9, 0xbd, 0x75, 0x44, 0x8e, 0xb6, + 0x7d, 0x26, 0xd2, 0xa6, 0x89, 0x35, 0x0a, 0xeb, 0x7d, 0x9f, 0x83, 0x79, 0x9c, 0x7b, 0xd3, 0xab, + 0x37, 0x79, 0xd4, 0x58, 0xaf, 0xfe, 0xa8, 0x49, 0xea, 0x37, 0xf7, 0xda, 0xf5, 0xfb, 0x11, 0x14, + 0x54, 0x13, 0xe9, 0xca, 0x3f, 0x75, 0x84, 0x72, 0xba, 0x3d, 0xcd, 0x88, 0x50, 0x22, 0xd2, 0xcf, + 0x0c, 0xea, 0xb8, 0xf8, 0xbb, 0x47, 0x38, 0x98, 0x1a, 0xee, 0x87, 0x47, 0x79, 0xef, 0x27, 0x0b, + 0x1a, 0x3a, 0x0d, 0xd4, 0x4b, 0x36, 0xf5, 0x1b, 0x26, 0x62, 0x1d, 0xca, 0xc9, 0x83, 0x21, 0x87, + 0x65, 0xdb, 0x79, 0xf9, 0x83, 0x01, 0x9f, 0x5f, 0x26, 0x1e, 0x23, 0xeb, 0x7d, 0x0e, 0x65, 0x53, + 0x2d, 0xe4, 0x3c, 0x14, 0xd5, 0xed, 0xc6, 0x2f, 0xed, 0x7f, 0xbd, 0xde, 0x98, 0xab, 0x07, 0xdb, + 0x67, 0x50, 0x50, 0x26, 0xb5, 0x15, 0xd5, 0x53, 0xc7, 0xca, 0x94, 0xbc, 0x02, 0x48, 0x0b, 0x8a, + 0x7b, 0x94, 0x8f, 0xe3, 0xed, 0x65, 0x2c, 0x31, 0xa4, 0x3d, 0x3c, 0xb4, 0xa0, 0x64, 0xda, 0xfb, + 0x2c, 0x54, 0x86, 0xf4, 0xbb, 0x50, 0x38, 0x7b, 0x94, 0xeb, 0x7c, 0xd4, 0x74, 0x3e, 0x8a, 0x37, + 0x94, 0xc1, 0x2e, 0xa3, 0xfd, 0x36, 0xe5, 0xc8, 0xf5, 0x03, 0xcd, 0xcd, 0x1d, 0xe2, 0x2a, 0x83, + 0x5d, 0x46, 0xbb, 0xe2, 0xb6, 0xa0, 0x38, 0xa2, 0xd2, 0xdd, 0x9d, 0xda, 0x8e, 0x31, 0xa4, 0xb6, + 0xf0, 0x38, 0x88, 0x24, 0xae, 0xd8, 0xec, 0x56, 0x4c, 0xd0, 0x38, 0xce, 0xb3, 0xb7, 0xa0, 0x9a, + 0x79, 0xf3, 0x92, 0x1a, 0x54, 0x6e, 0x87, 0x92, 0x89, 0xab, 0x63, 0xce, 0xeb, 0x33, 0xa4, 0x01, + 0x35, 0xfc, 0xb9, 0x11, 0xb8, 0xe1, 0xd0, 0x0f, 0x06, 0xf5, 0x5c, 0x02, 0xdd, 0x1a, 0xcb, 0x41, + 0xa8, 0xa0, 0x3c, 0xa9, 0x42, 0xe9, 0x3a, 0xa3, 0x22, 0x60, 0xa2, 0x6e, 0xf5, 0xcf, 0x3c, 0xfa, + 0xbd, 0x3d, 0xf3, 0xe8, 0xa0, 0x6d, 0x3d, 0x3e, 0x68, 0x5b, 0x4f, 0x0e, 0xda, 0xd6, 0xd3, 0x83, + 0xb6, 0xf5, 0xf3, 0xb3, 0xf6, 0xcc, 0xe3, 0x67, 0xed, 0x99, 0x27, 0xcf, 0xda, 0x33, 0xdf, 0x94, + 0x74, 0xf6, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x84, 0x14, 0xea, 0x91, 0x90, 0x0d, 0x00, 0x00, } diff --git a/pkg/roachpb/metadata.proto b/pkg/roachpb/metadata.proto index cf5f8d2dad00..7d09c2927e74 100644 --- a/pkg/roachpb/metadata.proto +++ b/pkg/roachpb/metadata.proto @@ -36,17 +36,44 @@ message ReplicationTarget { } // ReplicaType identifies which raft activities a replica participates in. +// In normal operation, Voter and Learner are the only used states. However, +// atomic replication changes require a transition through a "joint config"; +// in this joint config, the VoterOutgoing and VoterIncoming types are used +// as well to denote voters which are being removed and newly added by the +// change, respectively. +// +// All voter types indicate a replica that participates in all raft activities, +// including voting for leadership and committing entries. Typically, this +// requires a majority of voters to reach a decision. In the joint config, +// two separate majorities are required: one from the set of replicas that +// have either type Voter or VoterOutgoing, as well as that of the set of +// types Voter and VoterIncoming. For example, when type Voter is assigned +// to replicas 1 and 2, while 3 is VoterOutgoing and 4 is VoterIncoming, then +// the two sets over which quorums need to be achieved are {1,2,3} and {1,2,4}. +// Thus, {1,2} is a quorum of both, {1,3} is a quorum of the first but not the +// second, {1,4} is a quorum of the second but not the first, and {3,4} is a +// quorum of neither. enum ReplicaType { - // ReplicaType_Voter indicates a replica that participates in all raft - // activities, including voting for leadership and committing entries. - // Notably, voters are considered for quorum size of the raft group. - Voter = 0; + // ReplicaType_VoterFull indicates a replica that is a voter both in the + // incoming and outgoing set. + VoterFull = 0; + // ReplicaType_VoterIncoming indicates a voting replica that will be a + // VoterFull once the ongoing atomic replication change is finalized; that is, + // it is in the process of being added. In practice, this replica type should + // be treated like a VoterFull. + VoterIncoming = 2; + // ReplicaType_VoterOutgoing indicates a voting replica that will not be part + // of the descriptor once the ongoing atomic replication change is finalized; + // that is, it is in the process of being removed. In practice, a replica of + // this type should be treated accordingly and no work should be assigned to + // it. + VoterOutgoing = 3; // ReplicaType_Learner indicates a replica that applies committed entries, but - // does not count towards the quorum. Learners do not vote for leadership nor - // do their acknowledged log entries get taken into account for determining - // the committed index. At the time of writing, learners in CockroachDB are a - // short-term transient state: a replica being added and on its way to being a - // VOTER. + // does not count towards the quorum(s). Learners do not vote for leadership + // nor do their acknowledged log entries get taken into account for + // determining the committed index. At the time of writing, learners in + // CockroachDB are a short-term transient state: a replica being added and on + // its way to being a VOTER. Learner = 1; } diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index e7924970010e..629214d5f16d 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -15,10 +15,10 @@ import ( "strings" ) -// ReplicaTypeVoter returns a ReplicaType_Voter pointer suitable for use in a +// ReplicaTypeVoterFull returns a ReplicaType_VoterFull pointer suitable for use in a // nullable proto field. -func ReplicaTypeVoter() *ReplicaType { - t := ReplicaType_Voter +func ReplicaTypeVoterFull() *ReplicaType { + t := ReplicaType_VoterFull return &t } @@ -77,7 +77,7 @@ func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { // save the alloc. fastpath := true for i := range d.wrapped { - if d.wrapped[i].GetType() != ReplicaType_Voter { + if d.wrapped[i].GetType() != ReplicaType_VoterFull { fastpath = false break } @@ -87,7 +87,7 @@ func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { } voters := make([]ReplicaDescriptor, 0, len(d.wrapped)) for i := range d.wrapped { - if d.wrapped[i].GetType() == ReplicaType_Voter { + if d.wrapped[i].GetType() == ReplicaType_VoterFull { voters = append(voters, d.wrapped[i]) } } @@ -243,7 +243,7 @@ func (d *ReplicaDescriptors) RemoveReplica( func (d ReplicaDescriptors) QuorumSize() int { var numVoters int for i := range d.wrapped { - if d.wrapped[i].GetType() == ReplicaType_Voter { + if d.wrapped[i].GetType() == ReplicaType_VoterFull { numVoters++ } } diff --git a/pkg/roachpb/metadata_replicas_test.go b/pkg/roachpb/metadata_replicas_test.go index d298c13143e4..21f7845a4a8b 100644 --- a/pkg/roachpb/metadata_replicas_test.go +++ b/pkg/roachpb/metadata_replicas_test.go @@ -17,7 +17,7 @@ import ( ) func TestVotersLearnersAll(t *testing.T) { - voter := ReplicaTypeVoter() + voter := ReplicaTypeVoterFull() learner := ReplicaTypeLearner() tests := [][]ReplicaDescriptor{ {}, @@ -32,7 +32,7 @@ func TestVotersLearnersAll(t *testing.T) { for i, test := range tests { r := MakeReplicaDescriptors(test) for _, voter := range r.Voters() { - assert.Equal(t, ReplicaType_Voter, voter.GetType(), "testcase %d", i) + assert.Equal(t, ReplicaType_VoterFull, voter.GetType(), "testcase %d", i) } for _, learner := range r.Learners() { assert.Equal(t, ReplicaType_Learner, learner.GetType(), "testcase %d", i) @@ -86,7 +86,7 @@ func TestReplicaDescriptorsRemove(t *testing.T) { assert.Equal(t, lenBefore, len(r.All()), "testcase %d", i) } for _, voter := range r.Voters() { - assert.Equal(t, ReplicaType_Voter, voter.GetType(), "testcase %d", i) + assert.Equal(t, ReplicaType_VoterFull, voter.GetType(), "testcase %d", i) } for _, learner := range r.Learners() { assert.Equal(t, ReplicaType_Learner, learner.GetType(), "testcase %d", i) diff --git a/pkg/storage/batcheval/cmd_lease.go b/pkg/storage/batcheval/cmd_lease.go index 8c02d4c6d46f..23b6d8a5bfcc 100644 --- a/pkg/storage/batcheval/cmd_lease.go +++ b/pkg/storage/batcheval/cmd_lease.go @@ -47,7 +47,7 @@ func checkCanReceiveLease(rec EvalContext) error { if !ok { return errors.AssertionFailedf( `could not find replica for store %s in %s`, rec.StoreID(), rec.Desc()) - } else if t := repDesc.GetType(); t != roachpb.ReplicaType_Voter { + } else if t := repDesc.GetType(); t != roachpb.ReplicaType_VoterFull { return errors.Errorf(`cannot transfer lease to replica of type %s`, t) } return nil diff --git a/pkg/storage/batcheval/cmd_lease_test.go b/pkg/storage/batcheval/cmd_lease_test.go index e1a0f274f354..f6480f148a5e 100644 --- a/pkg/storage/batcheval/cmd_lease_test.go +++ b/pkg/storage/batcheval/cmd_lease_test.go @@ -113,7 +113,7 @@ func TestLeaseCommandLearnerReplica(t *testing.T) { ctx := context.Background() const voterStoreID, learnerStoreID roachpb.StoreID = 1, 2 replicas := []roachpb.ReplicaDescriptor{ - {StoreID: voterStoreID, Type: roachpb.ReplicaTypeVoter()}, + {StoreID: voterStoreID, Type: roachpb.ReplicaTypeVoterFull()}, {StoreID: learnerStoreID, Type: roachpb.ReplicaTypeLearner()}, } desc := roachpb.RangeDescriptor{} diff --git a/pkg/storage/below_raft_protos_test.go b/pkg/storage/below_raft_protos_test.go index 06500e1819e4..7b55d4f47ffc 100644 --- a/pkg/storage/below_raft_protos_test.go +++ b/pkg/storage/below_raft_protos_test.go @@ -96,7 +96,7 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{ return roachpb.NewPopulatedRangeDescriptor(r, false) }, emptySum: 5524024218313206949, - populatedSum: 8797706987540299345, + populatedSum: 3507997910320018655, }, reflect.TypeOf(&storagepb.Liveness{}): { populatedConstructor: func(r *rand.Rand) protoutil.Message { diff --git a/pkg/storage/client_test.go b/pkg/storage/client_test.go index e4782757d887..173cfbf9477d 100644 --- a/pkg/storage/client_test.go +++ b/pkg/storage/client_test.go @@ -1204,7 +1204,7 @@ func (m *multiTestContext) replicateRangeNonFatal(rangeID roachpb.RangeID, dests if e := expectedReplicaIDs[i]; repDesc.ReplicaID != e { return errors.Errorf("expected replica %s to have ID %d", repl, e) } - if t := repDesc.GetType(); t != roachpb.ReplicaType_Voter { + if t := repDesc.GetType(); t != roachpb.ReplicaType_VoterFull { return errors.Errorf("expected replica %s to be a voter was %s", repl, t) } if !repl.Desc().ContainsKey(startKey) { diff --git a/pkg/storage/merge_queue.go b/pkg/storage/merge_queue.go index 8a55aeb57749..84baa46f2d6d 100644 --- a/pkg/storage/merge_queue.go +++ b/pkg/storage/merge_queue.go @@ -295,12 +295,12 @@ func (mq *mergeQueue) process( // Defensive sanity check that everything is now a voter. for i := range lhsReplicas { - if lhsReplicas[i].GetType() != roachpb.ReplicaType_Voter { + if lhsReplicas[i].GetType() != roachpb.ReplicaType_VoterFull { return errors.Errorf(`cannot merge non-voter replicas on lhs: %v`, lhsReplicas) } } for i := range rhsReplicas { - if rhsReplicas[i].GetType() != roachpb.ReplicaType_Voter { + if rhsReplicas[i].GetType() != roachpb.ReplicaType_VoterFull { return errors.Errorf(`cannot merge non-voter replicas on rhs: %v`, rhsReplicas) } } diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index a4ff71639378..06fc81226cb2 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -1072,7 +1072,7 @@ func (r *Replica) finalizeChangeReplicas( return nil, errors.Errorf("programming error: cannot promote replica of type %s", rDesc.Type) } - rDesc.Type = roachpb.ReplicaTypeVoter() + rDesc.Type = roachpb.ReplicaTypeVoterFull() updatedDesc.AddReplica(rDesc) replsAdded = append(replsAdded, rDesc) diff --git a/pkg/storage/replica_proposal_buf.go b/pkg/storage/replica_proposal_buf.go index f14e7746e676..17c336584f1e 100644 --- a/pkg/storage/replica_proposal_buf.go +++ b/pkg/storage/replica_proposal_buf.go @@ -456,7 +456,7 @@ func (b *propBuf) FlushLockedWithRaftGroup(raftGroup *raft.RawNode) error { replicaID = added[0].ReplicaID typ := added[0].GetType() switch typ { - case roachpb.ReplicaType_Voter: + case roachpb.ReplicaType_VoterFull: changeType = raftpb.ConfChangeAddNode case roachpb.ReplicaType_Learner: changeType = raftpb.ConfChangeAddLearnerNode diff --git a/pkg/storage/replica_range_lease.go b/pkg/storage/replica_range_lease.go index a7a6c23712a4..d662e51ec53e 100644 --- a/pkg/storage/replica_range_lease.go +++ b/pkg/storage/replica_range_lease.go @@ -683,7 +683,7 @@ func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID // `r.mu.minLeaseProposedTS = status.Timestamp` line below will likely play // badly with that. This would be an issue even without learners, but // omitting this check would make it worse. Fixme. - if t := nextLeaseHolder.GetType(); t != roachpb.ReplicaType_Voter { + if t := nextLeaseHolder.GetType(); t != roachpb.ReplicaType_VoterFull { return nil, nil, errors.Errorf(`cannot transfer lease to replica of type %s`, t) } From 4dccd3c439178b79d7381b2c1f7851add4c1e26a Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 16:48:17 +0200 Subject: [PATCH 08/23] roachpb: return "desired" voters from ReplicaDescriptors.Voters Previous commits introduced (yet unused) voter types to encode joint consensus configurations which occur during atomic replication changes. Access to the slice of replicas is unfortunately common, though at least it's compartmentalized via the getters Voters() and Learners(). The main problem solved in this commit is figuring out what should be returned from Voters(): is it all VoterX types, or only voters in one of the two majority configs part of a joint quorum? The useful answer is returning the set of voters corresponding to what the config will be once the joint state is exited; this happens to be what most callers care about. Incoming and full voters are really the same thing in our code; we just need to distinguish them from outgoing voters to correctly maintain the quorum sizes. Of course there are some callers that do care about quorum sizes, and a number of cleanups were made for them. This commit also adds a ReplicaDescriptors.ConfState helper which is then used in all of the places that were previously cobbling together a ConfState manually. Release note: None --- pkg/roachpb/metadata_replicas.go | 125 +++++++++++-- pkg/roachpb/metadata_replicas_test.go | 241 ++++++++++++++++++++++++-- pkg/storage/client_test.go | 19 +- pkg/storage/helpers_test.go | 9 - pkg/storage/replica_metrics.go | 9 +- pkg/storage/replica_raftstorage.go | 25 +-- pkg/storage/replicate_queue.go | 17 +- 7 files changed, 372 insertions(+), 73 deletions(-) diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index 629214d5f16d..5cc0aea73c2f 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -13,6 +13,11 @@ package roachpb import ( "fmt" "strings" + + "go.etcd.io/etcd/raft/confchange" + "go.etcd.io/etcd/raft/quorum" + "go.etcd.io/etcd/raft/raftpb" + "go.etcd.io/etcd/raft/tracker" ) // ReplicaTypeVoterFull returns a ReplicaType_VoterFull pointer suitable for use in a @@ -22,6 +27,20 @@ func ReplicaTypeVoterFull() *ReplicaType { return &t } +// ReplicaTypeVoterIncoming returns a ReplicaType_VoterIncoming pointer suitable +// for use in a nullable proto field. +func ReplicaTypeVoterIncoming() *ReplicaType { + t := ReplicaType_VoterIncoming + return &t +} + +// ReplicaTypeVoterOutgoing returns a ReplicaType_VoterOutgoing pointer suitable +// for use in a nullable proto field. +func ReplicaTypeVoterOutgoing() *ReplicaType { + t := ReplicaType_VoterOutgoing + return &t +} + // ReplicaTypeLearner returns a ReplicaType_Learner pointer suitable for use in // a nullable proto field. func ReplicaTypeLearner() *ReplicaType { @@ -69,9 +88,19 @@ func (d ReplicaDescriptors) All() []ReplicaDescriptor { return d.wrapped } -// Voters returns the voter replicas in the set. This may allocate, but it also -// may return the underlying slice as a performance optimization, so it's not -// safe to modify the returned value. +// Voters returns the current and future voter replicas in the set. This means +// that during an atomic replication change, only the replicas that will be +// voters once the change completes will be returned; "outgoing" voters will not +// be returned even though they do in the current state retain their voting +// rights. When no atomic membership change is ongoing, this is simply the set +// of all non-learners. +// +// This may allocate, but it also may return the underlying slice as a +// performance optimization, so it's not safe to modify the returned value. +// +// TODO(tbg): go through the callers and figure out the few which want a +// different subset of voters. Consider renaming this method so that it's +// more descriptive. func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { // Fastpath, most of the time, everything is a voter, so special case that and // save the alloc. @@ -87,7 +116,8 @@ func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { } voters := make([]ReplicaDescriptor, 0, len(d.wrapped)) for i := range d.wrapped { - if d.wrapped[i].GetType() == ReplicaType_VoterFull { + switch d.wrapped[i].GetType() { + case ReplicaType_VoterFull, ReplicaType_VoterIncoming: voters = append(voters, d.wrapped[i]) } } @@ -238,14 +268,85 @@ func (d *ReplicaDescriptors) RemoveReplica( return removed, true } -// QuorumSize returns the number of voter replicas required for quorum in a raft -// group consisting of this set of replicas. -func (d ReplicaDescriptors) QuorumSize() int { - var numVoters int - for i := range d.wrapped { - if d.wrapped[i].GetType() == ReplicaType_VoterFull { - numVoters++ +// InAtomicReplicationChange returns true if the descriptor is in the middle of +// an atomic replication change. +func (d ReplicaDescriptors) InAtomicReplicationChange() bool { + for _, rDesc := range d.wrapped { + switch rDesc.GetType() { + case ReplicaType_VoterFull: + case ReplicaType_VoterIncoming: + return true + case ReplicaType_VoterOutgoing: + return true + case ReplicaType_Learner: + default: + panic(fmt.Sprintf("unknown replica type %d", rDesc.GetType())) } } - return (numVoters / 2) + 1 + return false +} + +// ConfState returns the Raft configuration described by the set of replicas. +func (d ReplicaDescriptors) ConfState() raftpb.ConfState { + var cs raftpb.ConfState + joint := d.InAtomicReplicationChange() + // The incoming config is taken verbatim from the full voters when the config is not + // joint. If it is joint, slot the voters into the right category. + // We never need to populate LearnersNext because this would correspond to + // demoting a voter, which we don't do. If we wanted to add that, we'd add + // ReplicaType_VoterDemoting and populate both VotersOutgoing and LearnersNext from it. + for _, rep := range d.wrapped { + id := uint64(rep.ReplicaID) + typ := rep.GetType() + switch typ { + case ReplicaType_VoterFull: + cs.Voters = append(cs.Voters, id) + if joint { + cs.VotersOutgoing = append(cs.VotersOutgoing, id) + } + case ReplicaType_VoterIncoming: + cs.Voters = append(cs.Voters, id) + case ReplicaType_VoterOutgoing: + cs.VotersOutgoing = append(cs.VotersOutgoing, id) + case ReplicaType_Learner: + cs.Learners = append(cs.Learners, id) + default: + panic(fmt.Sprintf("unknown ReplicaType %d", typ)) + } + } + return cs +} + +// CanMakeProgress reports whether the given descriptors can make progress at the +// replication layer. This is more complicated than just counting the number +// of replicas due to the existence of joint quorums. +func (d ReplicaDescriptors) CanMakeProgress(liveFunc func(descriptor ReplicaDescriptor) bool) bool { + voters := d.Voters() + var c int + // Take the fast path when there are only "current and future" voters, i.e. + // no learners and no voters of type VoterOutgoing. The config may be joint, + // but the outgoing conf is subsumed by the incoming one. + if n := len(d.wrapped); len(voters) == n { + for _, rDesc := range voters { + if liveFunc(rDesc) { + c++ + } + } + return c >= n/2+1 + } + + // Slow path. For simplicity, don't try to duplicate the logic that already + // exists in raft. + cfg, _, err := confchange.Restore( + confchange.Changer{Tracker: tracker.MakeProgressTracker(1)}, + d.ConfState(), + ) + if err != nil { + panic(err) + } + votes := make(map[uint64]bool, len(d.wrapped)) + for _, rDesc := range d.wrapped { + votes[uint64(rDesc.ReplicaID)] = liveFunc(rDesc) + } + return cfg.Voters.VoteResult(votes) == quorum.VoteWon } diff --git a/pkg/roachpb/metadata_replicas_test.go b/pkg/roachpb/metadata_replicas_test.go index 21f7845a4a8b..3249ead10025 100644 --- a/pkg/roachpb/metadata_replicas_test.go +++ b/pkg/roachpb/metadata_replicas_test.go @@ -13,31 +13,73 @@ package roachpb import ( "testing" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "go.etcd.io/etcd/raft" ) +func rd(typ *ReplicaType, id uint64) ReplicaDescriptor { + return ReplicaDescriptor{ + Type: typ, + NodeID: NodeID(100 * id), + StoreID: StoreID(10 * id), + ReplicaID: ReplicaID(id), + } +} + +var vn = (*ReplicaType)(nil) // should be treated like VoterFull +var v = ReplicaTypeVoterFull() +var vi = ReplicaTypeVoterIncoming() +var vo = ReplicaTypeVoterOutgoing() +var l = ReplicaTypeLearner() + func TestVotersLearnersAll(t *testing.T) { - voter := ReplicaTypeVoterFull() - learner := ReplicaTypeLearner() + tests := [][]ReplicaDescriptor{ {}, - {{Type: voter}}, - {{Type: nil}}, - {{Type: learner}}, - {{Type: voter}, {Type: learner}, {Type: voter}}, - {{Type: nil}, {Type: learner}, {Type: nil}}, - {{Type: learner}, {Type: voter}, {Type: learner}}, - {{Type: learner}, {Type: nil}, {Type: learner}}, + {rd(v, 1)}, + {rd(vn, 1)}, + {rd(l, 1)}, + {rd(v, 1), rd(l, 2), rd(v, 3)}, + {rd(vn, 1), rd(l, 2), rd(v, 3)}, + {rd(l, 1), rd(v, 2), rd(l, 3)}, + {rd(l, 1), rd(vn, 2), rd(l, 3)}, + {rd(vi, 1)}, + {rd(vo, 1)}, + {rd(l, 1), rd(vo, 2), rd(vi, 3), rd(vi, 4)}, } - for i, test := range tests { - r := MakeReplicaDescriptors(test) - for _, voter := range r.Voters() { - assert.Equal(t, ReplicaType_VoterFull, voter.GetType(), "testcase %d", i) - } - for _, learner := range r.Learners() { - assert.Equal(t, ReplicaType_Learner, learner.GetType(), "testcase %d", i) - } - assert.Equal(t, len(test), len(r.All()), "testcase %d", i) + for _, test := range tests { + t.Run("", func(t *testing.T) { + r := MakeReplicaDescriptors(test) + seen := map[ReplicaDescriptor]struct{}{} + for _, voter := range r.Voters() { + typ := voter.GetType() + switch typ { + case ReplicaType_VoterFull, ReplicaType_VoterIncoming: + seen[voter] = struct{}{} + default: + assert.FailNow(t, "unexpectedly got a %s as Voter()", typ) + } + } + for _, learner := range r.Learners() { + seen[learner] = struct{}{} + assert.Equal(t, ReplicaType_Learner, learner.GetType()) + } + + all := r.All() + // Make sure that VoterOutgoing is the only type that is skipped both + // by Learners() and Voters() + for _, rd := range all { + typ := rd.GetType() + if _, seen := seen[rd]; !seen { + assert.Equal(t, ReplicaType_VoterOutgoing, typ) + } else { + assert.NotEqual(t, ReplicaType_VoterOutgoing, typ) + } + } + assert.Equal(t, len(test), len(all)) + }) } } @@ -93,3 +135,166 @@ func TestReplicaDescriptorsRemove(t *testing.T) { } } } + +func TestReplicaDescriptorsConfState(t *testing.T) { + tests := []struct { + in []ReplicaDescriptor + out string + }{ + { + []ReplicaDescriptor{rd(v, 1)}, + "Voters:[1] VotersOutgoing:[] Learners:[] LearnersNext:[]", + }, + // Make sure nil is treated like VoterFull. + { + []ReplicaDescriptor{rd(vn, 1)}, + "Voters:[1] VotersOutgoing:[] Learners:[] LearnersNext:[]", + }, + { + []ReplicaDescriptor{rd(l, 1), rd(vn, 2)}, + "Voters:[2] VotersOutgoing:[] Learners:[1] LearnersNext:[]", + }, + // First joint case. We're adding n3 (via atomic replication changes), so the outgoing + // config we have to get rid of consists only of n2 (even though n2 remains a voter). + // Note that we could simplify this config so that it's not joint, but raft expects + // the config exactly as described by the descriptor so we don't try. + { + []ReplicaDescriptor{rd(l, 1), rd(v, 2), rd(vi, 3)}, + "Voters:[2 3] VotersOutgoing:[2] Learners:[1] LearnersNext:[]", + }, + // More complex joint change: a replica swap, switching out n4 for n3 from the initial + // set of voters n2, n4 (plus learner n1 before and after). + { + []ReplicaDescriptor{rd(l, 1), rd(v, 2), rd(vi, 3), rd(vo, 4)}, + "Voters:[2 3] VotersOutgoing:[2 4] Learners:[1] LearnersNext:[]", + }, + // Upreplicating from n1,n2 to n1,n2,n3,n4. + { + []ReplicaDescriptor{rd(v, 1), rd(v, 2), rd(vi, 3), rd(vi, 4)}, + "Voters:[1 2 3 4] VotersOutgoing:[1 2] Learners:[] LearnersNext:[]", + }, + // Downreplicating from n1,n2,n3,n4 to n1,n2. + { + []ReplicaDescriptor{rd(v, 1), rd(v, 2), rd(vo, 3), rd(vo, 4)}, + "Voters:[1 2] VotersOutgoing:[1 2 3 4] Learners:[] LearnersNext:[]", + }, + // Completely switching to a new set of replicas: n1,n2 to n4,n5. Throw a learner in for fun. + { + []ReplicaDescriptor{rd(vo, 1), rd(vo, 2), rd(vi, 3), rd(vi, 4), rd(l, 5)}, + "Voters:[3 4] VotersOutgoing:[1 2] Learners:[5] LearnersNext:[]", + }, + } + + for _, test := range tests { + t.Run("", func(t *testing.T) { + r := MakeReplicaDescriptors(test.in) + cs := r.ConfState() + require.Equal(t, test.out, raft.DescribeConfState(cs)) + }) + } +} + +func TestReplicaDescriptorsCanMakeProgress(t *testing.T) { + defer leaktest.AfterTest(t)() + + type descWithLiveness struct { + live bool + ReplicaDescriptor + } + + for _, test := range []struct { + rds []descWithLiveness + exp bool + }{ + // One out of one voter dead. + {[]descWithLiveness{{false, rd(v, 1)}}, false}, + // Three out of three voters dead. + {[]descWithLiveness{ + {false, rd(v, 1)}, + {false, rd(v, 2)}, + {false, rd(v, 3)}, + }, false}, + // Two out of three voters dead. + {[]descWithLiveness{ + {false, rd(v, 1)}, + {true, rd(v, 2)}, + {false, rd(v, 3)}, + }, false}, + // Two out of three voters alive. + {[]descWithLiveness{ + {true, rd(v, 1)}, + {false, rd(v, 2)}, + {true, rd(v, 3)}, + }, true}, + // Two out of three voters alive, but one is an incoming voter. (This + // still uses the fast path). + {[]descWithLiveness{ + {true, rd(v, 1)}, + {false, rd(v, 2)}, + {true, rd(vi, 3)}, + }, true}, + // Two out of three voters dead, and they're all incoming voters. (This + // can't happen in practice because it means there were zero voters prior + // to the conf change, but still this result is correct, similar to others + // below). + {[]descWithLiveness{ + {false, rd(vi, 1)}, + {false, rd(vi, 2)}, + {true, rd(vi, 3)}, + }, false}, + // Two out of three voters dead, and two are outgoing, one incoming. + {[]descWithLiveness{ + {false, rd(vi, 1)}, + {false, rd(vo, 2)}, + {true, rd(vo, 3)}, + }, false}, + // 1 and 3 are alive, but that's not a quorum for (1 3)&&(2 3) which is + // the config here. + {[]descWithLiveness{ + {true, rd(vi, 1)}, + {false, rd(vo, 2)}, + {true, rd(v, 3)}, + }, false}, + // Same as above, but all three alive. + {[]descWithLiveness{ + {true, rd(vi, 1)}, + {true, rd(vo, 2)}, + {true, rd(v, 3)}, + }, true}, + // Same, but there are a few learners that should not matter. + {[]descWithLiveness{ + {true, rd(vi, 1)}, + {true, rd(vo, 2)}, + {true, rd(v, 3)}, + {false, rd(l, 4)}, + {false, rd(l, 5)}, + {false, rd(l, 6)}, + {false, rd(l, 7)}, + }, true}, + // Non-joint case that should be live unless the learner is somehow taken + // into account. + {[]descWithLiveness{ + {true, rd(v, 1)}, + {true, rd(v, 2)}, + {false, rd(v, 4)}, + {false, rd(l, 4)}, + }, true}, + } { + t.Run("", func(t *testing.T) { + rds := make([]ReplicaDescriptor, 0, len(test.rds)) + for _, rDesc := range test.rds { + rds = append(rds, rDesc.ReplicaDescriptor) + } + + act := MakeReplicaDescriptors(rds).CanMakeProgress(func(rd ReplicaDescriptor) bool { + for _, rdi := range test.rds { + if rdi.ReplicaID == rd.ReplicaID { + return rdi.live + } + } + return false + }) + require.Equal(t, test.exp, act, "input: %+v", test) + }) + } +} diff --git a/pkg/storage/client_test.go b/pkg/storage/client_test.go index 173cfbf9477d..158eb62a1893 100644 --- a/pkg/storage/client_test.go +++ b/pkg/storage/client_test.go @@ -61,6 +61,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/kr/pretty" "github.com/pkg/errors" + "github.com/stretchr/testify/require" "go.etcd.io/etcd/raft" "google.golang.org/grpc" ) @@ -212,12 +213,18 @@ func createTestStoreWithOpts( } // Wait for the store's single range to have quorum before proceeding. repl := store.LookupReplica(roachpb.RKeyMin) - testutils.SucceedsSoon(t, func() error { - if !repl.HasQuorum() { - return errors.New("first range has not reached quorum") - } - return nil - }) + + // Send a request through the range to make sure everything is warmed up + // and works. + // NB: it's unclear if this code is necessary. + var ba roachpb.BatchRequest + get := roachpb.GetRequest{} + get.Key = keys.LocalMax + ba.Header.Replica = repl.Desc().Replicas().Voters()[0] + ba.Header.RangeID = repl.RangeID + ba.Add(&get) + _, pErr := store.Send(ctx, ba) + require.NoError(t, pErr.GoError()) // Wait for the system config to be available in gossip. All sorts of things // might not work properly while the system config is not available. diff --git a/pkg/storage/helpers_test.go b/pkg/storage/helpers_test.go index 3a61802fdc66..3e3530971c45 100644 --- a/pkg/storage/helpers_test.go +++ b/pkg/storage/helpers_test.go @@ -346,15 +346,6 @@ func (r *Replica) IsRaftGroupInitialized() bool { return r.mu.internalRaftGroup != nil } -// HasQuorum returns true iff the range that this replica is part of -// can achieve quorum. -func (r *Replica) HasQuorum() bool { - desc := r.Desc() - liveReplicas, _ := r.store.allocator.storePool.liveAndDeadReplicas(desc.RangeID, desc.InternalReplicas) - quorum := computeQuorum(len(desc.InternalReplicas)) - return len(liveReplicas) >= quorum -} - // GetStoreList exposes getStoreList for testing only, but with a hardcoded // storeFilter of storeFilterNone. func (sp *StorePool) GetStoreList(rangeID roachpb.RangeID) (StoreList, int, int) { diff --git a/pkg/storage/replica_metrics.go b/pkg/storage/replica_metrics.go index 86efe773965b..5b7cb165ba85 100644 --- a/pkg/storage/replica_metrics.go +++ b/pkg/storage/replica_metrics.go @@ -168,11 +168,12 @@ func calcRangeCounter( // We also compute an estimated per-range count of under-replicated and // unavailable ranges for each range based on the liveness table. if rangeCounter { - liveVoterReplicas := calcLiveVoterReplicas(desc, livenessMap) - if liveVoterReplicas < desc.Replicas().QuorumSize() { - unavailable = true - } + unavailable = !desc.Replicas().CanMakeProgress(func(rDesc roachpb.ReplicaDescriptor) bool { + _, live := livenessMap[rDesc.NodeID] + return live + }) needed := GetNeededReplicas(numReplicas, clusterNodes) + liveVoterReplicas := calcLiveVoterReplicas(desc, livenessMap) if needed > liveVoterReplicas { underreplicated = true } else if needed < liveVoterReplicas { diff --git a/pkg/storage/replica_raftstorage.go b/pkg/storage/replica_raftstorage.go index 3883089d41ae..bf3a7bf6873a 100644 --- a/pkg/storage/replica_raftstorage.go +++ b/pkg/storage/replica_raftstorage.go @@ -65,14 +65,7 @@ func (r *replicaRaftStorage) InitialState() (raftpb.HardState, raftpb.ConfState, if raft.IsEmptyHardState(hs) || err != nil { return raftpb.HardState{}, raftpb.ConfState{}, err } - var cs raftpb.ConfState - for _, rep := range r.mu.state.Desc.Replicas().Voters() { - cs.Voters = append(cs.Voters, uint64(rep.ReplicaID)) - } - for _, rep := range r.mu.state.Desc.Replicas().Learners() { - cs.Learners = append(cs.Learners, uint64(rep.ReplicaID)) - } - + cs := r.mu.state.Desc.Replicas().ConfState() return hs, cs, nil } @@ -533,15 +526,6 @@ func snapshot( return OutgoingSnapshot{}, err } - // Synthesize our raftpb.ConfState from desc. - var cs raftpb.ConfState - for _, rep := range desc.Replicas().Voters() { - cs.Voters = append(cs.Voters, uint64(rep.ReplicaID)) - } - for _, rep := range desc.Replicas().Learners() { - cs.Learners = append(cs.Learners, uint64(rep.ReplicaID)) - } - term, err := term(ctx, rsl, snap, rangeID, eCache, appliedIndex) if err != nil { return OutgoingSnapshot{}, errors.Errorf("failed to fetch term of %d: %s", appliedIndex, err) @@ -575,9 +559,10 @@ func snapshot( RaftSnap: raftpb.Snapshot{ Data: snapUUID.GetBytes(), Metadata: raftpb.SnapshotMetadata{ - Index: appliedIndex, - Term: term, - ConfState: cs, + Index: appliedIndex, + Term: term, + // Synthesize our raftpb.ConfState from desc. + ConfState: desc.Replicas().ConfState(), }, }, snapType: snapType, diff --git a/pkg/storage/replicate_queue.go b/pkg/storage/replicate_queue.go index 64b50204ca07..b98a4455c453 100644 --- a/pkg/storage/replicate_queue.go +++ b/pkg/storage/replicate_queue.go @@ -299,11 +299,20 @@ func (rq *replicateQueue) processOneChange( liveVoterReplicas, deadVoterReplicas := rq.allocator.storePool.liveAndDeadReplicas( desc.RangeID, voterReplicas) { - quorum := desc.Replicas().QuorumSize() - if lr := len(liveVoterReplicas); lr < quorum { + unavailable := !desc.Replicas().CanMakeProgress(func(rDesc roachpb.ReplicaDescriptor) bool { + for _, inner := range liveVoterReplicas { + if inner.ReplicaID == rDesc.ReplicaID { + return true + } + } + return false + }) + if unavailable { return false, newQuorumError( - "range requires a replication change, but lacks a quorum of live replicas (%d/%d)", - lr, quorum) + "range requires a replication change, but live replicas %v don't constitute a quorum for %v:", + liveVoterReplicas, + desc.Replicas().All(), + ) } } From b14c4546c24a871b85b181936d6f055dcd9c718d Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 17:52:57 +0200 Subject: [PATCH 09/23] roachpb: support zero-change ChangeReplicasTrigger We will use a ChangeReplicasTrigger without additions and removals when transitioning out of a joint configuration, so make sure it supports this properly. Release note: None --- pkg/roachpb/data.go | 28 ++++++++++++++++++++-- pkg/roachpb/data.pb.go | 52 ++++++++++++++++++++++------------------ pkg/roachpb/data.proto | 6 +++++ pkg/roachpb/data_test.go | 14 ++++++++++- 4 files changed, 74 insertions(+), 26 deletions(-) diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 9e02e5b93784..1dd30a91f730 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1335,6 +1335,18 @@ func writeTooOldRetryTimestamp(txn *Transaction, err *WriteTooOldError) hlc.Time var _ fmt.Stringer = &ChangeReplicasTrigger{} +// EnterJoint is true if applying this trigger will result in a joint +// configuration. +func (crt ChangeReplicasTrigger) EnterJoint() bool { + return len(crt.Added())+len(crt.Removed()) > 1 +} + +// LeaveJoint is true if applying this trigger will actively leave a joint +// configuration. +func (crt ChangeReplicasTrigger) LeaveJoint() bool { + return len(crt.Added())+len(crt.Removed()) == 0 +} + func (crt ChangeReplicasTrigger) String() string { var nextReplicaID ReplicaID var afterReplicas []ReplicaDescriptor @@ -1351,6 +1363,14 @@ func (crt ChangeReplicasTrigger) String() string { afterReplicas = crt.DeprecatedUpdatedReplicas } var chgS strings.Builder + if crt.LeaveJoint() { + // TODO(tbg): could list the replicas that will actually leave the + // voter set. + fmt.Fprintf(&chgS, "LEAVE_JOINT") + } + if crt.EnterJoint() { + fmt.Fprintf(&chgS, "ENTER_JOINT ") + } if len(added) > 0 { fmt.Fprintf(&chgS, "%s%s", ADD_REPLICA, added) } @@ -1364,9 +1384,13 @@ func (crt ChangeReplicasTrigger) String() string { return chgS.String() } +func (crt ChangeReplicasTrigger) legacy() bool { + return len(crt.InternalAddedReplicas)+len(crt.InternalRemovedReplicas) == 0 && crt.DeprecatedReplica != (ReplicaDescriptor{}) +} + // Added returns the replicas added by this change (if there are any). func (crt ChangeReplicasTrigger) Added() []ReplicaDescriptor { - if len(crt.InternalAddedReplicas)+len(crt.InternalRemovedReplicas) == 0 && crt.DeprecatedChangeType == ADD_REPLICA { + if crt.legacy() && crt.DeprecatedChangeType == ADD_REPLICA { return []ReplicaDescriptor{crt.DeprecatedReplica} } return crt.InternalAddedReplicas @@ -1374,7 +1398,7 @@ func (crt ChangeReplicasTrigger) Added() []ReplicaDescriptor { // Removed returns the replicas removed by this change (if there are any). func (crt ChangeReplicasTrigger) Removed() []ReplicaDescriptor { - if len(crt.InternalAddedReplicas)+len(crt.InternalRemovedReplicas) == 0 && crt.DeprecatedChangeType == REMOVE_REPLICA { + if crt.legacy() && crt.DeprecatedChangeType == REMOVE_REPLICA { return []ReplicaDescriptor{crt.DeprecatedReplica} } return crt.InternalRemovedReplicas diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 447a44003d3c..b4c73954a117 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -87,7 +87,7 @@ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } func (ValueType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{0} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -111,7 +111,7 @@ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{1} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -163,7 +163,7 @@ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } func (TransactionStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{2} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{2} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -180,7 +180,7 @@ type Span struct { func (m *Span) Reset() { *m = Span{} } func (*Span) ProtoMessage() {} func (*Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{0} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -232,7 +232,7 @@ func (m *Value) Reset() { *m = Value{} } func (m *Value) String() string { return proto.CompactTextString(m) } func (*Value) ProtoMessage() {} func (*Value) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{1} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -268,7 +268,7 @@ func (m *KeyValue) Reset() { *m = KeyValue{} } func (m *KeyValue) String() string { return proto.CompactTextString(m) } func (*KeyValue) ProtoMessage() {} func (*KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{2} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -306,7 +306,7 @@ func (m *StoreIdent) Reset() { *m = StoreIdent{} } func (m *StoreIdent) String() string { return proto.CompactTextString(m) } func (*StoreIdent) ProtoMessage() {} func (*StoreIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{3} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -346,7 +346,7 @@ func (m *SplitTrigger) Reset() { *m = SplitTrigger{} } func (m *SplitTrigger) String() string { return proto.CompactTextString(m) } func (*SplitTrigger) ProtoMessage() {} func (*SplitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{4} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -394,7 +394,7 @@ func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } func (m *MergeTrigger) String() string { return proto.CompactTextString(m) } func (*MergeTrigger) ProtoMessage() {} func (*MergeTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{5} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -419,6 +419,12 @@ func (m *MergeTrigger) XXX_DiscardUnknown() { var xxx_messageInfo_MergeTrigger proto.InternalMessageInfo +// ChangeReplicasTrigger carries out a replication change. The Added() and Removed() +// methods return the replicas being added and removed, respectively. If more than +// one change is specified (i.e. len(Added())+len(Removed()) exceeds one), this +// initiates an atomic replication change in which the "removed" replicas are +// of type ReplicaType_VoterOutgoing. This joint configuration is left via another +// ChangeReplicasTrigger which does not specify any additions nor removals. type ChangeReplicasTrigger struct { // TODO(tbg): remove once we know that no trigger using this will ever be // applied (this will require something like #39182). @@ -453,7 +459,7 @@ type ChangeReplicasTrigger struct { func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } func (*ChangeReplicasTrigger) ProtoMessage() {} func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{6} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -495,7 +501,7 @@ func (m *ModifiedSpanTrigger) Reset() { *m = ModifiedSpanTrigger{} } func (m *ModifiedSpanTrigger) String() string { return proto.CompactTextString(m) } func (*ModifiedSpanTrigger) ProtoMessage() {} func (*ModifiedSpanTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{7} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -541,7 +547,7 @@ func (m *StickyBitTrigger) Reset() { *m = StickyBitTrigger{} } func (m *StickyBitTrigger) String() string { return proto.CompactTextString(m) } func (*StickyBitTrigger) ProtoMessage() {} func (*StickyBitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{8} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -580,7 +586,7 @@ func (m *InternalCommitTrigger) Reset() { *m = InternalCommitTrigger{} } func (m *InternalCommitTrigger) String() string { return proto.CompactTextString(m) } func (*InternalCommitTrigger) ProtoMessage() {} func (*InternalCommitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{9} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -649,7 +655,7 @@ func (m *ObservedTimestamp) Reset() { *m = ObservedTimestamp{} } func (m *ObservedTimestamp) String() string { return proto.CompactTextString(m) } func (*ObservedTimestamp) ProtoMessage() {} func (*ObservedTimestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{10} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -877,7 +883,7 @@ type Transaction struct { func (m *Transaction) Reset() { *m = Transaction{} } func (*Transaction) ProtoMessage() {} func (*Transaction) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{11} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -927,7 +933,7 @@ func (m *TransactionRecord) Reset() { *m = TransactionRecord{} } func (m *TransactionRecord) String() string { return proto.CompactTextString(m) } func (*TransactionRecord) ProtoMessage() {} func (*TransactionRecord) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{12} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -963,7 +969,7 @@ func (m *Intent) Reset() { *m = Intent{} } func (m *Intent) String() string { return proto.CompactTextString(m) } func (*Intent) ProtoMessage() {} func (*Intent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{13} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1000,7 +1006,7 @@ func (m *SequencedWrite) Reset() { *m = SequencedWrite{} } func (m *SequencedWrite) String() string { return proto.CompactTextString(m) } func (*SequencedWrite) ProtoMessage() {} func (*SequencedWrite) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{14} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{14} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1064,7 +1070,7 @@ type Lease struct { func (m *Lease) Reset() { *m = Lease{} } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{15} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{15} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1109,7 +1115,7 @@ func (m *AbortSpanEntry) Reset() { *m = AbortSpanEntry{} } func (m *AbortSpanEntry) String() string { return proto.CompactTextString(m) } func (*AbortSpanEntry) ProtoMessage() {} func (*AbortSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{16} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{16} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1173,7 +1179,7 @@ func (m *TxnCoordMeta) Reset() { *m = TxnCoordMeta{} } func (m *TxnCoordMeta) String() string { return proto.CompactTextString(m) } func (*TxnCoordMeta) ProtoMessage() {} func (*TxnCoordMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_data_cc95704a62d85cea, []int{17} + return fileDescriptor_data_c57cbb7ed0a496ea, []int{17} } func (m *TxnCoordMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6289,9 +6295,9 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_cc95704a62d85cea) } +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_c57cbb7ed0a496ea) } -var fileDescriptor_data_cc95704a62d85cea = []byte{ +var fileDescriptor_data_c57cbb7ed0a496ea = []byte{ // 2110 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcd, 0x6f, 0xdb, 0xc8, 0x15, 0x37, 0x4d, 0x4a, 0xa2, 0x9e, 0x3e, 0x4c, 0x4f, 0xe2, 0x44, 0x49, 0x50, 0x29, 0x55, 0x8a, diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index e20947a797be..487470a6e131 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -157,6 +157,12 @@ enum ReplicaChangeType { REMOVE_REPLICA = 1; } +// ChangeReplicasTrigger carries out a replication change. The Added() and Removed() +// methods return the replicas being added and removed, respectively. If more than +// one change is specified (i.e. len(Added())+len(Removed()) exceeds one), this +// initiates an atomic replication change in which the "removed" replicas are +// of type ReplicaType_VoterOutgoing. This joint configuration is left via another +// ChangeReplicasTrigger which does not specify any additions nor removals. message ChangeReplicasTrigger { option (gogoproto.equal) = true; diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 9e251d32f772..1e9bccbaf5a8 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1656,7 +1656,19 @@ func TestChangeReplicasTrigger_String(t *testing.T) { GenerationComparable: proto.Bool(true), }, } + require.True(t, crt.EnterJoint()) + require.False(t, crt.LeaveJoint()) act := crt.String() - exp := "ADD_REPLICA[(n1,s2):3LEARNER], REMOVE_REPLICA[(n4,s5):6]: after=[(n1,s2):3LEARNER (n4,s5):6 (n7,s8):9LEARNER] next=10" + exp := "ENTER_JOINT ADD_REPLICA[(n1,s2):3LEARNER], REMOVE_REPLICA[(n4,s5):6]: after=[(n1,s2):3LEARNER (n4,s5):6 (n7,s8):9LEARNER] next=10" + require.Equal(t, exp, act) + + crt.InternalRemovedReplicas = nil + crt.InternalAddedReplicas = nil + require.False(t, crt.EnterJoint()) + require.True(t, crt.LeaveJoint()) + act = crt.String() + require.Empty(t, crt.Added()) + require.Empty(t, crt.Removed()) + exp = "LEAVE_JOINT: after=[(n1,s2):3LEARNER (n4,s5):6 (n7,s8):9LEARNER] next=10" require.Equal(t, exp, act) } From 68423fecb90490e6d1a8dc8ad6fc3a3eec2b7314 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 17:55:18 +0200 Subject: [PATCH 10/23] storage: add atomic replication changes cluster setting This defaults to false, and won't have an effect unless the newly introduced cluster version is also active. Release note: None --- pkg/settings/cluster/cockroach_versions.go | 12 ++++++++++++ pkg/settings/cluster/versionkey_string.go | 5 +++-- pkg/storage/replica.go | 6 ++++++ 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/pkg/settings/cluster/cockroach_versions.go b/pkg/settings/cluster/cockroach_versions.go index 0677488c4b27..7284c3fa1685 100644 --- a/pkg/settings/cluster/cockroach_versions.go +++ b/pkg/settings/cluster/cockroach_versions.go @@ -43,6 +43,7 @@ const ( VersionLearnerReplicas VersionTopLevelForeignKeys VersionAtomicChangeReplicasTrigger + VersionAtomicChangeReplicas // Add new versions here (step one of two). @@ -520,6 +521,17 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: VersionAtomicChangeReplicasTrigger, Version: roachpb.Version{Major: 19, Minor: 1, Unstable: 8}, }, + { + // VersionAtomicChangeReplicas is https://github.com/cockroachdb/cockroach/pull/39936. + // + // It provides an implementation of (*Replica).ChangeReplicas that uses + // atomic replication changes. The corresponding cluster setting + // 'kv.atomic_replication_changes.enabled' provides a killswitch (i.e. + // no atomic replication changes will be scheduled when it is set to + // 'false'). + Key: VersionAtomicChangeReplicas, + Version: roachpb.Version{Major: 19, Minor: 1, Unstable: 9}, + }, // Add new versions here (step two of two). diff --git a/pkg/settings/cluster/versionkey_string.go b/pkg/settings/cluster/versionkey_string.go index 15cb3fa0899f..03adc824bd13 100644 --- a/pkg/settings/cluster/versionkey_string.go +++ b/pkg/settings/cluster/versionkey_string.go @@ -20,11 +20,12 @@ func _() { _ = x[VersionLearnerReplicas-9] _ = x[VersionTopLevelForeignKeys-10] _ = x[VersionAtomicChangeReplicasTrigger-11] + _ = x[VersionAtomicChangeReplicas-12] } -const _VersionKey_name = "Version2_1VersionUnreplicatedRaftTruncatedStateVersionSideloadedStorageNoReplicaIDVersion19_1VersionStart19_2VersionQueryTxnTimestampVersionStickyBitVersionParallelCommitsVersionGenerationComparableVersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTrigger" +const _VersionKey_name = "Version2_1VersionUnreplicatedRaftTruncatedStateVersionSideloadedStorageNoReplicaIDVersion19_1VersionStart19_2VersionQueryTxnTimestampVersionStickyBitVersionParallelCommitsVersionGenerationComparableVersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicas" -var _VersionKey_index = [...]uint16{0, 10, 47, 82, 93, 109, 133, 149, 171, 198, 220, 246, 280} +var _VersionKey_index = [...]uint16{0, 10, 47, 82, 93, 109, 133, 149, 171, 198, 220, 246, 280, 307} func (i VersionKey) String() string { if i < 0 || i >= VersionKey(len(_VersionKey_index)-1) { diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index 6e9a9964b1a1..f72063cebb17 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -86,6 +86,12 @@ var disableSyncRaftLog = settings.RegisterBoolSetting( false, ) +var useAtomicReplicationChanges = settings.RegisterBoolSetting( + "kv.atomic_replication_changes.enabled", + "use atomic replication changes", + false, +) + // MaxCommandSizeFloor is the minimum allowed value for the MaxCommandSize // cluster setting. const MaxCommandSizeFloor = 4 << 20 // 4MB From 7d0e12ee11ff4557809032cd00b86dd6e214040c Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 18:09:56 +0200 Subject: [PATCH 11/23] storage: add fatals where atomic conf changes are unsupported These will be upgraded with proper handling when atomic replication changes are actually introduced, but for now it's convenient to stub out some code that will need to handle them and to make sure we won't forget to do so later. Release note: None --- pkg/storage/allocator.go | 42 +++++++++++++++++++++------------- pkg/storage/replica.go | 10 ++++++++ pkg/storage/replicate_queue.go | 2 ++ 3 files changed, 38 insertions(+), 16 deletions(-) diff --git a/pkg/storage/allocator.go b/pkg/storage/allocator.go index 904ef3739b47..25399942829e 100644 --- a/pkg/storage/allocator.go +++ b/pkg/storage/allocator.go @@ -47,14 +47,15 @@ const ( // algorithm. minReplicaWeight = 0.001 - // priorities for various repair operations. - removeLearnerReplicaPriority float64 = 12001 - addDeadReplacementPriority float64 = 12000 - addMissingReplicaPriority float64 = 10000 - addDecommissioningReplacementPriority float64 = 5000 - removeDeadReplicaPriority float64 = 1000 - removeDecommissioningReplicaPriority float64 = 200 - removeExtraReplicaPriority float64 = 100 + // Priorities for various repair operations. + finalizeAtomicReplicationChangePriority float64 = 12002 + removeLearnerReplicaPriority float64 = 12001 + addDeadReplacementPriority float64 = 12000 + addMissingReplicaPriority float64 = 10000 + addDecommissioningReplacementPriority float64 = 5000 + removeDeadReplicaPriority float64 = 1000 + removeDecommissioningReplicaPriority float64 = 200 + removeExtraReplicaPriority float64 = 100 ) // MinLeaseTransferStatsDuration configures the minimum amount of time a @@ -103,17 +104,19 @@ const ( AllocatorRemoveLearner AllocatorConsiderRebalance AllocatorRangeUnavailable + AllocatorFinalizeAtomicReplicationChange ) var allocatorActionNames = map[AllocatorAction]string{ - AllocatorNoop: "noop", - AllocatorRemove: "remove", - AllocatorAdd: "add", - AllocatorRemoveDead: "remove dead", - AllocatorRemoveDecommissioning: "remove decommissioning", - AllocatorRemoveLearner: "remove learner", - AllocatorConsiderRebalance: "consider rebalance", - AllocatorRangeUnavailable: "range unavailable", + AllocatorNoop: "noop", + AllocatorRemove: "remove", + AllocatorAdd: "add", + AllocatorRemoveDead: "remove dead", + AllocatorRemoveDecommissioning: "remove decommissioning", + AllocatorRemoveLearner: "remove learner", + AllocatorConsiderRebalance: "consider rebalance", + AllocatorRangeUnavailable: "range unavailable", + AllocatorFinalizeAtomicReplicationChange: "finalize conf change", } func (a AllocatorAction) String() string { @@ -299,6 +302,13 @@ func (a *Allocator) ComputeAction( return AllocatorNoop, 0 } + if desc.Replicas().InAtomicReplicationChange() { + // With a similar reasoning to the learner branch below, if we're in a + // joint configuration the top priority is to leave it before we can + // even think about doing anything else. + return AllocatorFinalizeAtomicReplicationChange, finalizeAtomicReplicationChangePriority + } + // Seeing a learner replica at this point is unexpected because learners are a // short-lived (ish) transient state in a learner+snapshot+voter cycle, which // is always done atomically. Only two places could have added a learner: the diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index f72063cebb17..9e6b527b3001 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -92,6 +92,16 @@ var useAtomicReplicationChanges = settings.RegisterBoolSetting( false, ) +// FatalAtomicReplicationChangeUnimplemented is called by code that will need +// to change when atomic replication changes are available to make sure it is +// updated accordingly. +// +// TODO(tbg): update all callers and remove this method in the commit that allows +// atomic replication changes. +func FatalAtomicReplicationChangeUnimplemented(ctx context.Context) { + log.Fatalf(ctx, "atomic configuration changes not yet implemented") +} + // MaxCommandSizeFloor is the minimum allowed value for the MaxCommandSize // cluster setting. const MaxCommandSizeFloor = 4 << 20 // 4MB diff --git a/pkg/storage/replicate_queue.go b/pkg/storage/replicate_queue.go index b98a4455c453..4d6735621372 100644 --- a/pkg/storage/replicate_queue.go +++ b/pkg/storage/replicate_queue.go @@ -346,6 +346,8 @@ func (rq *replicateQueue) processOneChange( return rq.removeLearner(ctx, repl, dryRun) case AllocatorConsiderRebalance: return rq.considerRebalance(ctx, repl, voterReplicas, canTransferLease, dryRun) + case AllocatorFinalizeAtomicReplicationChange: + FatalAtomicReplicationChangeUnimplemented(ctx) } return true, nil } From 75a231111ad57fba69bbc7838c9bdaa1d3911c6e Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 18:20:55 +0200 Subject: [PATCH 12/23] storage: add learners one by one Doing more than one change at once is going to force us into an atomic replication change. This isn't crazy, but seems unnecessary at this point, so just add the learners one by one. Release note: None --- pkg/storage/replica_command.go | 25 +++++++++++++++++-------- 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index 06fc81226cb2..6ff056c4688f 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -1017,24 +1017,33 @@ func addLearnerReplicas( details string, targets []roachpb.ReplicationTarget, ) (*roachpb.RangeDescriptor, error) { - newDesc := *desc - newDesc.SetReplicas(desc.Replicas().DeepCopy()) - var added []roachpb.ReplicaDescriptor + // TODO(tbg): we could add all learners in one go, but then we'd need to + // do it as an atomic replication change (raft doesn't know which config + // to apply the delta to, so we might be demoting more than one voter). + // This isn't crazy, we just need to transition out of the joint config + // before returning from this method, and it's unclear that it's worth + // doing. for _, target := range targets { + newDesc := *desc + newDesc.SetReplicas(desc.Replicas().DeepCopy()) + var added []roachpb.ReplicaDescriptor replDesc := roachpb.ReplicaDescriptor{ NodeID: target.NodeID, StoreID: target.StoreID, - ReplicaID: desc.NextReplicaID, + ReplicaID: newDesc.NextReplicaID, Type: roachpb.ReplicaTypeLearner(), } newDesc.NextReplicaID++ newDesc.AddReplica(replDesc) added = append(added, replDesc) + if err := execChangeReplicasTxn( + ctx, store, desc, &newDesc, reason, details, added, nil, /* removed */ + ); err != nil { + return nil, err + } + desc = &newDesc } - err := execChangeReplicasTxn( - ctx, store, desc, &newDesc, reason, details, added, nil, /* removed */ - ) - return &newDesc, err + return desc, nil } // finalizeChangeReplicas carries out the atomic membership change that finalizes From 7f54de673966f0b9852014e6fa14677c4e9c9c50 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 18:31:35 +0200 Subject: [PATCH 13/23] storage: un-embed decodedConfChange I ate a number of NPEs during development because nullable embedded fields are tricky; they hide the pointer derefs that often need a nil check. We'll embed the fields of decodedConfChange instead which works out better. This commit also adds the unmarshaling code necessary for ConfChangeV2 needed once we issue atomic replication changes. Release note: None --- pkg/storage/replica_application_cmd.go | 38 +++++++++++++------ .../replica_application_state_machine.go | 4 +- 2 files changed, 29 insertions(+), 13 deletions(-) diff --git a/pkg/storage/replica_application_cmd.go b/pkg/storage/replica_application_cmd.go index 7712eb826917..53e06bc37f92 100644 --- a/pkg/storage/replica_application_cmd.go +++ b/pkg/storage/replica_application_cmd.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/errors" opentracing "github.com/opentracing/opentracing-go" "go.etcd.io/etcd/raft/raftpb" ) @@ -79,15 +80,15 @@ type replicatedCmd struct { // decodedRaftEntry represents the deserialized content of a raftpb.Entry. type decodedRaftEntry struct { - idKey storagebase.CmdIDKey - raftCmd storagepb.RaftCommand - *decodedConfChange // only non-nil for config changes + idKey storagebase.CmdIDKey + raftCmd storagepb.RaftCommand + confChange *decodedConfChange // only non-nil for config changes } // decodedConfChange represents the fields of a config change raft command. type decodedConfChange struct { - cc raftpb.ConfChange - ccCtx ConfChangeContext + raftpb.ConfChangeI + ConfChangeContext } // decode decodes the entry e into the replicatedCmd. @@ -191,17 +192,32 @@ func (d *decodedRaftEntry) decodeNormalEntry(e *raftpb.Entry) error { } func (d *decodedRaftEntry) decodeConfChangeEntry(e *raftpb.Entry) error { - d.decodedConfChange = &decodedConfChange{} - if err := protoutil.Unmarshal(e.Data, &d.cc); err != nil { - return wrapWithNonDeterministicFailure(err, "while unmarshaling ConfChange") + d.confChange = &decodedConfChange{} + + switch e.Type { + case raftpb.EntryConfChange: + var cc raftpb.ConfChange + if err := protoutil.Unmarshal(e.Data, &cc); err != nil { + return wrapWithNonDeterministicFailure(err, "while unmarshaling ConfChange") + } + d.confChange.ConfChangeI = cc + case raftpb.EntryConfChangeV2: + var cc raftpb.ConfChangeV2 + if err := protoutil.Unmarshal(e.Data, &cc); err != nil { + return wrapWithNonDeterministicFailure(err, "while unmarshaling ConfChangeV2") + } + d.confChange.ConfChangeI = cc + default: + err := errors.New("unknown entry type") + return wrapWithNonDeterministicFailure(err, err.Error()) } - if err := protoutil.Unmarshal(d.cc.Context, &d.ccCtx); err != nil { + if err := protoutil.Unmarshal(d.confChange.AsV2().Context, &d.confChange.ConfChangeContext); err != nil { return wrapWithNonDeterministicFailure(err, "while unmarshaling ConfChangeContext") } - if err := protoutil.Unmarshal(d.ccCtx.Payload, &d.raftCmd); err != nil { + if err := protoutil.Unmarshal(d.confChange.Payload, &d.raftCmd); err != nil { return wrapWithNonDeterministicFailure(err, "while unmarshaling RaftCommand") } - d.idKey = storagebase.CmdIDKey(d.ccCtx.CommandID) + d.idKey = storagebase.CmdIDKey(d.confChange.CommandID) return nil } diff --git a/pkg/storage/replica_application_state_machine.go b/pkg/storage/replica_application_state_machine.go index 02f3ffb889d2..0f579702b410 100644 --- a/pkg/storage/replica_application_state_machine.go +++ b/pkg/storage/replica_application_state_machine.go @@ -1001,10 +1001,10 @@ func (sm *replicaStateMachine) maybeApplyConfChange(ctx context.Context, cmd *re case raftpb.EntryConfChange: if cmd.replicatedResult().ChangeReplicas == nil { // The command was rejected. - cmd.cc = raftpb.ConfChange{} + cmd.confChange.ConfChangeI = raftpb.ConfChange{} } return sm.r.withRaftGroup(true, func(raftGroup *raft.RawNode) (bool, error) { - raftGroup.ApplyConfChange(cmd.cc) + raftGroup.ApplyConfChange(cmd.confChange.ConfChangeI) return true, nil }) default: From 50ffa41e9ce38bd06803d1cfbd7ce565554e73a4 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 18:32:56 +0200 Subject: [PATCH 14/23] storage: skip ApplyConfChange on rejected entry When in a joint configuration, passing an empty conf change to ApplyConfChange doesn't do the right thing any more: it tells Raft that we're leaving the joint config. It's not a good idea to try to tell Raft anything about a ConfChange that got rejected. Raft internally knows that we handled it because it knows the applied index. This also adds a case match for ConfChangeV2 which is necessary to route atomic replication changes (ConfChangeV2). See https://github.com/etcd-io/etcd/pull/11046 Release note: None --- pkg/storage/replica_application_state_machine.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pkg/storage/replica_application_state_machine.go b/pkg/storage/replica_application_state_machine.go index 0f579702b410..11d95503b665 100644 --- a/pkg/storage/replica_application_state_machine.go +++ b/pkg/storage/replica_application_state_machine.go @@ -998,10 +998,11 @@ func (sm *replicaStateMachine) maybeApplyConfChange(ctx context.Context, cmd *re log.Fatalf(ctx, "unexpected replication change from command %s", &cmd.raftCmd) } return nil - case raftpb.EntryConfChange: + case raftpb.EntryConfChange, raftpb.EntryConfChangeV2: if cmd.replicatedResult().ChangeReplicas == nil { - // The command was rejected. - cmd.confChange.ConfChangeI = raftpb.ConfChange{} + // The command was rejected. There is no need to report a ConfChange + // to raft. + return nil } return sm.r.withRaftGroup(true, func(raftGroup *raft.RawNode) (bool, error) { raftGroup.ApplyConfChange(cmd.confChange.ConfChangeI) From fcf045225dac882b899a45d0ae5f87bbbd8ba8bd Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Wed, 21 Aug 2019 18:54:28 +0200 Subject: [PATCH 15/23] storage: push replication change unrolling into ChangeReplicas There are various callers to ChangeReplicas, so it makes more sense to unroll at that level. The code was updated to - in principle - do the right thing when atomic replication changes are requested, except that they are still unimplemented and a fatal error will serve as a reminder of that. Of course nothing issues them yet. Release note: None --- pkg/storage/replica.go | 18 +++--------------- pkg/storage/replica_command.go | 34 ++++++++++++++++++++++++++++++++++ 2 files changed, 37 insertions(+), 15 deletions(-) diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index 9e6b527b3001..2d84d179b663 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -1328,26 +1328,14 @@ func (r *Replica) executeAdminBatch( resp = &roachpb.AdminTransferLeaseResponse{} case *roachpb.AdminChangeReplicasRequest: - var err error - expDesc := &tArgs.ExpDesc chgs := tArgs.Changes() - for i := range chgs { - // Update expDesc to the outcome of the previous run to enable detection - // of concurrent updates while applying a series of changes. - // - // TODO(tbg): stop unrolling this once atomic replication changes are - // ready. Do any callers prefer unrolling though? We could add a flag. - expDesc, err = r.ChangeReplicas(ctx, expDesc, SnapshotRequest_REBALANCE, storagepb.ReasonAdminRequest, "", chgs[i:i+1]) - if err != nil { - break - } - } + desc, err := r.ChangeReplicas(ctx, &tArgs.ExpDesc, SnapshotRequest_REBALANCE, storagepb.ReasonAdminRequest, "", chgs) pErr = roachpb.NewError(err) - if err != nil { + if pErr != nil { resp = &roachpb.AdminChangeReplicasResponse{} } else { resp = &roachpb.AdminChangeReplicasResponse{ - Desc: *expDesc, + Desc: *desc, } } diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index 6ff056c4688f..22034aefce0c 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -898,6 +898,40 @@ func (r *Replica) ChangeReplicas( return nil, errors.Errorf("%s: the current RangeDescriptor must not be nil", r) } + // We execute the change serially if we're not allowed to run atomic replication changes + // or if that was explicitly disabled. We also unroll if learners are disabled because + // that's undertested and the expectation is that learners cannot be disabled in 19.2. + st := r.ClusterSettings() + unroll := !st.Version.IsActive(cluster.VersionAtomicChangeReplicas) || + !useAtomicReplicationChanges.Get(&st.SV) || + !useLearnerReplicas.Get(&st.SV) + + if unroll { + // Legacy behavior. + for i := range chgs { + var err error + desc, err = r.changeReplicasImpl(ctx, desc, priority, reason, details, chgs[i:i+1]) + if err != nil { + return nil, err + } + } + return desc, nil + } + if len(chgs) > 1 { + FatalAtomicReplicationChangeUnimplemented(ctx) + } + // Atomic replication change. + return r.changeReplicasImpl(ctx, desc, priority, reason, details, chgs) +} + +func (r *Replica) changeReplicasImpl( + ctx context.Context, + desc *roachpb.RangeDescriptor, + priority SnapshotRequest_Priority, + reason storagepb.RangeLogEventReason, + details string, + chgs roachpb.ReplicationChanges, +) (updatedDesc *roachpb.RangeDescriptor, _ error) { if len(chgs) != 1 { // TODO(tbg): lift this restriction when atomic membership changes are // plumbed into raft. From 957c6d3f0598dafad4270cd1b77d983f5f4e29be Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Thu, 22 Aug 2019 10:10:01 +0200 Subject: [PATCH 16/23] storage: emit ConfChangeV2 from ChangeReplicasTrigger where appropriate This prepares the trigger -> raft translation code to properly handle atomic replication changes. This carries out a lot of validation to give us confidence that any unusual transitions would be caught quickly. This change also establishes more clearly which added and removed replicas are to be passed into the trigger when transitioning into a joint configuration. For example, when adding a voter, one technically replaces a Learner with a VoterIncoming and so the question is which type the replica in the `added` slice should have. Picking the Learner would give the trigger the most power to validate the input, but it's annoying to have divergent descriptors floating around, so by convention we say that it is always the updated version of the descriptor (i.e. for fully removed replicas, just whatever it was before it disappeared). I spent more time on this than I'm willing to admit, in particular looking removing the redundancy here, but it made things more awkward than was worth it. Release note: None --- pkg/roachpb/data.go | 213 +++++++++++++++++++++++--- pkg/roachpb/data.pb.go | 60 ++++---- pkg/roachpb/data.proto | 14 +- pkg/roachpb/data_test.go | 230 ++++++++++++++++++++++++++-- pkg/storage/replica_proposal_buf.go | 32 +--- 5 files changed, 454 insertions(+), 95 deletions(-) diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 1dd30a91f730..9d4fde9c71c7 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/pkg/errors" + "go.etcd.io/etcd/raft/raftpb" ) var ( @@ -1333,20 +1334,175 @@ func writeTooOldRetryTimestamp(txn *Transaction, err *WriteTooOldError) hlc.Time return err.ActualTimestamp } -var _ fmt.Stringer = &ChangeReplicasTrigger{} +// Replicas returns all of the replicas present in the descriptor after this +// trigger applies. +func (crt ChangeReplicasTrigger) Replicas() []ReplicaDescriptor { + if crt.Desc != nil { + return crt.Desc.Replicas().All() + } + return crt.DeprecatedUpdatedReplicas +} -// EnterJoint is true if applying this trigger will result in a joint -// configuration. -func (crt ChangeReplicasTrigger) EnterJoint() bool { - return len(crt.Added())+len(crt.Removed()) > 1 +// ConfChange returns the configuration change described by the trigger. +func (crt ChangeReplicasTrigger) ConfChange(encodedCtx []byte) (raftpb.ConfChangeI, error) { + return confChangeImpl(crt, encodedCtx) } -// LeaveJoint is true if applying this trigger will actively leave a joint -// configuration. -func (crt ChangeReplicasTrigger) LeaveJoint() bool { - return len(crt.Added())+len(crt.Removed()) == 0 +func (crt ChangeReplicasTrigger) alwaysV2() bool { + // NB: we can return true in 20.1, but we don't win anything unless + // we are actively trying to migrate out of V1 membership changes, which + // could modestly simplify small areas of our codebase. + return false } +// confChangeImpl is the implementation of (ChangeReplicasTrigger).ConfChange +// narrowed down to the inputs it actually needs for better testability. +func confChangeImpl( + crt interface { + Added() []ReplicaDescriptor + Removed() []ReplicaDescriptor + Replicas() []ReplicaDescriptor + alwaysV2() bool + }, + encodedCtx []byte, +) (raftpb.ConfChangeI, error) { + added, removed, replicas := crt.Added(), crt.Removed(), crt.Replicas() + + var sl []raftpb.ConfChangeSingle + + checkExists := func(in ReplicaDescriptor) error { + for _, rDesc := range replicas { + if rDesc.ReplicaID == in.ReplicaID { + if a, b := in.GetType(), rDesc.GetType(); a != b { + return errors.Errorf("have %s, but descriptor has %s", in, rDesc) + } + return nil + } + } + return errors.Errorf("%s missing from descriptors %v", in, replicas) + } + checkNotExists := func(in ReplicaDescriptor) error { + for _, rDesc := range replicas { + if rDesc.ReplicaID == in.ReplicaID { + return errors.Errorf("%s must no longer be present in descriptor", in) + } + } + return nil + } + + for _, rDesc := range added { + // The incoming descriptor must also be present in the set of all + // replicas, which is ultimately the authoritative one because that's + // what's written to the KV store. + if err := checkExists(rDesc); err != nil { + return nil, err + } + + var changeType raftpb.ConfChangeType + switch rDesc.GetType() { + case ReplicaType_VoterFull: + // We're adding a new voter. + changeType = raftpb.ConfChangeAddNode + case ReplicaType_VoterIncoming: + // We're adding a voter, but will transition into a joint config + // first. + changeType = raftpb.ConfChangeAddNode + case ReplicaType_Learner: + // We're adding a new learner. Note that we're guaranteed by + // virtue of the upstream ChangeReplicas txn that this learner + // is not currently a voter. If we wanted to support that (i.e. + // demotions) we'd need to introduce a new + // ReplicaType_VoterDemoting for that purpose. + changeType = raftpb.ConfChangeAddLearnerNode + default: + return nil, errors.Errorf("can't add replica in state %v", rDesc.GetType()) + } + sl = append(sl, raftpb.ConfChangeSingle{ + Type: changeType, + NodeID: uint64(rDesc.ReplicaID), + }) + } + + for _, rDesc := range removed { + switch rDesc.GetType() { + case ReplicaType_VoterOutgoing: + // If a voter is removed through joint consensus, it will + // be turned into an outgoing voter first. + if err := checkExists(rDesc); err != nil { + return nil, err + } + case ReplicaType_VoterFull, ReplicaType_Learner: + // A learner or full voter can't be in the desc after. + if err := checkNotExists(rDesc); err != nil { + return nil, err + } + default: + return nil, errors.Errorf("can't remove replica in state %v", rDesc.GetType()) + } + sl = append(sl, raftpb.ConfChangeSingle{ + Type: raftpb.ConfChangeRemoveNode, + NodeID: uint64(rDesc.ReplicaID), + }) + } + + // Check whether we're entering a joint state. This is the case precisely when + // the resulting descriptors tells us that this is the case. Note that we've + // made sure above that all of the additions/removals are in tune with that + // descriptor already. + var enteringJoint bool + for _, rDesc := range replicas { + switch rDesc.GetType() { + case ReplicaType_VoterIncoming, ReplicaType_VoterOutgoing: + enteringJoint = true + default: + } + } + wantLeaveJoint := len(added)+len(removed) == 0 + if !enteringJoint { + if len(added)+len(removed) > 1 { + return nil, errors.Errorf("change requires joint consensus") + } + } else if wantLeaveJoint { + return nil, errors.Errorf("descriptor enters joint state, but trigger is requesting to leave one") + } + + var cc raftpb.ConfChangeI + + if enteringJoint || crt.alwaysV2() { + // V2 membership changes, which allow atomic replication changes. We + // track the joint state in the range descriptor and thus we need to be + // in charge of when to leave the joint state. + transition := raftpb.ConfChangeTransitionJointExplicit + if !enteringJoint { + // If we're using V2 just to avoid V1 (and not because we actually + // have a change that requires V2), then use an auto transition + // which skips the joint state. This is necessary: our descriptor + // says we're not supposed to go through one. + transition = raftpb.ConfChangeTransitionAuto + } + cc = raftpb.ConfChangeV2{ + Transition: transition, + Changes: sl, + Context: encodedCtx, + } + } else if wantLeaveJoint { + // Transitioning out of a joint config. + cc = raftpb.ConfChangeV2{ + Context: encodedCtx, + } + } else { + // Legacy path with exactly one change. + cc = raftpb.ConfChange{ + Type: sl[0].Type, + NodeID: sl[0].NodeID, + Context: encodedCtx, + } + } + return cc, nil +} + +var _ fmt.Stringer = &ChangeReplicasTrigger{} + func (crt ChangeReplicasTrigger) String() string { var nextReplicaID ReplicaID var afterReplicas []ReplicaDescriptor @@ -1363,19 +1519,27 @@ func (crt ChangeReplicasTrigger) String() string { afterReplicas = crt.DeprecatedUpdatedReplicas } var chgS strings.Builder - if crt.LeaveJoint() { - // TODO(tbg): could list the replicas that will actually leave the - // voter set. - fmt.Fprintf(&chgS, "LEAVE_JOINT") - } - if crt.EnterJoint() { - fmt.Fprintf(&chgS, "ENTER_JOINT ") + cc, err := crt.ConfChange(nil) + if err != nil { + fmt.Fprintf(&chgS, "", err) + } else { + ccv2 := cc.AsV2() + if ccv2.LeaveJoint() { + // NB: this isn't missing a trailing space. + // + // TODO(tbg): could list the replicas that will actually leave the + // voter set. + fmt.Fprintf(&chgS, "LEAVE_JOINT") + } + if _, ok := ccv2.EnterJoint(); ok { + fmt.Fprintf(&chgS, "ENTER_JOINT ") + } } if len(added) > 0 { fmt.Fprintf(&chgS, "%s%s", ADD_REPLICA, added) } if len(removed) > 0 { - if chgS.Len() > 0 { + if len(added) > 0 { chgS.WriteString(", ") } fmt.Fprintf(&chgS, "%s%s", REMOVE_REPLICA, removed) @@ -1384,22 +1548,25 @@ func (crt ChangeReplicasTrigger) String() string { return chgS.String() } -func (crt ChangeReplicasTrigger) legacy() bool { - return len(crt.InternalAddedReplicas)+len(crt.InternalRemovedReplicas) == 0 && crt.DeprecatedReplica != (ReplicaDescriptor{}) +func (crt ChangeReplicasTrigger) legacy() (ReplicaDescriptor, bool) { + if len(crt.InternalAddedReplicas)+len(crt.InternalRemovedReplicas) == 0 && crt.DeprecatedReplica.ReplicaID != 0 { + return crt.DeprecatedReplica, true + } + return ReplicaDescriptor{}, false } // Added returns the replicas added by this change (if there are any). func (crt ChangeReplicasTrigger) Added() []ReplicaDescriptor { - if crt.legacy() && crt.DeprecatedChangeType == ADD_REPLICA { - return []ReplicaDescriptor{crt.DeprecatedReplica} + if rDesc, ok := crt.legacy(); ok && crt.DeprecatedChangeType == ADD_REPLICA { + return []ReplicaDescriptor{rDesc} } return crt.InternalAddedReplicas } // Removed returns the replicas removed by this change (if there are any). func (crt ChangeReplicasTrigger) Removed() []ReplicaDescriptor { - if crt.legacy() && crt.DeprecatedChangeType == REMOVE_REPLICA { - return []ReplicaDescriptor{crt.DeprecatedReplica} + if rDesc, ok := crt.legacy(); ok && crt.DeprecatedChangeType == REMOVE_REPLICA { + return []ReplicaDescriptor{rDesc} } return crt.InternalRemovedReplicas } diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index b4c73954a117..8a74d10edcaa 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -87,7 +87,7 @@ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } func (ValueType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{0} + return fileDescriptor_data_01eb55a5b54e32dc, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -111,7 +111,7 @@ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{1} + return fileDescriptor_data_01eb55a5b54e32dc, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -163,7 +163,7 @@ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } func (TransactionStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{2} + return fileDescriptor_data_01eb55a5b54e32dc, []int{2} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -180,7 +180,7 @@ type Span struct { func (m *Span) Reset() { *m = Span{} } func (*Span) ProtoMessage() {} func (*Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{0} + return fileDescriptor_data_01eb55a5b54e32dc, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -232,7 +232,7 @@ func (m *Value) Reset() { *m = Value{} } func (m *Value) String() string { return proto.CompactTextString(m) } func (*Value) ProtoMessage() {} func (*Value) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{1} + return fileDescriptor_data_01eb55a5b54e32dc, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -268,7 +268,7 @@ func (m *KeyValue) Reset() { *m = KeyValue{} } func (m *KeyValue) String() string { return proto.CompactTextString(m) } func (*KeyValue) ProtoMessage() {} func (*KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{2} + return fileDescriptor_data_01eb55a5b54e32dc, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -306,7 +306,7 @@ func (m *StoreIdent) Reset() { *m = StoreIdent{} } func (m *StoreIdent) String() string { return proto.CompactTextString(m) } func (*StoreIdent) ProtoMessage() {} func (*StoreIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{3} + return fileDescriptor_data_01eb55a5b54e32dc, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -346,7 +346,7 @@ func (m *SplitTrigger) Reset() { *m = SplitTrigger{} } func (m *SplitTrigger) String() string { return proto.CompactTextString(m) } func (*SplitTrigger) ProtoMessage() {} func (*SplitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{4} + return fileDescriptor_data_01eb55a5b54e32dc, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -394,7 +394,7 @@ func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } func (m *MergeTrigger) String() string { return proto.CompactTextString(m) } func (*MergeTrigger) ProtoMessage() {} func (*MergeTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{5} + return fileDescriptor_data_01eb55a5b54e32dc, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -446,20 +446,22 @@ type ChangeReplicasTrigger struct { // TODO(jeffreyxiao): Remove deprecated_updated_replicas and // deprecated_next_replica_id in 20.1. Desc *RangeDescriptor `protobuf:"bytes,5,opt,name=desc,proto3" json:"desc,omitempty"` - // The new replicas added in this change (all reflected in desc, which is necessarily - // set when this field is used). Must not specify the same replica more than once or - // overlap with the removed replicas below. + // The new replicas added to the range descriptor in this change, exactly as + // they appear in the updated range descriptor. InternalAddedReplicas []ReplicaDescriptor `protobuf:"bytes,6,rep,name=internal_added_replicas,json=internalAddedReplicas,proto3" json:"internal_added_replicas"` - // The replicas removed in this change (all reflected in desc, which is necessarily - // set when this field is used). Must not specify the same replica more than once or - // overlap with the added replicas above. + // The replicas whose removal is being initiated in this change. If the + // replica is still present as an outgoing voter in the updated descriptor + // (i.e. if this is a full atomic replication change), then the replica here + // must match that in the descriptor; otherwise it must match the replica + // removed from the descriptor in the course of this change (which is itself + // not visible to this trigger). InternalRemovedReplicas []ReplicaDescriptor `protobuf:"bytes,7,rep,name=internal_removed_replicas,json=internalRemovedReplicas,proto3" json:"internal_removed_replicas"` } func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } func (*ChangeReplicasTrigger) ProtoMessage() {} func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{6} + return fileDescriptor_data_01eb55a5b54e32dc, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -501,7 +503,7 @@ func (m *ModifiedSpanTrigger) Reset() { *m = ModifiedSpanTrigger{} } func (m *ModifiedSpanTrigger) String() string { return proto.CompactTextString(m) } func (*ModifiedSpanTrigger) ProtoMessage() {} func (*ModifiedSpanTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{7} + return fileDescriptor_data_01eb55a5b54e32dc, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -547,7 +549,7 @@ func (m *StickyBitTrigger) Reset() { *m = StickyBitTrigger{} } func (m *StickyBitTrigger) String() string { return proto.CompactTextString(m) } func (*StickyBitTrigger) ProtoMessage() {} func (*StickyBitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{8} + return fileDescriptor_data_01eb55a5b54e32dc, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -586,7 +588,7 @@ func (m *InternalCommitTrigger) Reset() { *m = InternalCommitTrigger{} } func (m *InternalCommitTrigger) String() string { return proto.CompactTextString(m) } func (*InternalCommitTrigger) ProtoMessage() {} func (*InternalCommitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{9} + return fileDescriptor_data_01eb55a5b54e32dc, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -655,7 +657,7 @@ func (m *ObservedTimestamp) Reset() { *m = ObservedTimestamp{} } func (m *ObservedTimestamp) String() string { return proto.CompactTextString(m) } func (*ObservedTimestamp) ProtoMessage() {} func (*ObservedTimestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{10} + return fileDescriptor_data_01eb55a5b54e32dc, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -883,7 +885,7 @@ type Transaction struct { func (m *Transaction) Reset() { *m = Transaction{} } func (*Transaction) ProtoMessage() {} func (*Transaction) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{11} + return fileDescriptor_data_01eb55a5b54e32dc, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -933,7 +935,7 @@ func (m *TransactionRecord) Reset() { *m = TransactionRecord{} } func (m *TransactionRecord) String() string { return proto.CompactTextString(m) } func (*TransactionRecord) ProtoMessage() {} func (*TransactionRecord) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{12} + return fileDescriptor_data_01eb55a5b54e32dc, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -969,7 +971,7 @@ func (m *Intent) Reset() { *m = Intent{} } func (m *Intent) String() string { return proto.CompactTextString(m) } func (*Intent) ProtoMessage() {} func (*Intent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{13} + return fileDescriptor_data_01eb55a5b54e32dc, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1006,7 +1008,7 @@ func (m *SequencedWrite) Reset() { *m = SequencedWrite{} } func (m *SequencedWrite) String() string { return proto.CompactTextString(m) } func (*SequencedWrite) ProtoMessage() {} func (*SequencedWrite) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{14} + return fileDescriptor_data_01eb55a5b54e32dc, []int{14} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1070,7 +1072,7 @@ type Lease struct { func (m *Lease) Reset() { *m = Lease{} } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{15} + return fileDescriptor_data_01eb55a5b54e32dc, []int{15} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1115,7 +1117,7 @@ func (m *AbortSpanEntry) Reset() { *m = AbortSpanEntry{} } func (m *AbortSpanEntry) String() string { return proto.CompactTextString(m) } func (*AbortSpanEntry) ProtoMessage() {} func (*AbortSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{16} + return fileDescriptor_data_01eb55a5b54e32dc, []int{16} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1179,7 +1181,7 @@ func (m *TxnCoordMeta) Reset() { *m = TxnCoordMeta{} } func (m *TxnCoordMeta) String() string { return proto.CompactTextString(m) } func (*TxnCoordMeta) ProtoMessage() {} func (*TxnCoordMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_data_c57cbb7ed0a496ea, []int{17} + return fileDescriptor_data_01eb55a5b54e32dc, []int{17} } func (m *TxnCoordMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6295,9 +6297,9 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_c57cbb7ed0a496ea) } +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_01eb55a5b54e32dc) } -var fileDescriptor_data_c57cbb7ed0a496ea = []byte{ +var fileDescriptor_data_01eb55a5b54e32dc = []byte{ // 2110 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcd, 0x6f, 0xdb, 0xc8, 0x15, 0x37, 0x4d, 0x4a, 0xa2, 0x9e, 0x3e, 0x4c, 0x4f, 0xe2, 0x44, 0x49, 0x50, 0x29, 0x55, 0x8a, diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 487470a6e131..820396bd67d7 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -188,13 +188,15 @@ message ChangeReplicasTrigger { // TODO(jeffreyxiao): Remove deprecated_updated_replicas and // deprecated_next_replica_id in 20.1. RangeDescriptor desc = 5; - // The new replicas added in this change (all reflected in desc, which is necessarily - // set when this field is used). Must not specify the same replica more than once or - // overlap with the removed replicas below. + // The new replicas added to the range descriptor in this change, exactly as + // they appear in the updated range descriptor. repeated ReplicaDescriptor internal_added_replicas = 6 [(gogoproto.nullable) = false]; - // The replicas removed in this change (all reflected in desc, which is necessarily - // set when this field is used). Must not specify the same replica more than once or - // overlap with the added replicas above. + // The replicas whose removal is being initiated in this change. If the + // replica is still present as an outgoing voter in the updated descriptor + // (i.e. if this is a full atomic replication change), then the replica here + // must match that in the descriptor; otherwise it must match the replica + // removed from the descriptor in the course of this change (which is itself + // not visible to this trigger). repeated ReplicaDescriptor internal_removed_replicas = 7 [(gogoproto.nullable) = false]; } diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 1e9bccbaf5a8..56af173c8259 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -36,6 +36,7 @@ import ( "github.com/gogo/protobuf/proto" "github.com/kr/pretty" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/raft/raftpb" ) func makeTS(walltime int64, logical int32) hlc.Timestamp { @@ -1635,10 +1636,12 @@ func TestUpdateObservedTimestamps(t *testing.T) { func TestChangeReplicasTrigger_String(t *testing.T) { defer leaktest.AfterTest(t)() + vi := ReplicaType_VoterIncoming + vo := ReplicaType_VoterOutgoing l := ReplicaType_Learner - v := ReplicaType_VoterFull - repl1 := ReplicaDescriptor{NodeID: 1, StoreID: 2, ReplicaID: 3, Type: &l} - repl2 := ReplicaDescriptor{NodeID: 4, StoreID: 5, ReplicaID: 6, Type: &v} + repl1 := ReplicaDescriptor{NodeID: 1, StoreID: 2, ReplicaID: 3, Type: &vi} + repl2 := ReplicaDescriptor{NodeID: 4, StoreID: 5, ReplicaID: 6, Type: &vo} + learner := ReplicaDescriptor{NodeID: 7, StoreID: 8, ReplicaID: 9, Type: &l} crt := ChangeReplicasTrigger{ InternalAddedReplicas: []ReplicaDescriptor{repl1}, InternalRemovedReplicas: []ReplicaDescriptor{repl2}, @@ -1649,26 +1652,233 @@ func TestChangeReplicasTrigger_String(t *testing.T) { InternalReplicas: []ReplicaDescriptor{ repl1, repl2, - {NodeID: 7, StoreID: 8, ReplicaID: 9, Type: &l}, + learner, }, NextReplicaID: 10, Generation: proto.Int64(5), GenerationComparable: proto.Bool(true), }, } - require.True(t, crt.EnterJoint()) - require.False(t, crt.LeaveJoint()) act := crt.String() - exp := "ENTER_JOINT ADD_REPLICA[(n1,s2):3LEARNER], REMOVE_REPLICA[(n4,s5):6]: after=[(n1,s2):3LEARNER (n4,s5):6 (n7,s8):9LEARNER] next=10" + exp := "ENTER_JOINT ADD_REPLICA[(n1,s2):3VOTERINCOMING], REMOVE_REPLICA[(n4,s5):6VOTEROUTGOING]: after=[(n1,s2):3VOTERINCOMING (n4,s5):6VOTEROUTGOING (n7,s8):9LEARNER] next=10" require.Equal(t, exp, act) crt.InternalRemovedReplicas = nil crt.InternalAddedReplicas = nil - require.False(t, crt.EnterJoint()) - require.True(t, crt.LeaveJoint()) + repl1.Type = ReplicaTypeVoterFull() + crt.Desc.SetReplicas(MakeReplicaDescriptors([]ReplicaDescriptor{repl1, learner})) act = crt.String() require.Empty(t, crt.Added()) require.Empty(t, crt.Removed()) - exp = "LEAVE_JOINT: after=[(n1,s2):3LEARNER (n4,s5):6 (n7,s8):9LEARNER] next=10" + exp = "LEAVE_JOINT: after=[(n1,s2):3 (n7,s8):9LEARNER] next=10" require.Equal(t, exp, act) } + +type mockCRT struct { + v2 bool + ChangeReplicasTrigger +} + +func (m mockCRT) alwaysV2() bool { + return m.v2 +} + +func TestChangeReplicasTrigger_ConfChange(t *testing.T) { + defer leaktest.AfterTest(t)() + + sl := func(alt ...interface{}) []ReplicaDescriptor { + t.Helper() + if len(alt)%2 != 0 { + t.Fatal("need pairs") + } + var rDescs []ReplicaDescriptor + for i := 0; i < len(alt); i += 2 { + typ := alt[i].(ReplicaType) + id := alt[i+1].(int) + rDescs = append(rDescs, ReplicaDescriptor{ + Type: &typ, + NodeID: NodeID(3 * id), + StoreID: StoreID(2 * id), + ReplicaID: ReplicaID(id), + }) + } + return rDescs + } + + type in struct { + v2 bool + add, del, repls []ReplicaDescriptor + } + + mk := func(in in) mockCRT { + m := mockCRT{v2: in.v2} + m.ChangeReplicasTrigger.InternalAddedReplicas = in.add + m.ChangeReplicasTrigger.InternalRemovedReplicas = in.del + m.Desc = &RangeDescriptor{} + m.Desc.SetReplicas(MakeReplicaDescriptors(in.repls)) + return m + } + + vf1 := sl(ReplicaType_VoterFull, 1) + vo1 := sl(ReplicaType_VoterOutgoing, 1) + vi1 := sl(ReplicaType_VoterIncoming, 1) + vl1 := sl(ReplicaType_Learner, 1) + + testCases := []struct { + crt mockCRT + exp raftpb.ConfChangeI + err string + }{ + // A replica of type VoterOutgoing being added makes no sense. + {crt: mk(in{add: vo1, repls: vo1}), err: "can't add replica in state VoterOutgoing"}, + // But an incoming one can be added, and the result must be a joint change. + {crt: mk(in{add: vi1, repls: vi1}), exp: raftpb.ConfChangeV2{ + Transition: raftpb.ConfChangeTransitionJointExplicit, + Changes: []raftpb.ConfChangeSingle{{Type: raftpb.ConfChangeAddNode, NodeID: 1}}, + }}, + // A replica of type VoterIncoming being removed makes no sense. + {crt: mk(in{del: vi1}), err: "can't remove replica in state VoterIncoming"}, + // But during a joint removal we can see VoterOutgoing. + {crt: mk(in{del: vo1, repls: vo1}), exp: raftpb.ConfChangeV2{ + Transition: raftpb.ConfChangeTransitionJointExplicit, + Changes: []raftpb.ConfChangeSingle{{Type: raftpb.ConfChangeRemoveNode, NodeID: 1}}, + }}, + + // Adding a voter via the V1 path. + {crt: mk(in{add: vf1, repls: vf1}), exp: raftpb.ConfChange{ + Type: raftpb.ConfChangeAddNode, + NodeID: 1, + }}, + // Adding a learner via the V1 path. + {crt: mk(in{add: vl1, repls: vl1}), exp: raftpb.ConfChange{ + Type: raftpb.ConfChangeAddLearnerNode, + NodeID: 1, + }}, + + // Removing a voter or learner via the V1 path but falsely the replica is still in the descriptor. + {crt: mk(in{del: vf1, repls: vf1}), err: "(n3,s2):1 must no longer be present in descriptor"}, + {crt: mk(in{del: vl1, repls: vl1}), err: "(n3,s2):1LEARNER must no longer be present in descriptor"}, + // Well-formed examples. + {crt: mk(in{del: vf1}), exp: raftpb.ConfChange{ + Type: raftpb.ConfChangeRemoveNode, + NodeID: 1, + }}, + {crt: mk(in{del: vl1}), exp: raftpb.ConfChange{ + Type: raftpb.ConfChangeRemoveNode, + NodeID: 1, + }}, + // Adding a voter via the V2 path but without joint consensus. + {crt: mk(in{v2: true, add: vf1, repls: vf1}), exp: raftpb.ConfChangeV2{ + Transition: raftpb.ConfChangeTransitionAuto, + Changes: []raftpb.ConfChangeSingle{{ + Type: raftpb.ConfChangeAddNode, + NodeID: 1, + }}, + }}, + // Ditto, but with joint consensus requested. + {crt: mk(in{v2: true, add: vi1, repls: vi1}), exp: raftpb.ConfChangeV2{ + Transition: raftpb.ConfChangeTransitionJointExplicit, + Changes: []raftpb.ConfChangeSingle{{ + Type: raftpb.ConfChangeAddNode, + NodeID: 1, + }}, + }}, + + // Adding a learner via the V2 path and without joint consensus. (There is currently + // no way to request joint consensus when adding a single learner, but there is no + // reason one would ever want that). + {crt: mk(in{v2: true, add: vl1, repls: vl1}), exp: raftpb.ConfChangeV2{ + Transition: raftpb.ConfChangeTransitionAuto, + Changes: []raftpb.ConfChangeSingle{{ + Type: raftpb.ConfChangeAddLearnerNode, + NodeID: 1, + }}, + }}, + + // Removing a voter or learner via the V2 path without joint consensus. + // Note that this means that the replica is not in the desc any more. + {crt: mk(in{v2: true, del: vf1}), exp: raftpb.ConfChangeV2{ + Transition: raftpb.ConfChangeTransitionAuto, + Changes: []raftpb.ConfChangeSingle{{ + Type: raftpb.ConfChangeRemoveNode, + NodeID: 1, + }}, + }}, + {crt: mk(in{v2: true, del: vl1}), exp: raftpb.ConfChangeV2{ + Transition: raftpb.ConfChangeTransitionAuto, + Changes: []raftpb.ConfChangeSingle{{ + Type: raftpb.ConfChangeRemoveNode, + NodeID: 1, + }}, + }}, + + // Ditto but with joint consensus. (This can happen only with a voter; + // learners disappear immediately). + {crt: mk(in{v2: true, del: vo1, repls: vo1}), exp: raftpb.ConfChangeV2{ + Transition: raftpb.ConfChangeTransitionJointExplicit, + Changes: []raftpb.ConfChangeSingle{{ + Type: raftpb.ConfChangeRemoveNode, + NodeID: 1, + }}, + }}, + + // Run a more complex change (necessarily) via the V2 path. + {crt: mk(in{ + add: sl( // Additions. + ReplicaType_VoterIncoming, 6, ReplicaType_Learner, 4, ReplicaType_VoterIncoming, 3, + ), + del: sl( + // Removals. + ReplicaType_Learner, 2, ReplicaType_VoterOutgoing, 8, ReplicaType_VoterOutgoing, 9, + ), + repls: sl( + // Replicas. + ReplicaType_VoterFull, 1, + ReplicaType_VoterIncoming, 6, // added + ReplicaType_VoterIncoming, 3, // added + ReplicaType_VoterOutgoing, 9, // removing + ReplicaType_Learner, 4, // added + ReplicaType_VoterOutgoing, 8, // removing + ReplicaType_VoterFull, 10, + )}), + exp: raftpb.ConfChangeV2{ + Transition: raftpb.ConfChangeTransitionJointExplicit, + Changes: []raftpb.ConfChangeSingle{ + {NodeID: 6, Type: raftpb.ConfChangeAddNode}, + {NodeID: 4, Type: raftpb.ConfChangeAddLearnerNode}, + {NodeID: 3, Type: raftpb.ConfChangeAddNode}, + {NodeID: 2, Type: raftpb.ConfChangeRemoveNode}, + {NodeID: 8, Type: raftpb.ConfChangeRemoveNode}, + {NodeID: 9, Type: raftpb.ConfChangeRemoveNode}, + }}, + }, + + // Leave a joint config. + { + crt: mk(in{repls: sl(ReplicaType_VoterFull, 1)}), + exp: raftpb.ConfChangeV2{}, + }, + // If we're asked to leave a joint state but the descriptor is still joint, + // that's a problem. + { + crt: mk(in{v2: true, repls: sl(ReplicaType_VoterIncoming, 1)}), + err: "descriptor enters joint state, but trigger is requesting to leave one", + }, + { + crt: mk(in{v2: true, repls: sl(ReplicaType_VoterOutgoing, 1)}), + err: "descriptor enters joint state, but trigger is requesting to leave one", + }, + } + + for _, test := range testCases { + t.Run("", func(t *testing.T) { + cc, err := confChangeImpl(test.crt, nil /* payload */) + if test.err == "" { + require.NoError(t, err) + require.Equal(t, test.exp, cc) + } else { + require.EqualError(t, err, test.err) + } + }) + } +} diff --git a/pkg/storage/replica_proposal_buf.go b/pkg/storage/replica_proposal_buf.go index 17c336584f1e..dcf5c962f84f 100644 --- a/pkg/storage/replica_proposal_buf.go +++ b/pkg/storage/replica_proposal_buf.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/cockroachdb/errors" "go.etcd.io/etcd/raft" "go.etcd.io/etcd/raft/raftpb" ) @@ -445,35 +444,14 @@ func (b *propBuf) FlushLockedWithRaftGroup(raftGroup *raft.RawNode) error { continue } - added, removed := crt.Added(), crt.Removed() - if len(added)+len(removed) != 1 { - log.Fatalf(context.TODO(), "atomic replication changes not supported yet") - } - - var changeType raftpb.ConfChangeType - var replicaID roachpb.ReplicaID - if len(added) > 0 { - replicaID = added[0].ReplicaID - typ := added[0].GetType() - switch typ { - case roachpb.ReplicaType_VoterFull: - changeType = raftpb.ConfChangeAddNode - case roachpb.ReplicaType_Learner: - changeType = raftpb.ConfChangeAddLearnerNode - default: - panic(errors.Errorf("unknown replica type %v", typ)) - } - } else { - changeType, replicaID = raftpb.ConfChangeRemoveNode, removed[0].ReplicaID + cc, err := crt.ConfChange(encodedCtx) + if err != nil { + firstErr = err + continue } - confChange := raftpb.ConfChange{ - Type: changeType, - NodeID: uint64(replicaID), - Context: encodedCtx, - } if err := raftGroup.ProposeConfChange( - confChange, + cc, ); err != nil && err != raft.ErrProposalDropped { // Silently ignore dropped proposals (they were always silently // ignored prior to the introduction of ErrProposalDropped). From 3f5c708ff969edb5b8ea78ff823006fcaebf7037 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Thu, 22 Aug 2019 13:51:41 +0200 Subject: [PATCH 17/23] roachpb: auto-assign ReplicaID during AddReplica This is a cleanup leading up to a larger refactor of the contract around `execChangeReplicasTxn`. Release note: None --- pkg/kv/dist_sender_test.go | 25 +++++++++------------- pkg/roachpb/metadata.go | 31 +++++++++++++++++++++++++-- pkg/storage/replica_command.go | 39 +++++++++++++--------------------- 3 files changed, 54 insertions(+), 41 deletions(-) diff --git a/pkg/kv/dist_sender_test.go b/pkg/kv/dist_sender_test.go index d10ae9a218f7..167560122c07 100644 --- a/pkg/kv/dist_sender_test.go +++ b/pkg/kv/dist_sender_test.go @@ -310,9 +310,10 @@ func TestSendRPCOrder(t *testing.T) { } descriptor := roachpb.RangeDescriptor{ - StartKey: roachpb.RKeyMin, - EndKey: roachpb.RKeyMax, - RangeID: rangeID, + StartKey: roachpb.RKeyMin, + EndKey: roachpb.RKeyMax, + RangeID: rangeID, + NextReplicaID: 1, } for i := int32(1); i <= 5; i++ { addr := util.MakeUnresolvedAddr("tcp", fmt.Sprintf("node%d:1", i)) @@ -326,10 +327,7 @@ func TestSendRPCOrder(t *testing.T) { if err := g.AddInfoProto(gossip.MakeNodeIDKey(roachpb.NodeID(i)), nd, time.Hour); err != nil { t.Fatal(err) } - descriptor.AddReplica(roachpb.ReplicaDescriptor{ - NodeID: roachpb.NodeID(i), - StoreID: roachpb.StoreID(i), - }) + descriptor.AddReplica(roachpb.NodeID(i), roachpb.StoreID(i), roachpb.ReplicaType_VoterFull) } // Stub to be changed in each test case. @@ -1416,9 +1414,10 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { // Fill RangeDescriptor with three replicas. var descriptor = roachpb.RangeDescriptor{ - RangeID: 1, - StartKey: roachpb.RKey("a"), - EndKey: roachpb.RKey("z"), + RangeID: 1, + StartKey: roachpb.RKey("a"), + EndKey: roachpb.RKey("z"), + NextReplicaID: 1, } for i := 1; i <= 3; i++ { addr := util.MakeUnresolvedAddr("tcp", fmt.Sprintf("node%d", i)) @@ -1430,11 +1429,7 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { t.Fatal(err) } - descriptor.AddReplica(roachpb.ReplicaDescriptor{ - NodeID: roachpb.NodeID(i), - StoreID: roachpb.StoreID(i), - ReplicaID: roachpb.ReplicaID(i), - }) + descriptor.AddReplica(roachpb.NodeID(i), roachpb.StoreID(i), roachpb.ReplicaType_VoterFull) } descDB := mockRangeDescriptorDBForDescs( testMetaRangeDescriptor, diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index fc6248d62ed7..a8e373681afb 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -135,11 +135,38 @@ func (r *RangeDescriptor) SetReplicas(replicas ReplicaDescriptors) { r.InternalReplicas = replicas.AsProto() } -// AddReplica adds the given replica to this range's set. -func (r *RangeDescriptor) AddReplica(toAdd ReplicaDescriptor) { +// SetReplicaType changes the type of the replica with the given ID to the given +// type. Returns zero values if the replica was not found and the updated +// descriptor and true otherwise. +func (r *RangeDescriptor) SetReplicaType( + nodeID NodeID, storeID StoreID, typ ReplicaType, +) (ReplicaDescriptor, bool) { + for i := range r.InternalReplicas { + desc := &r.InternalReplicas[i] + if desc.StoreID == storeID && desc.NodeID == nodeID { + desc.Type = &typ + return *desc, true + } + } + return ReplicaDescriptor{}, false +} + +// AddReplica adds a replica on the given node and store with the supplied type. +// It auto-assigns a ReplicaID and returns the inserted ReplicaDescriptor. +func (r *RangeDescriptor) AddReplica( + nodeID NodeID, storeID StoreID, typ ReplicaType, +) ReplicaDescriptor { + toAdd := ReplicaDescriptor{ + NodeID: nodeID, + StoreID: storeID, + ReplicaID: r.NextReplicaID, + Type: &typ, + } rs := r.Replicas() rs.AddReplica(toAdd) r.SetReplicas(rs) + r.NextReplicaID++ + return toAdd } // RemoveReplica removes the matching replica from this range's set and returns diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index 22034aefce0c..543cd472f797 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -1060,15 +1060,9 @@ func addLearnerReplicas( for _, target := range targets { newDesc := *desc newDesc.SetReplicas(desc.Replicas().DeepCopy()) + replDesc := newDesc.AddReplica(target.NodeID, target.StoreID, roachpb.ReplicaType_Learner) + var added []roachpb.ReplicaDescriptor - replDesc := roachpb.ReplicaDescriptor{ - NodeID: target.NodeID, - StoreID: target.StoreID, - ReplicaID: newDesc.NextReplicaID, - Type: roachpb.ReplicaTypeLearner(), - } - newDesc.NextReplicaID++ - newDesc.AddReplica(replDesc) added = append(added, replDesc) if err := execChangeReplicasTxn( ctx, store, desc, &newDesc, reason, details, added, nil, /* removed */ @@ -1106,7 +1100,7 @@ func (r *Replica) finalizeChangeReplicas( for _, target := range adds { // All adds must be present as learners right now and they are removed, // upgraded, and then re-added. - rDesc, ok := updatedDesc.RemoveReplica(target.NodeID, target.StoreID) + rDesc, ok := updatedDesc.GetReplicaDescriptor(target.StoreID) if !ok { return nil, errors.Errorf("programming error: replica %v not found in %v", target, updatedDesc) } @@ -1115,8 +1109,7 @@ func (r *Replica) finalizeChangeReplicas( return nil, errors.Errorf("programming error: cannot promote replica of type %s", rDesc.Type) } - rDesc.Type = roachpb.ReplicaTypeVoterFull() - updatedDesc.AddReplica(rDesc) + rDesc, _ = updatedDesc.SetReplicaType(target.NodeID, target.StoreID, roachpb.ReplicaType_VoterFull) replsAdded = append(replsAdded, rDesc) // Note that raft snapshot queue will refuse to send a snapshot to a learner @@ -1217,10 +1210,11 @@ func (r *Replica) addReplicaLegacyPreemptiveSnapshot( return nil, errors.Errorf("%s: the current RangeDescriptor must not be nil", r) } - repDesc := roachpb.ReplicaDescriptor{ - NodeID: target.NodeID, - StoreID: target.StoreID, - } + updatedDesc := *desc + updatedDesc.SetReplicas(desc.Replicas().DeepCopy()) + + repDesc := updatedDesc.AddReplica(target.NodeID, target.StoreID, roachpb.ReplicaType_VoterFull) + repDescIdx := -1 // tracks NodeID && StoreID nodeUsed := false // tracks NodeID only for i, existingRep := range desc.Replicas().All() { @@ -1234,9 +1228,6 @@ func (r *Replica) addReplicaLegacyPreemptiveSnapshot( } } - updatedDesc := *desc - updatedDesc.SetReplicas(desc.Replicas().DeepCopy()) - // If the replica exists on the remote node, no matter in which store, // abort the replica add. if nodeUsed { @@ -1268,14 +1259,14 @@ func (r *Replica) addReplicaLegacyPreemptiveSnapshot( // operation is processed. This is important to allow other ranges to make // progress which might be required for this ChangeReplicas operation to // complete. See #10409. - if err := r.sendSnapshot(ctx, repDesc, SnapshotRequest_PREEMPTIVE, priority); err != nil { - return nil, err + { + preemptiveRepDesc := repDesc + preemptiveRepDesc.ReplicaID = 0 + if err := r.sendSnapshot(ctx, preemptiveRepDesc, SnapshotRequest_PREEMPTIVE, priority); err != nil { + return nil, err + } } - repDesc.ReplicaID = updatedDesc.NextReplicaID - updatedDesc.NextReplicaID++ - updatedDesc.AddReplica(repDesc) - added := []roachpb.ReplicaDescriptor{repDesc} err := execChangeReplicasTxn(ctx, r.store, desc, &updatedDesc, reason, details, added, nil /* removed */) return &updatedDesc, err From 385a15f0c7d148194d73efe834176a1fc7ff0a50 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Thu, 22 Aug 2019 15:40:51 +0200 Subject: [PATCH 18/23] storage: let execChangeReplicasTxn construct the descriptor Prior to this commit, the method took both an old and a new desc *plus* slices of added and removed replicas. This had grown organically, wasn't an easily understood interface, led to repetitive and tricky code at the callers, and most importantly isn't adequate any more in a world with atomic replication changes, where execChangeReplicasTxn in constructing the ChangeReplicasTrigger is essentially deciding whether a joint configuration needs to be entered (which in turn determines what the descriptor needs to look like in the first place). To start solving this, let execChangeReplicasTxn create (and on success return) the new descriptor. Callers instead pass in what they want to be done, which is accomplished via an []internalReplicationChange slice. Release note: None --- pkg/storage/replica_command.go | 213 +++++++++++++++++++-------------- 1 file changed, 120 insertions(+), 93 deletions(-) diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index 543cd472f797..dd5a2980458a 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -1058,18 +1058,14 @@ func addLearnerReplicas( // before returning from this method, and it's unclear that it's worth // doing. for _, target := range targets { - newDesc := *desc - newDesc.SetReplicas(desc.Replicas().DeepCopy()) - replDesc := newDesc.AddReplica(target.NodeID, target.StoreID, roachpb.ReplicaType_Learner) - - var added []roachpb.ReplicaDescriptor - added = append(added, replDesc) - if err := execChangeReplicasTxn( - ctx, store, desc, &newDesc, reason, details, added, nil, /* removed */ - ); err != nil { + iChgs := []internalReplicationChange{{target: target, typ: internalChangeTypeAddLearner}} + var err error + desc, err = execChangeReplicasTxn( + ctx, store, desc, reason, details, iChgs, + ) + if err != nil { return nil, err } - desc = &newDesc } return desc, nil } @@ -1092,26 +1088,21 @@ func (r *Replica) finalizeChangeReplicas( // this may want to detect that and retry, sending a snapshot and promoting // both sides. - updatedDesc := *desc - updatedDesc.SetReplicas(desc.Replicas().DeepCopy()) + iChgs := make([]internalReplicationChange, 0, len(chgs)) - adds, removes := chgs.Additions(), chgs.Removals() - var replsAdded, replsRemoved []roachpb.ReplicaDescriptor - for _, target := range adds { - // All adds must be present as learners right now and they are removed, - // upgraded, and then re-added. - rDesc, ok := updatedDesc.GetReplicaDescriptor(target.StoreID) + for _, target := range chgs.Additions() { + iChgs = append(iChgs, internalReplicationChange{target: target, typ: internalChangeTypePromoteLearner}) + // All adds must be present as learners right now, and we send them + // snapshots in anticipation of promoting them to voters. + rDesc, ok := desc.GetReplicaDescriptor(target.StoreID) if !ok { - return nil, errors.Errorf("programming error: replica %v not found in %v", target, updatedDesc) + return nil, errors.Errorf("programming error: replica %v not found in %v", target, desc) } if rDesc.GetType() != roachpb.ReplicaType_Learner { return nil, errors.Errorf("programming error: cannot promote replica of type %s", rDesc.Type) } - rDesc, _ = updatedDesc.SetReplicaType(target.NodeID, target.StoreID, roachpb.ReplicaType_VoterFull) - replsAdded = append(replsAdded, rDesc) - // Note that raft snapshot queue will refuse to send a snapshot to a learner // replica if its store is already sending a snapshot to that replica. That // races with this snapshot. Most of the time, this side will win the race, @@ -1142,16 +1133,11 @@ func (r *Replica) finalizeChangeReplicas( } } - for _, target := range removes { - rDesc, found := updatedDesc.RemoveReplica(target.NodeID, target.StoreID) - if !found { - return nil, errors.Errorf("target to remove %v not found in %s", target, &updatedDesc) - } - replsRemoved = append(replsRemoved, rDesc) + for _, target := range chgs.Removals() { + iChgs = append(iChgs, internalReplicationChange{target: target, typ: internalChangeTypeRemove}) } - err := execChangeReplicasTxn(ctx, r.store, desc, &updatedDesc, reason, details, replsAdded, replsRemoved) - return &updatedDesc, err + return execChangeReplicasTxn(ctx, r.store, desc, reason, details, iChgs) } // tryRollbackLearnerReplica attempts to remove a learner specified by the @@ -1165,10 +1151,8 @@ func (r *Replica) tryRollBackLearnerReplica( reason storagepb.RangeLogEventReason, details string, ) { - newDesc := *desc - newDesc.SetReplicas(desc.Replicas().DeepCopy()) - replDesc, ok := newDesc.RemoveReplica(target.NodeID, target.StoreID) - if !ok || replDesc.GetType() != roachpb.ReplicaType_Learner { + repDesc, ok := desc.GetReplicaDescriptor(target.StoreID) + if !ok || repDesc.GetType() != roachpb.ReplicaType_Learner { // There's no learner to roll back. log.Event(ctx, "learner to roll back not found; skipping") return @@ -1181,10 +1165,11 @@ func (r *Replica) tryRollBackLearnerReplica( const rollbackTimeout = 10 * time.Second rollbackFn := func(ctx context.Context) error { - removed := []roachpb.ReplicaDescriptor{replDesc} - return execChangeReplicasTxn( - ctx, r.store, desc, &newDesc, reason, details, nil /* added */, removed, + _, err := execChangeReplicasTxn( + ctx, r.store, desc, reason, details, + []internalReplicationChange{{target: target, typ: internalChangeTypeRemove}}, ) + return err } rollbackCtx := logtags.WithTags(context.Background(), logtags.FromContext(ctx)) if err := contextutil.RunWithTimeout( @@ -1194,7 +1179,7 @@ func (r *Replica) tryRollBackLearnerReplica( "failed to rollback learner %s, abandoning it for the replicate queue: %v", target, err) r.store.replicateQueue.MaybeAddAsync(ctx, r, r.store.Clock().Now()) } else { - log.Infof(ctx, "rolled back learner %s to %s", replDesc, &newDesc) + log.Infof(ctx, "rolled back learner %s in %s", target, desc) } } @@ -1210,31 +1195,16 @@ func (r *Replica) addReplicaLegacyPreemptiveSnapshot( return nil, errors.Errorf("%s: the current RangeDescriptor must not be nil", r) } - updatedDesc := *desc - updatedDesc.SetReplicas(desc.Replicas().DeepCopy()) - - repDesc := updatedDesc.AddReplica(target.NodeID, target.StoreID, roachpb.ReplicaType_VoterFull) - - repDescIdx := -1 // tracks NodeID && StoreID nodeUsed := false // tracks NodeID only - for i, existingRep := range desc.Replicas().All() { - nodeUsedByExistingRep := existingRep.NodeID == repDesc.NodeID + for _, existingRep := range desc.Replicas().All() { + nodeUsedByExistingRep := existingRep.NodeID == target.NodeID nodeUsed = nodeUsed || nodeUsedByExistingRep - - if nodeUsedByExistingRep && existingRep.StoreID == repDesc.StoreID { - repDescIdx = i - repDesc = existingRep - break - } } // If the replica exists on the remote node, no matter in which store, // abort the replica add. if nodeUsed { - if repDescIdx != -1 { - return nil, errors.Errorf("%s: unable to add replica %v which is already present", r, repDesc) - } - return nil, errors.Errorf("%s: unable to add replica %v; node already has a replica", r, repDesc) + return nil, errors.Errorf("%s: unable to add replica %v; node already has a replica", r, target) } // Send a pre-emptive snapshot. Note that the replica to which this @@ -1260,33 +1230,114 @@ func (r *Replica) addReplicaLegacyPreemptiveSnapshot( // progress which might be required for this ChangeReplicas operation to // complete. See #10409. { - preemptiveRepDesc := repDesc - preemptiveRepDesc.ReplicaID = 0 + preemptiveRepDesc := roachpb.ReplicaDescriptor{ + NodeID: target.NodeID, + StoreID: target.StoreID, + Type: roachpb.ReplicaTypeVoterFull(), + ReplicaID: 0, // intentional + } if err := r.sendSnapshot(ctx, preemptiveRepDesc, SnapshotRequest_PREEMPTIVE, priority); err != nil { return nil, err } } - added := []roachpb.ReplicaDescriptor{repDesc} - err := execChangeReplicasTxn(ctx, r.store, desc, &updatedDesc, reason, details, added, nil /* removed */) - return &updatedDesc, err + iChgs := []internalReplicationChange{{target: target, typ: internalChangeTypeAddVoterViaPreemptiveSnap}} + return execChangeReplicasTxn(ctx, r.store, desc, reason, details, iChgs) +} + +type internalChangeType byte + +const ( + internalChangeTypeAddVoterViaPreemptiveSnap internalChangeType = iota + internalChangeTypeAddLearner + internalChangeTypePromoteLearner + internalChangeTypeRemove +) + +// internalReplicationChange is a replication target together with an internal +// change type. The internal change type is needed to encode in which way the +// replica is mutated (i.e. in a sense, what its predecessor looked like). We +// need this to accurately transcribe the old into the updated range descriptor. +type internalReplicationChange struct { + target roachpb.ReplicationTarget + typ internalChangeType } func execChangeReplicasTxn( ctx context.Context, store *Store, desc *roachpb.RangeDescriptor, - updatedDesc *roachpb.RangeDescriptor, reason storagepb.RangeLogEventReason, details string, - added, removed []roachpb.ReplicaDescriptor, -) error { + chgs []internalReplicationChange, +) (*roachpb.RangeDescriptor, error) { + updatedDesc := *desc + updatedDesc.SetReplicas(desc.Replicas().DeepCopy()) + generationComparableEnabled := store.ClusterSettings().Version.IsActive(cluster.VersionGenerationComparable) if generationComparableEnabled { updatedDesc.IncrementGeneration() updatedDesc.GenerationComparable = proto.Bool(true) } + var added, removed []roachpb.ReplicaDescriptor + + for _, chg := range chgs { + switch chg.typ { + case internalChangeTypeAddVoterViaPreemptiveSnap: + // Legacy code. + added = append(added, + updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_VoterFull)) + case internalChangeTypeAddLearner: + added = append(added, + updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_Learner)) + case internalChangeTypePromoteLearner: + // TODO(tbg): set to VoterIncoming when going through joint config. + rDesc, ok := updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_VoterFull) + if !ok { + return nil, errors.Errorf("cannot promote target %v which is missing as Learner", chg.target) + } + added = append(added, rDesc) + case internalChangeTypeRemove: + // TODO(tbg): set to VoterOutgoing when going through joint config instead. + rDesc, ok := updatedDesc.RemoveReplica(chg.target.NodeID, chg.target.StoreID) + if !ok { + return nil, errors.Errorf("cannot remove nonexistent target %v", chg.target) + } + removed = append(removed, rDesc) + default: + return nil, errors.Errorf("unsupported internal change type %d", chg.typ) + } + } + + if err := updatedDesc.Validate(); err != nil { + return nil, errors.Wrapf(err, "validating updated descriptor %s", &updatedDesc) + } + + var crt *roachpb.ChangeReplicasTrigger + if !store.ClusterSettings().Version.IsActive(cluster.VersionAtomicChangeReplicasTrigger) { + var deprecatedChangeType roachpb.ReplicaChangeType + var deprecatedRepDesc roachpb.ReplicaDescriptor + if len(added) > 0 { + deprecatedChangeType = roachpb.ADD_REPLICA + deprecatedRepDesc = added[0] + } else { + deprecatedChangeType = roachpb.REMOVE_REPLICA + deprecatedRepDesc = removed[0] + } + crt = &roachpb.ChangeReplicasTrigger{ + DeprecatedChangeType: deprecatedChangeType, + DeprecatedReplica: deprecatedRepDesc, + Desc: &updatedDesc, + } + } else { + crt = &roachpb.ChangeReplicasTrigger{ + Desc: &updatedDesc, + InternalAddedReplicas: added, + InternalRemovedReplicas: removed, + } + } + descKey := keys.RangeDescriptorKey(desc.StartKey) if err := store.DB().Txn(ctx, func(ctx context.Context, txn *client.Txn) error { log.Event(ctx, "attempting txn") @@ -1302,7 +1353,7 @@ func execChangeReplicasTxn( // Important: the range descriptor must be the first thing touched in the transaction // so the transaction record is co-located with the range being modified. - if err := updateRangeDescriptor(b, descKey, dbDescValue, updatedDesc); err != nil { + if err := updateRangeDescriptor(b, descKey, dbDescValue, &updatedDesc); err != nil { return err } @@ -1322,7 +1373,7 @@ func execChangeReplicasTxn( } { for _, repDesc := range tup.repDescs { if err := store.logChange( - ctx, txn, tup.typ, repDesc, *updatedDesc, reason, details, + ctx, txn, tup.typ, repDesc, updatedDesc, reason, details, ); err != nil { return err } @@ -1334,34 +1385,10 @@ func execChangeReplicasTxn( b := txn.NewBatch() // Update range descriptor addressing record(s). - if err := updateRangeAddressing(b, updatedDesc); err != nil { + if err := updateRangeAddressing(b, &updatedDesc); err != nil { return err } - var crt *roachpb.ChangeReplicasTrigger - if !store.ClusterSettings().Version.IsActive(cluster.VersionAtomicChangeReplicasTrigger) { - var deprecatedChangeType roachpb.ReplicaChangeType - var deprecatedRepDesc roachpb.ReplicaDescriptor - if len(added) > 0 { - deprecatedChangeType = roachpb.ADD_REPLICA - deprecatedRepDesc = added[0] - } else { - deprecatedChangeType = roachpb.REMOVE_REPLICA - deprecatedRepDesc = removed[0] - } - crt = &roachpb.ChangeReplicasTrigger{ - DeprecatedChangeType: deprecatedChangeType, - DeprecatedReplica: deprecatedRepDesc, - Desc: updatedDesc, - } - } else { - crt = &roachpb.ChangeReplicasTrigger{ - Desc: updatedDesc, - InternalAddedReplicas: added, - InternalRemovedReplicas: removed, - } - } - b.AddRawRequest(&roachpb.EndTransactionRequest{ Commit: true, InternalCommitTrigger: &roachpb.InternalCommitTrigger{ @@ -1379,10 +1406,10 @@ func execChangeReplicasTxn( if msg, ok := maybeDescriptorChangedError(desc, err); ok { err = &benignError{errors.New(msg)} } - return errors.Wrapf(err, "change replicas of r%d failed", desc.RangeID) + return nil, errors.Wrapf(err, "change replicas of r%d failed", desc.RangeID) } log.Event(ctx, "txn complete") - return nil + return &updatedDesc, nil } // sendSnapshot sends a snapshot of the replica state to the specified replica. From f71a407bcee9800d6fa3b45feebcc4362f1df43e Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 23 Aug 2019 11:45:05 +0200 Subject: [PATCH 19/23] storage: fix replicaGCQueue addition on removal trigger Once we enter joint changes, the replica to be removed will show up in `crt.Removed()` when the joint state is entered, but it only becomes eligible for actual removal when we leave the joint state later. The new code triggers at the right time, namely when the replica is no longer in the descriptor. Release note: None --- pkg/storage/replica_application_result.go | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/pkg/storage/replica_application_result.go b/pkg/storage/replica_application_result.go index 42ee3c148b16..dafa093e6813 100644 --- a/pkg/storage/replica_application_result.go +++ b/pkg/storage/replica_application_result.go @@ -294,16 +294,21 @@ func (r *Replica) handleComputeChecksumResult(ctx context.Context, cc *storagepb } func (r *Replica) handleChangeReplicasResult(ctx context.Context, chng *storagepb.ChangeReplicas) { - for _, rDesc := range chng.Removed() { - if r.store.StoreID() == rDesc.StoreID { - // This wants to run as late as possible, maximizing the chances - // that the other nodes have finished this command as well (since - // processing the removal from the queue looks up the Range at the - // lease holder, being too early here turns this into a no-op). - r.store.replicaGCQueue.AddAsync(ctx, r, replicaGCPriorityRemoved) + storeID := r.store.StoreID() + var found bool + for _, rDesc := range chng.Replicas() { + if rDesc.StoreID == storeID { + found = true break } } + if !found { + // This wants to run as late as possible, maximizing the chances + // that the other nodes have finished this command as well (since + // processing the removal from the queue looks up the Range at the + // lease holder, being too early here turns this into a no-op). + r.store.replicaGCQueue.AddAsync(ctx, r, replicaGCPriorityRemoved) + } } func (r *Replica) handleRaftLogDeltaResult(ctx context.Context, delta int64) { From 626922c46cccf4d914f5119cc9fbf63e124b0c0f Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 23 Aug 2019 15:24:13 +0200 Subject: [PATCH 20/23] storage: allow atomic replication changes in ChangeReplicas They default to OFF. This needs a lot more tests which will be added separately in the course of switching the default to ON and will focus on the interactions of joint states with everything else in the system. We'll also need another audit of consumers of the replica descriptors to make sure nothing was missed in the first pass. Release note: None --- docs/generated/settings/settings.html | 3 +- .../client_atomic_membership_change_test.go | 118 ++++++++-- pkg/storage/replica.go | 10 - pkg/storage/replica_application_cmd.go | 2 +- pkg/storage/replica_command.go | 210 ++++++++++++------ pkg/storage/replicate_queue.go | 3 +- 6 files changed, 238 insertions(+), 108 deletions(-) diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 2fd81563fbb5..49d32aba3344 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -27,6 +27,7 @@ kv.allocator.load_based_rebalancingenumerationleases and replicaswhether to rebalance based on the distribution of QPS across stores [off = 0, leases = 1, leases and replicas = 2] kv.allocator.qps_rebalance_thresholdfloat0.25minimum fraction away from the mean a store's QPS (such as queries per second) can be before it is considered overfull or underfull kv.allocator.range_rebalance_thresholdfloat0.05minimum fraction away from the mean a store's range count can be before it is considered overfull or underfull +kv.atomic_replication_changes.enabledbooleanfalseuse atomic replication changes kv.bulk_ingest.batch_sizebyte size16 MiBthe maximum size of the payload in an AddSSTable request kv.bulk_ingest.buffer_incrementbyte size32 MiBthe size by which the BulkAdder attempts to grow its buffer before flushing kv.bulk_ingest.index_buffer_sizebyte size32 MiBthe initial size of the BulkAdder buffer handling secondary index imports @@ -129,6 +130,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen in the /debug page trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versioncustom validation19.1-8set the active cluster version in the format '.' +versioncustom validation19.1-9set the active cluster version in the format '.' diff --git a/pkg/storage/client_atomic_membership_change_test.go b/pkg/storage/client_atomic_membership_change_test.go index a731d01f0cfc..53bc697b2b85 100644 --- a/pkg/storage/client_atomic_membership_change_test.go +++ b/pkg/storage/client_atomic_membership_change_test.go @@ -12,50 +12,122 @@ package storage_test import ( "context" + "strings" "testing" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/kr/pretty" + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/raft/confchange" + "go.etcd.io/etcd/raft/tracker" ) -func TestAtomicMembershipChange(t *testing.T) { +// TestAtomicReplicationChange is a simple smoke test for atomic membership +// changes. +func TestAtomicReplicationChange(t *testing.T) { defer leaktest.AfterTest(t)() - - // This is a simple smoke test to spot obvious issues with atomic replica changes. - // These aren't implemented at the time of writing. The compound change below is - // internally unwound and executed step by step (see executeAdminBatch()). ctx := context.Background() args := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &storage.StoreTestingKnobs{}, + }, + }, ReplicationMode: base.ReplicationManual, } - tc := testcluster.StartTestCluster(t, 4, args) + tc := testcluster.StartTestCluster(t, 6, args) defer tc.Stopper().Stop(ctx) + _, err := tc.ServerConn(0).Exec(`SET CLUSTER SETTING kv.atomic_replication_changes.enabled = true`) + require.NoError(t, err) + + // Create a range and put it on n1, n2, n3. Intentionally do this one at a + // time so we're not using atomic replication changes yet. k := tc.ScratchRange(t) - expDesc, err := tc.AddReplicas(k, tc.Target(1), tc.Target(2)) + desc, err := tc.AddReplicas(k, tc.Target(1)) + require.NoError(t, err) + desc, err = tc.AddReplicas(k, tc.Target(2)) require.NoError(t, err) - // Range is now on s1,s2,s3. "Atomically" add it to s4 while removing from s3. - // This isn't really atomic yet. + runChange := func(expDesc roachpb.RangeDescriptor, chgs []roachpb.ReplicationChange) roachpb.RangeDescriptor { + t.Helper() + desc, err := tc.Servers[0].DB().AdminChangeReplicas( + // TODO(tbg): when 19.2 is out, remove this "feature gate" here and in + // AdminChangeReplicas. + context.WithValue(ctx, "testing", "testing"), + k, expDesc, chgs, + ) + require.NoError(t, err) - chgs := []roachpb.ReplicationChange{ - {ChangeType: roachpb.ADD_REPLICA, Target: tc.Target(3)}, - {ChangeType: roachpb.REMOVE_REPLICA, Target: tc.Target(2)}, + return *desc } - // TODO(tbg): when 19.2 is out, remove this "feature gate" here and in - // AdminChangeReplicas. - ctx = context.WithValue(ctx, "testing", "testing") - desc, err := tc.Servers[0].DB().AdminChangeReplicas(ctx, k, expDesc, chgs) - require.NoError(t, err) - var stores []roachpb.StoreID - for _, rDesc := range desc.Replicas().All() { - stores = append(stores, rDesc.StoreID) + + checkDesc := func(desc roachpb.RangeDescriptor, expStores ...roachpb.StoreID) { + testutils.SucceedsSoon(t, func() error { + var sawStores []roachpb.StoreID + for _, s := range tc.Servers { + r, _ := s.Stores().GetReplicaForRangeID(desc.RangeID) + if r == nil { + continue + } + if _, found := desc.GetReplicaDescriptor(r.StoreID()); !found { + // There's a replica but it's not in the new descriptor, so + // it should be replicaGC'ed soon. + return errors.Errorf("%s should have been removed", r) + } + sawStores = append(sawStores, r.StoreID()) + // Check that in-mem descriptor of repl is up-to-date. + if diff := pretty.Diff(&desc, r.Desc()); len(diff) > 0 { + return errors.Errorf("diff(want, have):\n%s", strings.Join(diff, "\n")) + } + // Check that conf state is up to date. This can fail even though + // the descriptor already matches since the descriptor is updated + // a hair earlier. + cfg, _, err := confchange.Restore(confchange.Changer{ + Tracker: tracker.MakeProgressTracker(1), + LastIndex: 1, + }, desc.Replicas().ConfState()) + require.NoError(t, err) + act := r.RaftStatus().Config.Voters + if diff := pretty.Diff(cfg.Voters, act); len(diff) > 0 { + return errors.Errorf("diff(exp,act):\n%s", strings.Join(diff, "\n")) + } + } + assert.Equal(t, expStores, sawStores) + return nil + }) } - exp := []roachpb.StoreID{1, 2, 4} - // TODO(tbg): test more details and scenarios (learners etc). - require.Equal(t, exp, stores) + + // Run a fairly general change. + desc = runChange(desc, []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_REPLICA, Target: tc.Target(3)}, + {ChangeType: roachpb.ADD_REPLICA, Target: tc.Target(5)}, + {ChangeType: roachpb.REMOVE_REPLICA, Target: tc.Target(2)}, + {ChangeType: roachpb.ADD_REPLICA, Target: tc.Target(4)}, + }) + + // Replicas should now live on all stores except s3. + checkDesc(desc, 1, 2, 4, 5, 6) + + // Transfer the lease to s5. + require.NoError(t, tc.TransferRangeLease(desc, tc.Target(4))) + + // Rebalance back down all the way. + desc = runChange(desc, []roachpb.ReplicationChange{ + {ChangeType: roachpb.REMOVE_REPLICA, Target: tc.Target(0)}, + {ChangeType: roachpb.REMOVE_REPLICA, Target: tc.Target(1)}, + {ChangeType: roachpb.REMOVE_REPLICA, Target: tc.Target(3)}, + {ChangeType: roachpb.REMOVE_REPLICA, Target: tc.Target(5)}, + }) + + // Only a lone voter on s5 should be left over. + checkDesc(desc, 5) } diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index 2d84d179b663..304c20532d33 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -92,16 +92,6 @@ var useAtomicReplicationChanges = settings.RegisterBoolSetting( false, ) -// FatalAtomicReplicationChangeUnimplemented is called by code that will need -// to change when atomic replication changes are available to make sure it is -// updated accordingly. -// -// TODO(tbg): update all callers and remove this method in the commit that allows -// atomic replication changes. -func FatalAtomicReplicationChangeUnimplemented(ctx context.Context) { - log.Fatalf(ctx, "atomic configuration changes not yet implemented") -} - // MaxCommandSizeFloor is the minimum allowed value for the MaxCommandSize // cluster setting. const MaxCommandSizeFloor = 4 << 20 // 4MB diff --git a/pkg/storage/replica_application_cmd.go b/pkg/storage/replica_application_cmd.go index 53e06bc37f92..d9bb3b2cfc92 100644 --- a/pkg/storage/replica_application_cmd.go +++ b/pkg/storage/replica_application_cmd.go @@ -170,7 +170,7 @@ func (d *decodedRaftEntry) decode(ctx context.Context, e *raftpb.Entry) error { switch e.Type { case raftpb.EntryNormal: return d.decodeNormalEntry(e) - case raftpb.EntryConfChange: + case raftpb.EntryConfChange, raftpb.EntryConfChangeV2: return d.decodeConfChangeEntry(e) default: log.Fatalf(ctx, "unexpected Raft entry: %v", e) diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index dd5a2980458a..e791cebf7817 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -143,6 +143,15 @@ func (r *Replica) adminSplitWithDescriptor( delayable bool, reason string, ) (roachpb.AdminSplitResponse, error) { + var err error + // The split queue doesn't care about the set of replicas, so if we somehow + // are being handed one that's in a joint state, finalize that before + // continuing. + desc, err = r.maybeLeaveAtomicChangeReplicas(ctx, desc) + if err != nil { + return roachpb.AdminSplitResponse{}, err + } + var reply roachpb.AdminSplitResponse // Determine split key if not provided with args. This scan is @@ -570,10 +579,11 @@ func (r *Replica) AdminMerge( // Should never happen, but just in case. return errors.Errorf("ranges are not adjacent; %s != %s", origLeftDesc.EndKey, rightDesc.StartKey) } - // For simplicity, don't handle learner replicas, expect the caller to - // resolve them first. (Defensively, we check that there are no non-voter - // replicas, in case some third type is later added). This behavior can be - // changed later if the complexity becomes worth it, but it's not right now. + // For simplicity, don't handle learner replicas or joint states, expect + // the caller to resolve them first. (Defensively, we check that there + // are no non-voter replicas, in case some third type is later added). + // This behavior can be changed later if the complexity becomes worth + // it, but it's not right now. lReplicas, rReplicas := origLeftDesc.Replicas(), rightDesc.Replicas() if len(lReplicas.Voters()) != len(lReplicas.All()) { return errors.Errorf("cannot merge range with non-voter replicas on lhs: %s", lReplicas) @@ -808,9 +818,6 @@ func IsSnapshotError(err error) bool { // leaseholder (which in particular implies that we can never remove all // replicas). // -// NB: at the time of writing, atomic replication changes are not implemented -// yet. Only a single change is supported, everything else returns an error. -// // The returned RangeDescriptor is the new value of the range's descriptor // following the successful commit of the transaction. // @@ -849,29 +856,12 @@ func IsSnapshotError(err error) bool { // protocol; otherwise it has to use joint consensus. In this latter mechanism, // a first configuration change is made which results in a configuration ("joint // configuration") in which a quorum of both the old replicas and the new -// replica sets is required for decision making. Since this joint configuration -// is not represented in the RangeDescriptor (which is the source of truth of -// the replication configuration), additional information about the joint state -// is persisted under RangeDescriptorJointKey, a replicated key located on the -// range similar to the range descriptor (but not versioned). Raft will -// automatically transition out of this joint configuration as soon as it has -// properly been applied (and we clear the extra replicated state atomically); -// ChangeReplicas ensures the final configuration is active before returning. -// -// TODO(tbg): figure out how the "waiting to transition out of the joint config" -// will happen. We want Raft to auto-transition out (rather than doing it -// manually) because that way we know we'll leave that joint state even if the -// coordinator crashes etc. On the other hand, the polling seems a little -// difficult to do idiomatically. If many replication changes are carried out -// back to back, what do we wait for? We only need to know that some replica -// (typically the leader) has transitioned out (i.e. there's no requirement that -// the local replica has done so). It seems most straightforward to stash the -// joint state in an inline key that can be read through KV (mutated below -// Raft), and make sure the generation of the replication change is preserved. -// Then we just need to poll that the key goes away or has a larger generation -// (indicating that we transitioned out of "our" conf change and into another -// joint config driven by someone else). We can avoid the poll in the common -// case by proposing an empty entry first. +// replica sets is required for decision making. Transitioning into this joint +// configuration, the RangeDescriptor (which is the source of truth of +// the replication configuration) is updated with corresponding replicas of +// type VoterIncoming and VoterOutgoing. +// Immediately after committing this change, a second transition updates the +// descriptor with and activates the final configuration. // // A replica that learns that it was removed will queue itself for replicaGC. // Note that a removed replica may never apply the configuration change removing @@ -917,9 +907,6 @@ func (r *Replica) ChangeReplicas( } return desc, nil } - if len(chgs) > 1 { - FatalAtomicReplicationChangeUnimplemented(ctx) - } // Atomic replication change. return r.changeReplicasImpl(ctx, desc, priority, reason, details, chgs) } @@ -932,10 +919,13 @@ func (r *Replica) changeReplicasImpl( details string, chgs roachpb.ReplicationChanges, ) (updatedDesc *roachpb.RangeDescriptor, _ error) { - if len(chgs) != 1 { - // TODO(tbg): lift this restriction when atomic membership changes are - // plumbed into raft. - return nil, errors.Errorf("need exactly one change, got %+v", chgs) + var err error + // If in a joint config, clean up. The assumption here is that the caller + // of ChangeReplicas didn't even realize that they were holding on to a + // joint descriptor and would rather not have to deal with that fact. + desc, err = r.maybeLeaveAtomicChangeReplicas(ctx, desc) + if err != nil { + return nil, err } if err := validateReplicationChanges(desc, chgs); err != nil { @@ -955,7 +945,7 @@ func (r *Replica) changeReplicasImpl( return r.addReplicaLegacyPreemptiveSnapshot(ctx, chgs[0].Target, desc, priority, reason, details) } // We're removing a single voter. - return r.finalizeChangeReplicas(ctx, desc, priority, reason, details, chgs) + return r.atomicReplicationChange(ctx, desc, priority, reason, details, chgs) } if adds := chgs.Additions(); len(adds) > 0 { @@ -972,8 +962,13 @@ func (r *Replica) changeReplicasImpl( // Catch up any learners, then run the atomic replication change that adds the // final voters and removes any undesirable replicas. - desc, err := r.finalizeChangeReplicas(ctx, desc, priority, reason, details, chgs) + desc, err = r.atomicReplicationChange(ctx, desc, priority, reason, details, chgs) if err != nil { + // If the error occurred while transitioning out of an atomic replication change, + // try again here with a fresh descriptor; this is a noop otherwise. + if _, err := r.maybeLeaveAtomicChangeReplicas(ctx, r.Desc()); err != nil { + return nil, err + } if fn := r.store.cfg.TestingKnobs.ReplicaAddSkipLearnerRollback; fn != nil && fn() { return nil, err } @@ -986,7 +981,29 @@ func (r *Replica) changeReplicasImpl( } return nil, err } - return desc, nil + return desc, err +} + +// maybeLeaveAtomicChangeReplicas transitions out of the joint configuration if +// the descriptor indicates one. This involves running a distributed transaction +// updating said descriptor, the result of which will be returned. +func (r *Replica) maybeLeaveAtomicChangeReplicas( + ctx context.Context, desc *roachpb.RangeDescriptor, +) (*roachpb.RangeDescriptor, error) { + // We want execChangeReplicasTxn to be able to make sure it's only tasked + // with leaving a joint state when it's in one, so make sure we don't call + // it if we're not. + if !desc.Replicas().InAtomicReplicationChange() { + return desc, nil + } + + // NB: reason and detail won't be used because no range log event will be + // emitted. + // + // TODO(tbg): reconsider this. + return execChangeReplicasTxn( + ctx, r.store, desc, storagepb.ReasonUnknown /* unused */, "", nil, /* iChgs */ + ) } func validateReplicationChanges( @@ -1070,13 +1087,18 @@ func addLearnerReplicas( return desc, nil } -// finalizeChangeReplicas carries out the atomic membership change that finalizes -// the addition and/or removal of replicas. Any voters in the process of being -// added (as reflected by the replication changes) must have been added as -// learners already and will be caught up before being promoted to voters. Any -// replica removals (from the replication changes) will be processed. All of this -// occurs in one atomic raft membership change. -func (r *Replica) finalizeChangeReplicas( +// atomicReplicationChange carries out the atomic membership change that +// finalizes the addition and/or removal of replicas. Any voters in the process +// of being added (as reflected by the replication changes) must have been added +// as learners already and will be caught up before being promoted to voters. +// Any replica removals (from the replication changes) will be processed. All of +// this occurs in one atomic raft membership change which is carried out across +// two distributed transactions. On error, it is possible that the range is in +// the intermediate ("joint") configuration in which a quorum of both the old +// and new sets of voters is required. If a range is encountered in this state, +// r.maybeLeaveAtomicReplicationChange can fix this, but it is the caller's +// job to do this when necessary. +func (r *Replica) atomicReplicationChange( ctx context.Context, desc *roachpb.RangeDescriptor, priority SnapshotRequest_Priority, @@ -1137,7 +1159,13 @@ func (r *Replica) finalizeChangeReplicas( iChgs = append(iChgs, internalReplicationChange{target: target, typ: internalChangeTypeRemove}) } - return execChangeReplicasTxn(ctx, r.store, desc, reason, details, iChgs) + var err error + desc, err = execChangeReplicasTxn(ctx, r.store, desc, reason, details, iChgs) + if err != nil { + return nil, err + } + // Leave the joint config if we entered one. + return r.maybeLeaveAtomicChangeReplicas(ctx, desc) } // tryRollbackLearnerReplica attempts to remove a learner specified by the @@ -1281,32 +1309,66 @@ func execChangeReplicasTxn( } var added, removed []roachpb.ReplicaDescriptor - - for _, chg := range chgs { - switch chg.typ { - case internalChangeTypeAddVoterViaPreemptiveSnap: - // Legacy code. - added = append(added, - updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_VoterFull)) - case internalChangeTypeAddLearner: - added = append(added, - updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_Learner)) - case internalChangeTypePromoteLearner: - // TODO(tbg): set to VoterIncoming when going through joint config. - rDesc, ok := updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_VoterFull) - if !ok { - return nil, errors.Errorf("cannot promote target %v which is missing as Learner", chg.target) + if len(chgs) > 0 { + if desc.Replicas().InAtomicReplicationChange() { + return nil, errors.Errorf("must transition out of joint config first: %s", desc) + } + + useJoint := len(chgs) > 1 + for _, chg := range chgs { + switch chg.typ { + case internalChangeTypeAddVoterViaPreemptiveSnap: + // Legacy code. + added = append(added, + updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_VoterFull)) + case internalChangeTypeAddLearner: + added = append(added, + updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_Learner)) + case internalChangeTypePromoteLearner: + typ := roachpb.ReplicaType_VoterFull + if useJoint { + typ = roachpb.ReplicaType_VoterIncoming + } + rDesc, ok := updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, typ) + if !ok { + return nil, errors.Errorf("cannot promote target %v which is missing as Learner", chg.target) + } + added = append(added, rDesc) + case internalChangeTypeRemove: + var rDesc roachpb.ReplicaDescriptor + var ok bool + if !useJoint { + rDesc, ok = updatedDesc.RemoveReplica(chg.target.NodeID, chg.target.StoreID) + } else { + rDesc, ok = updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_VoterOutgoing) + } + if !ok { + return nil, errors.Errorf("cannot remove nonexistent target %v", chg.target) + } + removed = append(removed, rDesc) + default: + return nil, errors.Errorf("unsupported internal change type %d", chg.typ) } - added = append(added, rDesc) - case internalChangeTypeRemove: - // TODO(tbg): set to VoterOutgoing when going through joint config instead. - rDesc, ok := updatedDesc.RemoveReplica(chg.target.NodeID, chg.target.StoreID) - if !ok { - return nil, errors.Errorf("cannot remove nonexistent target %v", chg.target) + } + } else { + // Want to leave a joint config. Note that we're not populating 'added' or 'removed', this + // is intentional; leaving the joint config corresponds to an "empty" raft conf change. + var isJoint bool + // NB: the DeepCopy is needed or we'll skip over an entry every time we + // call RemoveReplica below. + for _, rDesc := range updatedDesc.Replicas().DeepCopy().All() { + switch rDesc.GetType() { + case roachpb.ReplicaType_VoterIncoming: + updatedDesc.SetReplicaType(rDesc.NodeID, rDesc.StoreID, roachpb.ReplicaType_VoterFull) + isJoint = true + case roachpb.ReplicaType_VoterOutgoing: + updatedDesc.RemoveReplica(rDesc.NodeID, rDesc.StoreID) + isJoint = true + default: } - removed = append(removed, rDesc) - default: - return nil, errors.Errorf("unsupported internal change type %d", chg.typ) + } + if !isJoint { + return nil, errors.Errorf("cannot leave a joint config; desc not joint: %s", &updatedDesc) } } @@ -1338,6 +1400,10 @@ func execChangeReplicasTxn( } } + if _, err := crt.ConfChange(nil); err != nil { + return nil, errors.Wrapf(err, "programming error: malformed trigger created from desc %s to %s", desc, &updatedDesc) + } + descKey := keys.RangeDescriptorKey(desc.StartKey) if err := store.DB().Txn(ctx, func(ctx context.Context, txn *client.Txn) error { log.Event(ctx, "attempting txn") diff --git a/pkg/storage/replicate_queue.go b/pkg/storage/replicate_queue.go index 4d6735621372..a1e2a40af580 100644 --- a/pkg/storage/replicate_queue.go +++ b/pkg/storage/replicate_queue.go @@ -347,7 +347,8 @@ func (rq *replicateQueue) processOneChange( case AllocatorConsiderRebalance: return rq.considerRebalance(ctx, repl, voterReplicas, canTransferLease, dryRun) case AllocatorFinalizeAtomicReplicationChange: - FatalAtomicReplicationChangeUnimplemented(ctx) + _, err := repl.maybeLeaveAtomicChangeReplicas(ctx, repl.Desc()) + return false, err } return true, nil } From 037fd14741adc2fd76feba9dcd307de34ccbca31 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 23 Aug 2019 23:10:20 +0200 Subject: [PATCH 21/23] storage: minor !useLearners refactor Release note: None --- pkg/storage/replica_command.go | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index e791cebf7817..be4cc79645b5 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -941,11 +941,15 @@ func (r *Replica) changeReplicasImpl( if len(chgs) != 1 { return nil, errors.Errorf("need exactly one change, got %+v", chgs) } - if chgs[0].ChangeType == roachpb.ADD_REPLICA { + switch chgs[0].ChangeType { + case roachpb.ADD_REPLICA: return r.addReplicaLegacyPreemptiveSnapshot(ctx, chgs[0].Target, desc, priority, reason, details) + case roachpb.REMOVE_REPLICA: + // We're removing a single voter. + return r.atomicReplicationChange(ctx, desc, priority, reason, details, chgs) + default: + return nil, errors.Errorf("unknown change type %d", chgs[0].ChangeType) } - // We're removing a single voter. - return r.atomicReplicationChange(ctx, desc, priority, reason, details, chgs) } if adds := chgs.Additions(); len(adds) > 0 { From 38b60e632bfbc1a01e4c9902d2eabfad4b62709a Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 23 Aug 2019 23:27:13 +0200 Subject: [PATCH 22/23] storage: handle ConfChangeV2 in tryRaftLogEntry Release note: None --- pkg/storage/debug_print.go | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/pkg/storage/debug_print.go b/pkg/storage/debug_print.go index dfa40bdd52ae..94d57a0ffb2c 100644 --- a/pkg/storage/debug_print.go +++ b/pkg/storage/debug_print.go @@ -188,13 +188,24 @@ func tryRaftLogEntry(kv engine.MVCCKeyValue) (string, error) { &ent, leaseStr, &cmd, writeBatch), nil } return fmt.Sprintf("%s: EMPTY\n", &ent), nil - } else if ent.Type == raftpb.EntryConfChange { - var cc raftpb.ConfChange - if err := protoutil.Unmarshal(ent.Data, &cc); err != nil { - return "", err + } else if ent.Type == raftpb.EntryConfChange || ent.Type == raftpb.EntryConfChangeV2 { + var c raftpb.ConfChangeI + if ent.Type == raftpb.EntryConfChange { + var cc raftpb.ConfChange + if err := protoutil.Unmarshal(ent.Data, &cc); err != nil { + return "", err + } + c = cc + } else { + var cc raftpb.ConfChangeV2 + if err := protoutil.Unmarshal(ent.Data, &cc); err != nil { + return "", err + } + c = cc } + var ctx ConfChangeContext - if err := protoutil.Unmarshal(cc.Context, &ctx); err != nil { + if err := protoutil.Unmarshal(c.AsV2().Context, &ctx); err != nil { return "", err } var cmd storagepb.ReplicatedEvalResult From 371e289158972780ccd6c3658c063f53e260ba13 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Mon, 26 Aug 2019 13:38:45 +0200 Subject: [PATCH 23/23] roachpb: idiomatically name ReplicaType variants Release note: None --- c-deps/libroach/protos/roachpb/metadata.pb.h | 12 +- pkg/kv/dist_sender_test.go | 4 +- pkg/roachpb/data.go | 14 +- pkg/roachpb/data.pb.go | 48 ++-- pkg/roachpb/data.proto | 2 +- pkg/roachpb/data_test.go | 50 ++-- pkg/roachpb/metadata.go | 6 +- pkg/roachpb/metadata.pb.go | 268 ++++++++++--------- pkg/roachpb/metadata.proto | 44 +-- pkg/roachpb/metadata_replicas.go | 51 ++-- pkg/roachpb/metadata_replicas_test.go | 14 +- pkg/storage/allocator_test.go | 2 +- pkg/storage/batcheval/cmd_lease.go | 2 +- pkg/storage/batcheval/cmd_lease_test.go | 4 +- pkg/storage/client_test.go | 2 +- pkg/storage/merge_queue.go | 4 +- pkg/storage/raft_snapshot_queue.go | 2 +- pkg/storage/replica_command.go | 24 +- pkg/storage/replica_follower_read.go | 2 +- pkg/storage/replica_learner_test.go | 4 +- pkg/storage/replica_range_lease.go | 2 +- 21 files changed, 283 insertions(+), 278 deletions(-) diff --git a/c-deps/libroach/protos/roachpb/metadata.pb.h b/c-deps/libroach/protos/roachpb/metadata.pb.h index 9ce1ecffefae..8ffd2aace4bc 100644 --- a/c-deps/libroach/protos/roachpb/metadata.pb.h +++ b/c-deps/libroach/protos/roachpb/metadata.pb.h @@ -114,14 +114,14 @@ namespace cockroach { namespace roachpb { enum ReplicaType { - VoterFull = 0, - VoterIncoming = 2, - VoterOutgoing = 3, - Learner = 1 + VOTER_FULL = 0, + VOTER_INCOMING = 2, + VOTER_OUTGOING = 3, + LEARNER = 1 }; bool ReplicaType_IsValid(int value); -const ReplicaType ReplicaType_MIN = VoterFull; -const ReplicaType ReplicaType_MAX = VoterOutgoing; +const ReplicaType ReplicaType_MIN = VOTER_FULL; +const ReplicaType ReplicaType_MAX = VOTER_OUTGOING; const int ReplicaType_ARRAYSIZE = ReplicaType_MAX + 1; // =================================================================== diff --git a/pkg/kv/dist_sender_test.go b/pkg/kv/dist_sender_test.go index 167560122c07..8244ba59f05d 100644 --- a/pkg/kv/dist_sender_test.go +++ b/pkg/kv/dist_sender_test.go @@ -327,7 +327,7 @@ func TestSendRPCOrder(t *testing.T) { if err := g.AddInfoProto(gossip.MakeNodeIDKey(roachpb.NodeID(i)), nd, time.Hour); err != nil { t.Fatal(err) } - descriptor.AddReplica(roachpb.NodeID(i), roachpb.StoreID(i), roachpb.ReplicaType_VoterFull) + descriptor.AddReplica(roachpb.NodeID(i), roachpb.StoreID(i), roachpb.VOTER_FULL) } // Stub to be changed in each test case. @@ -1429,7 +1429,7 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { t.Fatal(err) } - descriptor.AddReplica(roachpb.NodeID(i), roachpb.StoreID(i), roachpb.ReplicaType_VoterFull) + descriptor.AddReplica(roachpb.NodeID(i), roachpb.StoreID(i), roachpb.VOTER_FULL) } descDB := mockRangeDescriptorDBForDescs( testMetaRangeDescriptor, diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 9d4fde9c71c7..cd2aef01bebb 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1400,19 +1400,19 @@ func confChangeImpl( var changeType raftpb.ConfChangeType switch rDesc.GetType() { - case ReplicaType_VoterFull: + case VOTER_FULL: // We're adding a new voter. changeType = raftpb.ConfChangeAddNode - case ReplicaType_VoterIncoming: + case VOTER_INCOMING: // We're adding a voter, but will transition into a joint config // first. changeType = raftpb.ConfChangeAddNode - case ReplicaType_Learner: + case LEARNER: // We're adding a new learner. Note that we're guaranteed by // virtue of the upstream ChangeReplicas txn that this learner // is not currently a voter. If we wanted to support that (i.e. // demotions) we'd need to introduce a new - // ReplicaType_VoterDemoting for that purpose. + // replica type VOTER_DEMOTING for that purpose. changeType = raftpb.ConfChangeAddLearnerNode default: return nil, errors.Errorf("can't add replica in state %v", rDesc.GetType()) @@ -1425,13 +1425,13 @@ func confChangeImpl( for _, rDesc := range removed { switch rDesc.GetType() { - case ReplicaType_VoterOutgoing: + case VOTER_OUTGOING: // If a voter is removed through joint consensus, it will // be turned into an outgoing voter first. if err := checkExists(rDesc); err != nil { return nil, err } - case ReplicaType_VoterFull, ReplicaType_Learner: + case VOTER_FULL, LEARNER: // A learner or full voter can't be in the desc after. if err := checkNotExists(rDesc); err != nil { return nil, err @@ -1452,7 +1452,7 @@ func confChangeImpl( var enteringJoint bool for _, rDesc := range replicas { switch rDesc.GetType() { - case ReplicaType_VoterIncoming, ReplicaType_VoterOutgoing: + case VOTER_INCOMING, VOTER_OUTGOING: enteringJoint = true default: } diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 8a74d10edcaa..04d2fff62d73 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -87,7 +87,7 @@ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } func (ValueType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{0} + return fileDescriptor_data_3e5eb143f6729682, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -111,7 +111,7 @@ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{1} + return fileDescriptor_data_3e5eb143f6729682, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -163,7 +163,7 @@ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } func (TransactionStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{2} + return fileDescriptor_data_3e5eb143f6729682, []int{2} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -180,7 +180,7 @@ type Span struct { func (m *Span) Reset() { *m = Span{} } func (*Span) ProtoMessage() {} func (*Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{0} + return fileDescriptor_data_3e5eb143f6729682, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -232,7 +232,7 @@ func (m *Value) Reset() { *m = Value{} } func (m *Value) String() string { return proto.CompactTextString(m) } func (*Value) ProtoMessage() {} func (*Value) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{1} + return fileDescriptor_data_3e5eb143f6729682, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -268,7 +268,7 @@ func (m *KeyValue) Reset() { *m = KeyValue{} } func (m *KeyValue) String() string { return proto.CompactTextString(m) } func (*KeyValue) ProtoMessage() {} func (*KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{2} + return fileDescriptor_data_3e5eb143f6729682, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -306,7 +306,7 @@ func (m *StoreIdent) Reset() { *m = StoreIdent{} } func (m *StoreIdent) String() string { return proto.CompactTextString(m) } func (*StoreIdent) ProtoMessage() {} func (*StoreIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{3} + return fileDescriptor_data_3e5eb143f6729682, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -346,7 +346,7 @@ func (m *SplitTrigger) Reset() { *m = SplitTrigger{} } func (m *SplitTrigger) String() string { return proto.CompactTextString(m) } func (*SplitTrigger) ProtoMessage() {} func (*SplitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{4} + return fileDescriptor_data_3e5eb143f6729682, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -394,7 +394,7 @@ func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } func (m *MergeTrigger) String() string { return proto.CompactTextString(m) } func (*MergeTrigger) ProtoMessage() {} func (*MergeTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{5} + return fileDescriptor_data_3e5eb143f6729682, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -423,7 +423,7 @@ var xxx_messageInfo_MergeTrigger proto.InternalMessageInfo // methods return the replicas being added and removed, respectively. If more than // one change is specified (i.e. len(Added())+len(Removed()) exceeds one), this // initiates an atomic replication change in which the "removed" replicas are -// of type ReplicaType_VoterOutgoing. This joint configuration is left via another +// of type VOTER_OUTGOING. This joint configuration is left via another // ChangeReplicasTrigger which does not specify any additions nor removals. type ChangeReplicasTrigger struct { // TODO(tbg): remove once we know that no trigger using this will ever be @@ -461,7 +461,7 @@ type ChangeReplicasTrigger struct { func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } func (*ChangeReplicasTrigger) ProtoMessage() {} func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{6} + return fileDescriptor_data_3e5eb143f6729682, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -503,7 +503,7 @@ func (m *ModifiedSpanTrigger) Reset() { *m = ModifiedSpanTrigger{} } func (m *ModifiedSpanTrigger) String() string { return proto.CompactTextString(m) } func (*ModifiedSpanTrigger) ProtoMessage() {} func (*ModifiedSpanTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{7} + return fileDescriptor_data_3e5eb143f6729682, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -549,7 +549,7 @@ func (m *StickyBitTrigger) Reset() { *m = StickyBitTrigger{} } func (m *StickyBitTrigger) String() string { return proto.CompactTextString(m) } func (*StickyBitTrigger) ProtoMessage() {} func (*StickyBitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{8} + return fileDescriptor_data_3e5eb143f6729682, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -588,7 +588,7 @@ func (m *InternalCommitTrigger) Reset() { *m = InternalCommitTrigger{} } func (m *InternalCommitTrigger) String() string { return proto.CompactTextString(m) } func (*InternalCommitTrigger) ProtoMessage() {} func (*InternalCommitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{9} + return fileDescriptor_data_3e5eb143f6729682, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -657,7 +657,7 @@ func (m *ObservedTimestamp) Reset() { *m = ObservedTimestamp{} } func (m *ObservedTimestamp) String() string { return proto.CompactTextString(m) } func (*ObservedTimestamp) ProtoMessage() {} func (*ObservedTimestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{10} + return fileDescriptor_data_3e5eb143f6729682, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -885,7 +885,7 @@ type Transaction struct { func (m *Transaction) Reset() { *m = Transaction{} } func (*Transaction) ProtoMessage() {} func (*Transaction) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{11} + return fileDescriptor_data_3e5eb143f6729682, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -935,7 +935,7 @@ func (m *TransactionRecord) Reset() { *m = TransactionRecord{} } func (m *TransactionRecord) String() string { return proto.CompactTextString(m) } func (*TransactionRecord) ProtoMessage() {} func (*TransactionRecord) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{12} + return fileDescriptor_data_3e5eb143f6729682, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -971,7 +971,7 @@ func (m *Intent) Reset() { *m = Intent{} } func (m *Intent) String() string { return proto.CompactTextString(m) } func (*Intent) ProtoMessage() {} func (*Intent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{13} + return fileDescriptor_data_3e5eb143f6729682, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1008,7 +1008,7 @@ func (m *SequencedWrite) Reset() { *m = SequencedWrite{} } func (m *SequencedWrite) String() string { return proto.CompactTextString(m) } func (*SequencedWrite) ProtoMessage() {} func (*SequencedWrite) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{14} + return fileDescriptor_data_3e5eb143f6729682, []int{14} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1072,7 +1072,7 @@ type Lease struct { func (m *Lease) Reset() { *m = Lease{} } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{15} + return fileDescriptor_data_3e5eb143f6729682, []int{15} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1117,7 +1117,7 @@ func (m *AbortSpanEntry) Reset() { *m = AbortSpanEntry{} } func (m *AbortSpanEntry) String() string { return proto.CompactTextString(m) } func (*AbortSpanEntry) ProtoMessage() {} func (*AbortSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{16} + return fileDescriptor_data_3e5eb143f6729682, []int{16} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1181,7 +1181,7 @@ func (m *TxnCoordMeta) Reset() { *m = TxnCoordMeta{} } func (m *TxnCoordMeta) String() string { return proto.CompactTextString(m) } func (*TxnCoordMeta) ProtoMessage() {} func (*TxnCoordMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_data_01eb55a5b54e32dc, []int{17} + return fileDescriptor_data_3e5eb143f6729682, []int{17} } func (m *TxnCoordMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6297,9 +6297,9 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_01eb55a5b54e32dc) } +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_3e5eb143f6729682) } -var fileDescriptor_data_01eb55a5b54e32dc = []byte{ +var fileDescriptor_data_3e5eb143f6729682 = []byte{ // 2110 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcd, 0x6f, 0xdb, 0xc8, 0x15, 0x37, 0x4d, 0x4a, 0xa2, 0x9e, 0x3e, 0x4c, 0x4f, 0xe2, 0x44, 0x49, 0x50, 0x29, 0x55, 0x8a, diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 820396bd67d7..e8ad8048d033 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -161,7 +161,7 @@ enum ReplicaChangeType { // methods return the replicas being added and removed, respectively. If more than // one change is specified (i.e. len(Added())+len(Removed()) exceeds one), this // initiates an atomic replication change in which the "removed" replicas are -// of type ReplicaType_VoterOutgoing. This joint configuration is left via another +// of type VOTER_OUTGOING. This joint configuration is left via another // ChangeReplicasTrigger which does not specify any additions nor removals. message ChangeReplicasTrigger { option (gogoproto.equal) = true; diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 56af173c8259..115a232b529b 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1636,9 +1636,9 @@ func TestUpdateObservedTimestamps(t *testing.T) { func TestChangeReplicasTrigger_String(t *testing.T) { defer leaktest.AfterTest(t)() - vi := ReplicaType_VoterIncoming - vo := ReplicaType_VoterOutgoing - l := ReplicaType_Learner + vi := VOTER_INCOMING + vo := VOTER_OUTGOING + l := LEARNER repl1 := ReplicaDescriptor{NodeID: 1, StoreID: 2, ReplicaID: 3, Type: &vi} repl2 := ReplicaDescriptor{NodeID: 4, StoreID: 5, ReplicaID: 6, Type: &vo} learner := ReplicaDescriptor{NodeID: 7, StoreID: 8, ReplicaID: 9, Type: &l} @@ -1660,7 +1660,7 @@ func TestChangeReplicasTrigger_String(t *testing.T) { }, } act := crt.String() - exp := "ENTER_JOINT ADD_REPLICA[(n1,s2):3VOTERINCOMING], REMOVE_REPLICA[(n4,s5):6VOTEROUTGOING]: after=[(n1,s2):3VOTERINCOMING (n4,s5):6VOTEROUTGOING (n7,s8):9LEARNER] next=10" + exp := "ENTER_JOINT ADD_REPLICA[(n1,s2):3VOTER_INCOMING], REMOVE_REPLICA[(n4,s5):6VOTER_OUTGOING]: after=[(n1,s2):3VOTER_INCOMING (n4,s5):6VOTER_OUTGOING (n7,s8):9LEARNER] next=10" require.Equal(t, exp, act) crt.InternalRemovedReplicas = nil @@ -1719,26 +1719,26 @@ func TestChangeReplicasTrigger_ConfChange(t *testing.T) { return m } - vf1 := sl(ReplicaType_VoterFull, 1) - vo1 := sl(ReplicaType_VoterOutgoing, 1) - vi1 := sl(ReplicaType_VoterIncoming, 1) - vl1 := sl(ReplicaType_Learner, 1) + vf1 := sl(VOTER_FULL, 1) + vo1 := sl(VOTER_OUTGOING, 1) + vi1 := sl(VOTER_INCOMING, 1) + vl1 := sl(LEARNER, 1) testCases := []struct { crt mockCRT exp raftpb.ConfChangeI err string }{ - // A replica of type VoterOutgoing being added makes no sense. - {crt: mk(in{add: vo1, repls: vo1}), err: "can't add replica in state VoterOutgoing"}, + // A replica of type VOTER_OUTGOING being added makes no sense. + {crt: mk(in{add: vo1, repls: vo1}), err: "can't add replica in state VOTER_OUTGOING"}, // But an incoming one can be added, and the result must be a joint change. {crt: mk(in{add: vi1, repls: vi1}), exp: raftpb.ConfChangeV2{ Transition: raftpb.ConfChangeTransitionJointExplicit, Changes: []raftpb.ConfChangeSingle{{Type: raftpb.ConfChangeAddNode, NodeID: 1}}, }}, - // A replica of type VoterIncoming being removed makes no sense. - {crt: mk(in{del: vi1}), err: "can't remove replica in state VoterIncoming"}, - // But during a joint removal we can see VoterOutgoing. + // A replica of type VOTER_INCOMING being removed makes no sense. + {crt: mk(in{del: vi1}), err: "can't remove replica in state VOTER_INCOMING"}, + // But during a joint removal we can see VOTER_OUTGOING. {crt: mk(in{del: vo1, repls: vo1}), exp: raftpb.ConfChangeV2{ Transition: raftpb.ConfChangeTransitionJointExplicit, Changes: []raftpb.ConfChangeSingle{{Type: raftpb.ConfChangeRemoveNode, NodeID: 1}}, @@ -1825,21 +1825,21 @@ func TestChangeReplicasTrigger_ConfChange(t *testing.T) { // Run a more complex change (necessarily) via the V2 path. {crt: mk(in{ add: sl( // Additions. - ReplicaType_VoterIncoming, 6, ReplicaType_Learner, 4, ReplicaType_VoterIncoming, 3, + VOTER_INCOMING, 6, LEARNER, 4, VOTER_INCOMING, 3, ), del: sl( // Removals. - ReplicaType_Learner, 2, ReplicaType_VoterOutgoing, 8, ReplicaType_VoterOutgoing, 9, + LEARNER, 2, VOTER_OUTGOING, 8, VOTER_OUTGOING, 9, ), repls: sl( // Replicas. - ReplicaType_VoterFull, 1, - ReplicaType_VoterIncoming, 6, // added - ReplicaType_VoterIncoming, 3, // added - ReplicaType_VoterOutgoing, 9, // removing - ReplicaType_Learner, 4, // added - ReplicaType_VoterOutgoing, 8, // removing - ReplicaType_VoterFull, 10, + VOTER_FULL, 1, + VOTER_INCOMING, 6, // added + VOTER_INCOMING, 3, // added + VOTER_OUTGOING, 9, // removing + LEARNER, 4, // added + VOTER_OUTGOING, 8, // removing + VOTER_FULL, 10, )}), exp: raftpb.ConfChangeV2{ Transition: raftpb.ConfChangeTransitionJointExplicit, @@ -1855,17 +1855,17 @@ func TestChangeReplicasTrigger_ConfChange(t *testing.T) { // Leave a joint config. { - crt: mk(in{repls: sl(ReplicaType_VoterFull, 1)}), + crt: mk(in{repls: sl(VOTER_FULL, 1)}), exp: raftpb.ConfChangeV2{}, }, // If we're asked to leave a joint state but the descriptor is still joint, // that's a problem. { - crt: mk(in{v2: true, repls: sl(ReplicaType_VoterIncoming, 1)}), + crt: mk(in{v2: true, repls: sl(VOTER_INCOMING, 1)}), err: "descriptor enters joint state, but trigger is requesting to leave one", }, { - crt: mk(in{v2: true, repls: sl(ReplicaType_VoterOutgoing, 1)}), + crt: mk(in{v2: true, repls: sl(VOTER_OUTGOING, 1)}), err: "descriptor enters joint state, but trigger is requesting to leave one", }, } diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index a8e373681afb..63855311beb5 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -312,8 +312,8 @@ func (r ReplicaDescriptor) String() string { } else { fmt.Fprintf(&buf, "%d", r.ReplicaID) } - if typ := r.GetType(); typ != ReplicaType_VoterFull { - buf.WriteString(strings.ToUpper(typ.String())) + if typ := r.GetType(); typ != VOTER_FULL { + buf.WriteString(typ.String()) } return buf.String() } @@ -335,7 +335,7 @@ func (r ReplicaDescriptor) Validate() error { // GetType returns the type of this ReplicaDescriptor. func (r ReplicaDescriptor) GetType() ReplicaType { if r.Type == nil { - return ReplicaType_VoterFull + return VOTER_FULL } return *r.Type } diff --git a/pkg/roachpb/metadata.pb.go b/pkg/roachpb/metadata.pb.go index 4758722931ed..77d08338084b 100644 --- a/pkg/roachpb/metadata.pb.go +++ b/pkg/roachpb/metadata.pb.go @@ -26,20 +26,20 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package -// ReplicaType identifies which raft activities a replica participates in. -// In normal operation, Voter and Learner are the only used states. However, -// atomic replication changes require a transition through a "joint config"; -// in this joint config, the VoterOutgoing and VoterIncoming types are used -// as well to denote voters which are being removed and newly added by the -// change, respectively. +// ReplicaType identifies which raft activities a replica participates in. In +// normal operation, VOTER_FULL and LEARNER are the only used states. However, +// atomic replication changes require a transition through a "joint config"; in +// this joint config, the VOTER_OUTGOING and VOTER_INCOMING types are used as well +// to denote voters which are being removed and newly added by the change, +// respectively. // // All voter types indicate a replica that participates in all raft activities, // including voting for leadership and committing entries. Typically, this -// requires a majority of voters to reach a decision. In the joint config, +// requires a majority of voters to reach a decision. In a joint config, // two separate majorities are required: one from the set of replicas that -// have either type Voter or VoterOutgoing, as well as that of the set of -// types Voter and VoterIncoming. For example, when type Voter is assigned -// to replicas 1 and 2, while 3 is VoterOutgoing and 4 is VoterIncoming, then +// have either type VOTER or VOTER_OUTOING, as well as that of the set of +// types VOTER and VOTER_INCOMING . For example, when type VOTER_FULL is assigned +// to replicas 1 and 2, while 3 is VOTER_OUTGOING and 4 is VOTER_INCOMING, then // the two sets over which quorums need to be achieved are {1,2,3} and {1,2,4}. // Thus, {1,2} is a quorum of both, {1,3} is a quorum of the first but not the // second, {1,4} is a quorum of the second but not the first, and {3,4} is a @@ -47,40 +47,40 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package type ReplicaType int32 const ( - // ReplicaType_VoterFull indicates a replica that is a voter both in the + // VOTER_FULL indicates a replica that is a voter both in the // incoming and outgoing set. - ReplicaType_VoterFull ReplicaType = 0 - // ReplicaType_VoterIncoming indicates a voting replica that will be a - // VoterFull once the ongoing atomic replication change is finalized; that is, + VOTER_FULL ReplicaType = 0 + // VOTER_INCOMING indicates a voting replica that will be a + // VOTER_FULL once the ongoing atomic replication change is finalized; that is, // it is in the process of being added. In practice, this replica type should - // be treated like a VoterFull. - ReplicaType_VoterIncoming ReplicaType = 2 - // ReplicaType_VoterOutgoing indicates a voting replica that will not be part + // be treated like a VOTER_FULL. + VOTER_INCOMING ReplicaType = 2 + // VOTER_OUTGOING indicates a voting replica that will not be part // of the descriptor once the ongoing atomic replication change is finalized; // that is, it is in the process of being removed. In practice, a replica of // this type should be treated accordingly and no work should be assigned to // it. - ReplicaType_VoterOutgoing ReplicaType = 3 - // ReplicaType_Learner indicates a replica that applies committed entries, but + VOTER_OUTGOING ReplicaType = 3 + // LEARNER indicates a replica that applies committed entries, but // does not count towards the quorum(s). Learners do not vote for leadership // nor do their acknowledged log entries get taken into account for // determining the committed index. At the time of writing, learners in // CockroachDB are a short-term transient state: a replica being added and on - // its way to being a VOTER. - ReplicaType_Learner ReplicaType = 1 + // its way to being a VOTER_FULL. + LEARNER ReplicaType = 1 ) var ReplicaType_name = map[int32]string{ - 0: "VoterFull", - 2: "VoterIncoming", - 3: "VoterOutgoing", - 1: "Learner", + 0: "VOTER_FULL", + 2: "VOTER_INCOMING", + 3: "VOTER_OUTGOING", + 1: "LEARNER", } var ReplicaType_value = map[string]int32{ - "VoterFull": 0, - "VoterIncoming": 2, - "VoterOutgoing": 3, - "Learner": 1, + "VOTER_FULL": 0, + "VOTER_INCOMING": 2, + "VOTER_OUTGOING": 3, + "LEARNER": 1, } func (x ReplicaType) Enum() *ReplicaType { @@ -100,7 +100,7 @@ func (x *ReplicaType) UnmarshalJSON(data []byte) error { return nil } func (ReplicaType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{0} + return fileDescriptor_metadata_af2fa38859548c68, []int{0} } // Attributes specifies a list of arbitrary strings describing @@ -112,7 +112,7 @@ type Attributes struct { func (m *Attributes) Reset() { *m = Attributes{} } func (*Attributes) ProtoMessage() {} func (*Attributes) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{0} + return fileDescriptor_metadata_af2fa38859548c68, []int{0} } func (m *Attributes) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -146,7 +146,7 @@ type ReplicationTarget struct { func (m *ReplicationTarget) Reset() { *m = ReplicationTarget{} } func (*ReplicationTarget) ProtoMessage() {} func (*ReplicationTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{1} + return fileDescriptor_metadata_af2fa38859548c68, []int{1} } func (m *ReplicationTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -191,7 +191,7 @@ type ReplicaDescriptor struct { func (m *ReplicaDescriptor) Reset() { *m = ReplicaDescriptor{} } func (*ReplicaDescriptor) ProtoMessage() {} func (*ReplicaDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{2} + return fileDescriptor_metadata_af2fa38859548c68, []int{2} } func (m *ReplicaDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -226,7 +226,7 @@ func (m *ReplicaIdent) Reset() { *m = ReplicaIdent{} } func (m *ReplicaIdent) String() string { return proto.CompactTextString(m) } func (*ReplicaIdent) ProtoMessage() {} func (*ReplicaIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{3} + return fileDescriptor_metadata_af2fa38859548c68, []int{3} } func (m *ReplicaIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -383,7 +383,7 @@ type RangeDescriptor struct { func (m *RangeDescriptor) Reset() { *m = RangeDescriptor{} } func (*RangeDescriptor) ProtoMessage() {} func (*RangeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{4} + return fileDescriptor_metadata_af2fa38859548c68, []int{4} } func (m *RangeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -422,7 +422,7 @@ type Percentiles struct { func (m *Percentiles) Reset() { *m = Percentiles{} } func (*Percentiles) ProtoMessage() {} func (*Percentiles) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{5} + return fileDescriptor_metadata_af2fa38859548c68, []int{5} } func (m *Percentiles) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -484,7 +484,7 @@ type StoreCapacity struct { func (m *StoreCapacity) Reset() { *m = StoreCapacity{} } func (*StoreCapacity) ProtoMessage() {} func (*StoreCapacity) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{6} + return fileDescriptor_metadata_af2fa38859548c68, []int{6} } func (m *StoreCapacity) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -529,7 +529,7 @@ func (m *NodeDescriptor) Reset() { *m = NodeDescriptor{} } func (m *NodeDescriptor) String() string { return proto.CompactTextString(m) } func (*NodeDescriptor) ProtoMessage() {} func (*NodeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{7} + return fileDescriptor_metadata_af2fa38859548c68, []int{7} } func (m *NodeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -565,7 +565,7 @@ func (m *LocalityAddress) Reset() { *m = LocalityAddress{} } func (m *LocalityAddress) String() string { return proto.CompactTextString(m) } func (*LocalityAddress) ProtoMessage() {} func (*LocalityAddress) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{8} + return fileDescriptor_metadata_af2fa38859548c68, []int{8} } func (m *LocalityAddress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -603,7 +603,7 @@ func (m *StoreDescriptor) Reset() { *m = StoreDescriptor{} } func (m *StoreDescriptor) String() string { return proto.CompactTextString(m) } func (*StoreDescriptor) ProtoMessage() {} func (*StoreDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{9} + return fileDescriptor_metadata_af2fa38859548c68, []int{9} } func (m *StoreDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -640,7 +640,7 @@ func (m *StoreDeadReplicas) Reset() { *m = StoreDeadReplicas{} } func (m *StoreDeadReplicas) String() string { return proto.CompactTextString(m) } func (*StoreDeadReplicas) ProtoMessage() {} func (*StoreDeadReplicas) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{10} + return fileDescriptor_metadata_af2fa38859548c68, []int{10} } func (m *StoreDeadReplicas) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -674,7 +674,7 @@ type Locality struct { func (m *Locality) Reset() { *m = Locality{} } func (*Locality) ProtoMessage() {} func (*Locality) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{11} + return fileDescriptor_metadata_af2fa38859548c68, []int{11} } func (m *Locality) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -710,7 +710,7 @@ type Tier struct { func (m *Tier) Reset() { *m = Tier{} } func (*Tier) ProtoMessage() {} func (*Tier) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{12} + return fileDescriptor_metadata_af2fa38859548c68, []int{12} } func (m *Tier) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -751,7 +751,7 @@ type Version struct { func (m *Version) Reset() { *m = Version{} } func (*Version) ProtoMessage() {} func (*Version) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_e31f96670cc94504, []int{13} + return fileDescriptor_metadata_af2fa38859548c68, []int{13} } func (m *Version) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4031,94 +4031,96 @@ var ( ErrIntOverflowMetadata = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_e31f96670cc94504) } +func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_af2fa38859548c68) } -var fileDescriptor_metadata_e31f96670cc94504 = []byte{ - // 1376 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xcd, 0x6f, 0x1b, 0x45, - 0x14, 0xcf, 0xfa, 0x23, 0xb6, 0x9f, 0xe3, 0xc6, 0x1e, 0xb5, 0xc5, 0x32, 0xc2, 0x76, 0x0d, 0x15, - 0x69, 0x41, 0x49, 0x9a, 0x2a, 0xaa, 0x1a, 0x28, 0x10, 0xb7, 0x54, 0x0a, 0xfd, 0x0a, 0x9b, 0x50, - 0x24, 0x2e, 0xab, 0xc9, 0xee, 0xe0, 0x2c, 0x1d, 0xef, 0xba, 0xb3, 0xe3, 0xb4, 0xbe, 0x23, 0x71, - 0x40, 0x48, 0x5c, 0x90, 0x38, 0xf6, 0xd2, 0xff, 0x81, 0x3f, 0xa1, 0xc7, 0x1e, 0x7b, 0x8a, 0x4a, - 0x7a, 0xe1, 0xcc, 0xb1, 0x07, 0x84, 0xe6, 0xed, 0xcc, 0xee, 0x3a, 0x4d, 0xa1, 0x2d, 0x12, 0xb7, - 0xf5, 0xef, 0xfd, 0x7e, 0xcf, 0x6f, 0xde, 0xbc, 0x8f, 0x81, 0x93, 0x22, 0xa4, 0xee, 0xee, 0x68, - 0x67, 0x69, 0xc8, 0x24, 0xf5, 0xa8, 0xa4, 0x8b, 0x23, 0x11, 0xca, 0x90, 0x34, 0xdc, 0xd0, 0xbd, - 0x83, 0xb6, 0x45, 0xcd, 0x68, 0xb5, 0xc6, 0xd2, 0xe7, 0x4b, 0xe3, 0x40, 0xb0, 0x28, 0xe4, 0x7b, - 0xcc, 0x73, 0xa8, 0xe7, 0x89, 0x98, 0xde, 0x6a, 0xa2, 0x6d, 0x97, 0xbb, 0x4b, 0xd2, 0x1f, 0xb2, - 0x48, 0xd2, 0xe1, 0x48, 0x5b, 0x8e, 0x0f, 0xc2, 0x41, 0x88, 0x9f, 0x4b, 0xea, 0x2b, 0x46, 0x7b, - 0x9f, 0x02, 0xac, 0x4b, 0x29, 0xfc, 0x9d, 0xb1, 0x64, 0x11, 0xf9, 0x00, 0x8a, 0x54, 0x4a, 0x11, - 0x35, 0xad, 0x6e, 0x7e, 0xa1, 0xd2, 0x3f, 0xf1, 0xe7, 0x7e, 0xa7, 0x31, 0xa1, 0x43, 0xbe, 0xd6, - 0x43, 0xf8, 0xc3, 0x6f, 0x79, 0x78, 0xaf, 0x67, 0xc7, 0x9c, 0xb5, 0xc2, 0xaf, 0x0f, 0x3a, 0x33, - 0xbd, 0x1f, 0x2c, 0x68, 0xd8, 0x6c, 0xc4, 0x7d, 0x97, 0x4a, 0x3f, 0x0c, 0xb6, 0xa9, 0x18, 0x30, - 0x49, 0xce, 0x41, 0x29, 0x08, 0x3d, 0xe6, 0xf8, 0x5e, 0xd3, 0xea, 0x5a, 0x0b, 0xc5, 0x7e, 0xf3, - 0xd1, 0x7e, 0x67, 0xe6, 0x60, 0xbf, 0x33, 0x7b, 0x33, 0xf4, 0xd8, 0xc6, 0x95, 0xe7, 0xc9, 0x97, - 0x3d, 0xab, 0x88, 0x1b, 0x1e, 0x59, 0x85, 0x72, 0x24, 0x43, 0x81, 0x9a, 0x1c, 0x6a, 0x5a, 0x5a, - 0x53, 0xda, 0x52, 0x38, 0x8a, 0xcc, 0xa7, 0x5d, 0x42, 0xee, 0x86, 0xb7, 0x56, 0x56, 0x51, 0xfc, - 0xf1, 0xa0, 0x63, 0xf5, 0xfe, 0x4a, 0x23, 0xb9, 0xc2, 0x22, 0x57, 0xf8, 0x23, 0x19, 0x8a, 0xff, - 0x2f, 0x12, 0x72, 0x09, 0x40, 0xc4, 0x7f, 0xaf, 0x84, 0x79, 0x14, 0xb6, 0xb5, 0xb0, 0xa2, 0x03, - 0x43, 0x69, 0xfa, 0xc3, 0xae, 0x68, 0xc5, 0x86, 0x47, 0x56, 0xa0, 0x20, 0x27, 0x23, 0xd6, 0x2c, - 0x74, 0xad, 0x85, 0x63, 0x2b, 0xed, 0xc5, 0x17, 0xee, 0x7d, 0x51, 0xcb, 0xb6, 0x27, 0x23, 0x66, - 0x23, 0x77, 0x6d, 0x4e, 0x1d, 0xfe, 0xb7, 0x07, 0x1d, 0x0b, 0x13, 0xf0, 0xa3, 0x05, 0x73, 0xc6, - 0xb5, 0xc7, 0x02, 0xa9, 0x0e, 0x22, 0x68, 0x30, 0x48, 0x0e, 0x9f, 0x4f, 0x0f, 0x62, 0x2b, 0x3c, - 0x3e, 0x88, 0xfe, 0xb4, 0x4b, 0xc8, 0xdd, 0xf0, 0xc8, 0x15, 0x28, 0xe9, 0xb0, 0xf0, 0xf8, 0xd5, - 0x95, 0xf7, 0x5e, 0x1e, 0x4c, 0x9a, 0xe9, 0x7e, 0x41, 0xf9, 0xb6, 0x8d, 0xb4, 0xf7, 0x24, 0x0f, - 0xf3, 0xe8, 0x3a, 0x73, 0x19, 0x6f, 0x18, 0xd0, 0x69, 0xa8, 0x44, 0x92, 0x0a, 0xe9, 0xdc, 0x61, - 0x13, 0x0c, 0x69, 0xae, 0x5f, 0x7e, 0xbe, 0xdf, 0x29, 0xd8, 0xd7, 0xd8, 0xc4, 0x2e, 0xa3, 0xe9, - 0x1a, 0x9b, 0x90, 0x53, 0x50, 0x62, 0x81, 0x87, 0xa4, 0xfc, 0x21, 0xd2, 0x2c, 0x0b, 0x3c, 0x45, - 0xf9, 0x1a, 0x1a, 0x7e, 0x20, 0x99, 0x08, 0x28, 0x77, 0x74, 0xa0, 0x51, 0xb3, 0xd0, 0xcd, 0xbf, - 0xe6, 0x21, 0xeb, 0xc6, 0x89, 0x26, 0x44, 0xe4, 0x0b, 0x98, 0x0f, 0xd8, 0x7d, 0xe9, 0x64, 0x2a, - 0xa0, 0x88, 0x15, 0xd0, 0xd3, 0x07, 0xac, 0xdd, 0x64, 0xf7, 0xe5, 0x4b, 0xaa, 0xa0, 0x16, 0x64, - 0x6c, 0x1e, 0x69, 0x03, 0x0c, 0x58, 0xc0, 0x04, 0x36, 0x54, 0x73, 0x56, 0xe5, 0xc9, 0xce, 0x20, - 0xe4, 0x63, 0x80, 0x48, 0xfa, 0xee, 0x9d, 0x89, 0xb3, 0xe3, 0xcb, 0x66, 0x09, 0xaf, 0xe8, 0x9d, - 0x4c, 0xf4, 0x6a, 0x04, 0x2c, 0xee, 0x72, 0x77, 0x71, 0xdb, 0x8c, 0x00, 0xbb, 0x12, 0x0b, 0xfa, - 0xbe, 0x24, 0xe7, 0xe1, 0x44, 0xea, 0xcb, 0x71, 0xc3, 0xe1, 0x88, 0x0a, 0xba, 0xc3, 0x59, 0xb3, - 0xdc, 0xb5, 0x16, 0xca, 0xf6, 0xf1, 0xd4, 0x78, 0x39, 0xb1, 0x1d, 0x2a, 0xb4, 0x87, 0x16, 0x54, - 0x37, 0x99, 0x70, 0x59, 0x20, 0x7d, 0xce, 0x22, 0x72, 0x12, 0xf2, 0xa3, 0x73, 0xcb, 0x78, 0xa3, - 0x96, 0xce, 0x90, 0x02, 0x10, 0x5f, 0x59, 0xc5, 0x1b, 0x4b, 0xf1, 0x95, 0x55, 0xc4, 0x57, 0x97, - 0xf1, 0x92, 0x52, 0x7c, 0x35, 0xe6, 0x5f, 0x58, 0xc5, 0x0e, 0x48, 0xf1, 0x0b, 0x31, 0xff, 0xe2, - 0x32, 0x26, 0x34, 0xc5, 0x2f, 0x2e, 0x93, 0x26, 0x14, 0x46, 0x37, 0xe8, 0x7d, 0x4c, 0x91, 0x31, - 0x20, 0xa2, 0x67, 0xd3, 0xf3, 0x3c, 0xd4, 0xb0, 0x4d, 0x2f, 0xd3, 0x11, 0x75, 0x7d, 0x39, 0x21, - 0x5d, 0x28, 0xbb, 0xfa, 0x5b, 0x17, 0x60, 0xac, 0x4a, 0x50, 0xd2, 0x83, 0x0a, 0xdd, 0xa3, 0x3e, - 0xc7, 0x94, 0xe4, 0x32, 0x94, 0x14, 0x26, 0xa7, 0xa1, 0x1a, 0x97, 0xb1, 0x1b, 0x8e, 0x03, 0xa9, - 0x5b, 0x3d, 0x66, 0x01, 0x1a, 0x2e, 0x2b, 0x5c, 0xd1, 0x38, 0xa3, 0x91, 0xa1, 0x15, 0xb2, 0x34, - 0x34, 0xc4, 0xb4, 0x65, 0x68, 0xdc, 0x13, 0xbe, 0x64, 0x91, 0x33, 0x62, 0xc2, 0x89, 0x98, 0x1b, - 0x06, 0xde, 0xd4, 0x59, 0xe7, 0x63, 0xf3, 0x26, 0x13, 0x5b, 0x68, 0x24, 0x9b, 0xd0, 0xd8, 0x99, - 0x18, 0x81, 0x69, 0xd5, 0x59, 0xac, 0x83, 0xa3, 0xe6, 0x46, 0xe6, 0xaa, 0x8c, 0x47, 0x94, 0x6f, - 0x32, 0xa1, 0x8b, 0x8e, 0xd8, 0x40, 0x32, 0x31, 0x18, 0x97, 0xa5, 0xd7, 0x70, 0x59, 0x4f, 0x82, - 0x34, 0x3e, 0x9b, 0x50, 0x18, 0x47, 0xcc, 0xc3, 0xba, 0x32, 0x49, 0x44, 0x84, 0x9c, 0x81, 0x1a, - 0x0f, 0x07, 0xbe, 0x4b, 0xb9, 0x83, 0x81, 0x34, 0x2b, 0x19, 0xca, 0x9c, 0x36, 0xf5, 0x95, 0x85, - 0xac, 0x00, 0xb9, 0x3b, 0x66, 0xc2, 0x9f, 0xce, 0x0e, 0x64, 0xb2, 0x53, 0xd7, 0xf6, 0x24, 0x3d, - 0xfa, 0xf2, 0x9f, 0x16, 0xe0, 0x98, 0x1a, 0xec, 0xff, 0x6d, 0x17, 0x7c, 0x02, 0x25, 0xb5, 0x5d, - 0x59, 0x14, 0xe9, 0x59, 0xd8, 0x3e, 0xdc, 0x68, 0x5f, 0x25, 0x7b, 0x78, 0xdd, 0xf3, 0x92, 0x29, - 0xa8, 0x45, 0xe4, 0xa2, 0xd9, 0xa8, 0xf9, 0x17, 0xda, 0xd4, 0xe4, 0x32, 0xdd, 0xbf, 0x5a, 0x1c, - 0x2b, 0xc8, 0x25, 0x28, 0xf3, 0xd0, 0xa5, 0x5c, 0xd5, 0x6a, 0x01, 0xd5, 0x6f, 0x1f, 0xa1, 0xbe, - 0xae, 0x29, 0xa6, 0x90, 0x8d, 0x84, 0x5c, 0x85, 0xda, 0x16, 0x13, 0x7b, 0x4c, 0xdc, 0x66, 0x22, - 0x52, 0x83, 0xa4, 0x88, 0x3e, 0x5a, 0x47, 0xf8, 0xd0, 0x0c, 0xed, 0x62, 0x5a, 0x46, 0x4e, 0x41, - 0x65, 0x67, 0xec, 0x73, 0xcf, 0x91, 0x74, 0x80, 0x45, 0x56, 0x31, 0x7f, 0x85, 0xf0, 0x36, 0x1d, - 0x90, 0x77, 0xd5, 0x40, 0xa2, 0x42, 0xaa, 0xa7, 0x48, 0x3c, 0x90, 0x92, 0xa6, 0xd1, 0xf8, 0xba, - 0x24, 0x5b, 0x50, 0x37, 0xb1, 0x39, 0x26, 0xa5, 0x65, 0x9c, 0xbc, 0xbd, 0x7f, 0x38, 0xd6, 0x7a, - 0xcc, 0x34, 0x75, 0xcb, 0xa7, 0x61, 0xf2, 0x3e, 0xcc, 0xb9, 0x7c, 0x1c, 0x49, 0x26, 0x9c, 0x80, - 0x0e, 0x19, 0x16, 0x92, 0x89, 0xaf, 0xaa, 0x2d, 0x37, 0xe9, 0x90, 0x91, 0x2d, 0xa8, 0x46, 0x77, - 0x79, 0xf2, 0xc7, 0xf0, 0x4a, 0x77, 0x49, 0x74, 0x79, 0xc0, 0xd6, 0x97, 0xd7, 0xf5, 0x3f, 0xda, - 0x10, 0xdd, 0xe5, 0xfa, 0xbb, 0xf7, 0x8b, 0x05, 0xf3, 0x87, 0x02, 0xcd, 0x16, 0x8c, 0xf5, 0x26, - 0x05, 0xd3, 0x57, 0xbd, 0xa1, 0xd3, 0x24, 0x7d, 0x26, 0x74, 0xd9, 0xbd, 0x75, 0x44, 0x8e, 0xb6, - 0x7d, 0x26, 0xd2, 0xa6, 0x89, 0x35, 0x0a, 0xeb, 0x7d, 0x9f, 0x83, 0x79, 0x9c, 0x7b, 0xd3, 0xab, - 0x37, 0x79, 0xd4, 0x58, 0xaf, 0xfe, 0xa8, 0x49, 0xea, 0x37, 0xf7, 0xda, 0xf5, 0xfb, 0x11, 0x14, - 0x54, 0x13, 0xe9, 0xca, 0x3f, 0x75, 0x84, 0x72, 0xba, 0x3d, 0xcd, 0x88, 0x50, 0x22, 0xd2, 0xcf, - 0x0c, 0xea, 0xb8, 0xf8, 0xbb, 0x47, 0x38, 0x98, 0x1a, 0xee, 0x87, 0x47, 0x79, 0xef, 0x27, 0x0b, - 0x1a, 0x3a, 0x0d, 0xd4, 0x4b, 0x36, 0xf5, 0x1b, 0x26, 0x62, 0x1d, 0xca, 0xc9, 0x83, 0x21, 0x87, - 0x65, 0xdb, 0x79, 0xf9, 0x83, 0x01, 0x9f, 0x5f, 0x26, 0x1e, 0x23, 0xeb, 0x7d, 0x0e, 0x65, 0x53, - 0x2d, 0xe4, 0x3c, 0x14, 0xd5, 0xed, 0xc6, 0x2f, 0xed, 0x7f, 0xbd, 0xde, 0x98, 0xab, 0x07, 0xdb, - 0x67, 0x50, 0x50, 0x26, 0xb5, 0x15, 0xd5, 0x53, 0xc7, 0xca, 0x94, 0xbc, 0x02, 0x48, 0x0b, 0x8a, - 0x7b, 0x94, 0x8f, 0xe3, 0xed, 0x65, 0x2c, 0x31, 0xa4, 0x3d, 0x3c, 0xb4, 0xa0, 0x64, 0xda, 0xfb, - 0x2c, 0x54, 0x86, 0xf4, 0xbb, 0x50, 0x38, 0x7b, 0x94, 0xeb, 0x7c, 0xd4, 0x74, 0x3e, 0x8a, 0x37, - 0x94, 0xc1, 0x2e, 0xa3, 0xfd, 0x36, 0xe5, 0xc8, 0xf5, 0x03, 0xcd, 0xcd, 0x1d, 0xe2, 0x2a, 0x83, - 0x5d, 0x46, 0xbb, 0xe2, 0xb6, 0xa0, 0x38, 0xa2, 0xd2, 0xdd, 0x9d, 0xda, 0x8e, 0x31, 0xa4, 0xb6, - 0xf0, 0x38, 0x88, 0x24, 0xae, 0xd8, 0xec, 0x56, 0x4c, 0xd0, 0x38, 0xce, 0xb3, 0xb7, 0xa0, 0x9a, - 0x79, 0xf3, 0x92, 0x1a, 0x54, 0x6e, 0x87, 0x92, 0x89, 0xab, 0x63, 0xce, 0xeb, 0x33, 0xa4, 0x01, - 0x35, 0xfc, 0xb9, 0x11, 0xb8, 0xe1, 0xd0, 0x0f, 0x06, 0xf5, 0x5c, 0x02, 0xdd, 0x1a, 0xcb, 0x41, - 0xa8, 0xa0, 0x3c, 0xa9, 0x42, 0xe9, 0x3a, 0xa3, 0x22, 0x60, 0xa2, 0x6e, 0xf5, 0xcf, 0x3c, 0xfa, - 0xbd, 0x3d, 0xf3, 0xe8, 0xa0, 0x6d, 0x3d, 0x3e, 0x68, 0x5b, 0x4f, 0x0e, 0xda, 0xd6, 0xd3, 0x83, - 0xb6, 0xf5, 0xf3, 0xb3, 0xf6, 0xcc, 0xe3, 0x67, 0xed, 0x99, 0x27, 0xcf, 0xda, 0x33, 0xdf, 0x94, - 0x74, 0xf6, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x84, 0x14, 0xea, 0x91, 0x90, 0x0d, 0x00, 0x00, +var fileDescriptor_metadata_af2fa38859548c68 = []byte{ + // 1396 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0x4d, 0x6f, 0xdb, 0xc6, + 0x16, 0x15, 0x25, 0xca, 0x92, 0xae, 0xfc, 0x21, 0x0d, 0x92, 0x3c, 0x41, 0x0f, 0x4f, 0x52, 0xf8, + 0x5e, 0xf0, 0x9c, 0xb4, 0xb0, 0x1d, 0x07, 0x46, 0x10, 0xb7, 0x69, 0x6b, 0x39, 0x4e, 0xa0, 0xc6, + 0xb1, 0x5d, 0x5a, 0x49, 0x8b, 0x6e, 0x88, 0x31, 0x39, 0x95, 0xd9, 0x50, 0x24, 0x33, 0x1c, 0x39, + 0xd1, 0xbe, 0x40, 0x0b, 0x14, 0x05, 0xba, 0x29, 0xd0, 0x65, 0x80, 0x22, 0xff, 0xa1, 0x3f, 0x21, + 0xcb, 0x2c, 0xb3, 0x32, 0x52, 0x67, 0xd3, 0x75, 0x97, 0x59, 0x14, 0xc5, 0x0c, 0x67, 0x48, 0xca, + 0x71, 0xda, 0x24, 0x05, 0xba, 0xa3, 0xce, 0x3d, 0xe7, 0xea, 0xce, 0x9d, 0xfb, 0x31, 0x70, 0x86, + 0x06, 0xd8, 0xde, 0x0f, 0xf7, 0x16, 0x87, 0x84, 0x61, 0x07, 0x33, 0xbc, 0x10, 0xd2, 0x80, 0x05, + 0xa8, 0x6e, 0x07, 0xf6, 0x5d, 0x61, 0x5b, 0x90, 0x8c, 0x66, 0x73, 0xc4, 0x5c, 0x6f, 0x71, 0xe4, + 0x53, 0x12, 0x05, 0xde, 0x01, 0x71, 0x2c, 0xec, 0x38, 0x34, 0xa6, 0x37, 0x1b, 0xc2, 0xb6, 0xef, + 0xd9, 0x8b, 0xcc, 0x1d, 0x92, 0x88, 0xe1, 0x61, 0x28, 0x2d, 0xa7, 0x06, 0xc1, 0x20, 0x10, 0x9f, + 0x8b, 0xfc, 0x2b, 0x46, 0x8d, 0x0f, 0x01, 0xd6, 0x18, 0xa3, 0xee, 0xde, 0x88, 0x91, 0x08, 0xbd, + 0x03, 0x45, 0xcc, 0x18, 0x8d, 0x1a, 0x5a, 0xa7, 0x30, 0x5f, 0xe9, 0x9e, 0xfe, 0xed, 0xb0, 0x5d, + 0x1f, 0xe3, 0xa1, 0xb7, 0x6a, 0x08, 0xf8, 0xdd, 0x2f, 0xbc, 0xe0, 0xbe, 0x61, 0xc6, 0x9c, 0x55, + 0xfd, 0xc7, 0x87, 0xed, 0x9c, 0xf1, 0xb5, 0x06, 0x75, 0x93, 0x84, 0x9e, 0x6b, 0x63, 0xe6, 0x06, + 0x7e, 0x1f, 0xd3, 0x01, 0x61, 0xe8, 0x22, 0x94, 0xfc, 0xc0, 0x21, 0x96, 0xeb, 0x34, 0xb4, 0x8e, + 0x36, 0x5f, 0xec, 0x36, 0x1e, 0x1f, 0xb6, 0x73, 0x47, 0x87, 0xed, 0xa9, 0xad, 0xc0, 0x21, 0xbd, + 0x6b, 0x2f, 0x92, 0x2f, 0x73, 0x8a, 0x13, 0x7b, 0x0e, 0x5a, 0x81, 0x72, 0xc4, 0x02, 0x2a, 0x34, + 0x79, 0xa1, 0x69, 0x4a, 0x4d, 0x69, 0x97, 0xe3, 0x42, 0xa4, 0x3e, 0xcd, 0x92, 0xe0, 0xf6, 0x9c, + 0xd5, 0x32, 0x8f, 0xe2, 0xd7, 0x87, 0x6d, 0xcd, 0xf8, 0x3d, 0x8d, 0xe4, 0x1a, 0x89, 0x6c, 0xea, + 0x86, 0x2c, 0xa0, 0xff, 0x5c, 0x24, 0xe8, 0x2a, 0x00, 0x8d, 0xff, 0x9e, 0x0b, 0x0b, 0x42, 0xd8, + 0x92, 0xc2, 0x8a, 0x0c, 0x4c, 0x48, 0xd3, 0x1f, 0x66, 0x45, 0x2a, 0x7a, 0x0e, 0x5a, 0x06, 0x9d, + 0x8d, 0x43, 0xd2, 0xd0, 0x3b, 0xda, 0xfc, 0xec, 0x72, 0x6b, 0xe1, 0xa5, 0x7b, 0x5f, 0x90, 0xb2, + 0xfe, 0x38, 0x24, 0xa6, 0xe0, 0xae, 0x4e, 0xf3, 0xc3, 0xff, 0xfc, 0xb0, 0xad, 0x89, 0x04, 0x7c, + 0xab, 0xc1, 0xb4, 0x72, 0xed, 0x10, 0x9f, 0xf1, 0x83, 0x50, 0xec, 0x0f, 0x92, 0xc3, 0x17, 0xd2, + 0x83, 0x98, 0x1c, 0x8f, 0x0f, 0x22, 0x3f, 0xcd, 0x92, 0xe0, 0xf6, 0x1c, 0x74, 0x0d, 0x4a, 0x32, + 0x2c, 0x71, 0xfc, 0xea, 0xf2, 0xff, 0x5e, 0x1d, 0x4c, 0x9a, 0xe9, 0xae, 0xce, 0x7d, 0x9b, 0x4a, + 0x6a, 0x3c, 0x2d, 0xc0, 0x9c, 0x70, 0x9d, 0xb9, 0x8c, 0xb7, 0x0c, 0xe8, 0x1c, 0x54, 0x22, 0x86, + 0x29, 0xb3, 0xee, 0x92, 0xb1, 0x08, 0x69, 0xba, 0x5b, 0x7e, 0x71, 0xd8, 0xd6, 0xcd, 0x9b, 0x64, + 0x6c, 0x96, 0x85, 0xe9, 0x26, 0x19, 0xa3, 0xb3, 0x50, 0x22, 0xbe, 0x23, 0x48, 0x85, 0x63, 0xa4, + 0x29, 0xe2, 0x3b, 0x9c, 0xf2, 0x29, 0xd4, 0x5d, 0x9f, 0x11, 0xea, 0x63, 0xcf, 0x92, 0x81, 0x46, + 0x0d, 0xbd, 0x53, 0x78, 0xc3, 0x43, 0xd6, 0x94, 0x13, 0x49, 0x88, 0xd0, 0xc7, 0x30, 0xe7, 0x93, + 0x07, 0xcc, 0xca, 0x54, 0x40, 0x51, 0x54, 0x80, 0x21, 0x0f, 0x38, 0xb3, 0x45, 0x1e, 0xb0, 0x57, + 0x54, 0xc1, 0x8c, 0x9f, 0xb1, 0x39, 0xa8, 0x05, 0x30, 0x20, 0x3e, 0xa1, 0xa2, 0xa1, 0x1a, 0x53, + 0x3c, 0x4f, 0x66, 0x06, 0x41, 0xef, 0x03, 0x44, 0xcc, 0xb5, 0xef, 0x8e, 0xad, 0x3d, 0x97, 0x35, + 0x4a, 0xe2, 0x8a, 0xfe, 0x93, 0x89, 0x9e, 0x8f, 0x80, 0x85, 0x7d, 0xcf, 0x5e, 0xe8, 0xab, 0x11, + 0x60, 0x56, 0x62, 0x41, 0xd7, 0x65, 0xe8, 0x12, 0x9c, 0x4e, 0x7d, 0x59, 0x76, 0x30, 0x0c, 0x31, + 0xc5, 0x7b, 0x1e, 0x69, 0x94, 0x3b, 0xda, 0x7c, 0xd9, 0x3c, 0x95, 0x1a, 0xd7, 0x13, 0xdb, 0xb1, + 0x42, 0x7b, 0xa4, 0x41, 0x75, 0x87, 0x50, 0x9b, 0xf8, 0xcc, 0xf5, 0x48, 0x84, 0xce, 0x40, 0x21, + 0xbc, 0xb8, 0x24, 0x6e, 0x54, 0x93, 0x19, 0xe2, 0x80, 0xc0, 0x97, 0x57, 0xc4, 0x8d, 0xa5, 0xf8, + 0xf2, 0x8a, 0xc0, 0x57, 0x96, 0xc4, 0x25, 0xa5, 0xf8, 0x4a, 0xcc, 0xbf, 0xbc, 0x22, 0x3a, 0x20, + 0xc5, 0x2f, 0xc7, 0xfc, 0x2b, 0x4b, 0x22, 0xa1, 0x29, 0x7e, 0x65, 0x09, 0x35, 0x40, 0x0f, 0x6f, + 0xe1, 0x07, 0x22, 0x45, 0xca, 0x20, 0x10, 0x39, 0x9b, 0x5e, 0x14, 0x60, 0x46, 0xb4, 0xe9, 0x3a, + 0x0e, 0xb1, 0xed, 0xb2, 0x31, 0xea, 0x40, 0xd9, 0x96, 0xdf, 0xb2, 0x00, 0x63, 0x55, 0x82, 0x22, + 0x03, 0x2a, 0xf8, 0x00, 0xbb, 0x9e, 0x48, 0x49, 0x3e, 0x43, 0x49, 0x61, 0x74, 0x0e, 0xaa, 0x71, + 0x19, 0xdb, 0xc1, 0xc8, 0x67, 0xb2, 0xd5, 0x63, 0x16, 0x08, 0xc3, 0x3a, 0xc7, 0x39, 0xcd, 0x23, + 0x38, 0x52, 0x34, 0x3d, 0x4b, 0x13, 0x86, 0x98, 0xb6, 0x04, 0xf5, 0xfb, 0xd4, 0x65, 0x24, 0xb2, + 0x42, 0x42, 0xad, 0x88, 0xd8, 0x81, 0xef, 0x4c, 0x9c, 0x75, 0x2e, 0x36, 0xef, 0x10, 0xba, 0x2b, + 0x8c, 0x68, 0x07, 0xea, 0x7b, 0x63, 0x25, 0x50, 0xad, 0x3a, 0x25, 0xea, 0xe0, 0xa4, 0xb9, 0x91, + 0xb9, 0x2a, 0xe5, 0x51, 0xc8, 0x77, 0x08, 0x95, 0x45, 0x87, 0x4c, 0x40, 0x99, 0x18, 0x94, 0xcb, + 0xd2, 0x1b, 0xb8, 0xac, 0x25, 0x41, 0x2a, 0x9f, 0x0d, 0xd0, 0x47, 0x11, 0x71, 0x44, 0x5d, 0xa9, + 0x24, 0x0a, 0x04, 0x9d, 0x87, 0x19, 0x2f, 0x18, 0xb8, 0x36, 0xf6, 0x2c, 0x11, 0x48, 0xa3, 0x92, + 0xa1, 0x4c, 0x4b, 0x53, 0x97, 0x5b, 0xd0, 0x32, 0xa0, 0x7b, 0x23, 0x42, 0xdd, 0xc9, 0xec, 0x40, + 0x26, 0x3b, 0x35, 0x69, 0x4f, 0xd2, 0x23, 0x2f, 0xff, 0x99, 0x0e, 0xb3, 0x7c, 0xb0, 0xff, 0xbd, + 0x5d, 0xf0, 0x01, 0x94, 0xf8, 0x76, 0x25, 0x51, 0x24, 0x67, 0x61, 0xeb, 0x78, 0xa3, 0xdd, 0x4e, + 0xf6, 0xf0, 0x9a, 0xe3, 0x24, 0x53, 0x50, 0x8a, 0xd0, 0x15, 0xb5, 0x51, 0x0b, 0x2f, 0xb5, 0xa9, + 0xca, 0x65, 0xba, 0x7f, 0xa5, 0x38, 0x56, 0xa0, 0xab, 0x50, 0xf6, 0x02, 0x1b, 0x7b, 0xbc, 0x56, + 0x75, 0xa1, 0xfe, 0xf7, 0x09, 0xea, 0x4d, 0x49, 0x51, 0x85, 0xac, 0x24, 0xe8, 0x3a, 0xcc, 0xec, + 0x12, 0x7a, 0x40, 0xe8, 0x1d, 0x42, 0x23, 0x3e, 0x48, 0x8a, 0xc2, 0x47, 0xf3, 0x04, 0x1f, 0x92, + 0x21, 0x5d, 0x4c, 0xca, 0xd0, 0x59, 0xa8, 0xec, 0x8d, 0x5c, 0xcf, 0xb1, 0x18, 0x1e, 0x88, 0x22, + 0xab, 0xa8, 0xbf, 0x12, 0x70, 0x1f, 0x0f, 0xd0, 0x7f, 0xf9, 0x40, 0xc2, 0x94, 0xf1, 0xa7, 0x48, + 0x3c, 0x90, 0x92, 0xa6, 0x91, 0xf8, 0x1a, 0x43, 0xbb, 0x50, 0x53, 0xb1, 0x59, 0x2a, 0xa5, 0x65, + 0x31, 0x79, 0x8d, 0x3f, 0x39, 0xd6, 0x5a, 0xcc, 0x54, 0x75, 0xeb, 0x4d, 0xc2, 0xe8, 0xff, 0x30, + 0x6d, 0x7b, 0xa3, 0x88, 0x11, 0x6a, 0xf9, 0x78, 0x48, 0x44, 0x21, 0xa9, 0xf8, 0xaa, 0xd2, 0xb2, + 0x85, 0x87, 0x04, 0xed, 0x42, 0x35, 0xba, 0xe7, 0x25, 0x7f, 0x0c, 0xaf, 0x75, 0x97, 0x48, 0x96, + 0x07, 0xec, 0x7e, 0xb2, 0x29, 0xff, 0xd1, 0x84, 0xe8, 0x9e, 0x27, 0xbf, 0x8d, 0x1f, 0x34, 0x98, + 0x3b, 0x16, 0x68, 0xb6, 0x60, 0xb4, 0xb7, 0x29, 0x98, 0x2e, 0xef, 0x0d, 0x99, 0x26, 0xe6, 0x12, + 0x2a, 0xcb, 0xee, 0x5f, 0x27, 0xe4, 0xa8, 0xef, 0x12, 0x9a, 0x36, 0x4d, 0xac, 0xe1, 0x98, 0xf1, + 0x55, 0x1e, 0xe6, 0xc4, 0xdc, 0x9b, 0x5c, 0xbd, 0xc9, 0xa3, 0x46, 0x7b, 0xfd, 0x47, 0x4d, 0x52, + 0xbf, 0xf9, 0x37, 0xae, 0xdf, 0xf7, 0x40, 0xe7, 0x4d, 0x24, 0x2b, 0xff, 0xec, 0x09, 0xca, 0xc9, + 0xf6, 0x54, 0x23, 0x82, 0x8b, 0x50, 0x37, 0x33, 0xa8, 0xe3, 0xe2, 0xef, 0x9c, 0xe0, 0x60, 0x62, + 0xb8, 0x1f, 0x1f, 0xe5, 0xc6, 0x77, 0x1a, 0xd4, 0x65, 0x1a, 0xb0, 0x93, 0x6c, 0xea, 0xb7, 0x4c, + 0xc4, 0x1a, 0x94, 0x93, 0x07, 0x43, 0x5e, 0x94, 0x6d, 0xfb, 0xd5, 0x0f, 0x06, 0xf1, 0xfc, 0x52, + 0xf1, 0x28, 0x99, 0xb1, 0x01, 0x65, 0x55, 0x2d, 0xe8, 0x12, 0x14, 0xf9, 0xed, 0xc6, 0x2f, 0xed, + 0xbf, 0xbc, 0xde, 0x98, 0x2b, 0x07, 0xdb, 0x47, 0xa0, 0x73, 0x13, 0xdf, 0x8a, 0xfc, 0xa9, 0xa3, + 0x65, 0x4a, 0x9e, 0x03, 0xa8, 0x09, 0xc5, 0x03, 0xec, 0x8d, 0xe2, 0xed, 0xa5, 0x2c, 0x31, 0x24, + 0x3d, 0x3c, 0xd2, 0xa0, 0xa4, 0xda, 0xfb, 0x02, 0x54, 0x86, 0xf8, 0xcb, 0x80, 0x5a, 0x07, 0xd8, + 0x93, 0xf9, 0x98, 0x91, 0xf9, 0x28, 0xde, 0xe2, 0x06, 0xb3, 0x2c, 0xec, 0x77, 0xb0, 0x27, 0xb8, + 0xae, 0x2f, 0xb9, 0xf9, 0x63, 0x5c, 0x6e, 0x30, 0xcb, 0xc2, 0xce, 0xb9, 0x4d, 0x28, 0x86, 0x98, + 0xd9, 0xfb, 0x13, 0xdb, 0x31, 0x86, 0xf8, 0x16, 0x1e, 0xf9, 0x11, 0x13, 0x2b, 0x36, 0xbb, 0x15, + 0x13, 0x34, 0x8e, 0xf3, 0xc2, 0x67, 0x50, 0xcd, 0xbc, 0x79, 0xd1, 0x2c, 0xc0, 0x9d, 0xed, 0xfe, + 0x86, 0x69, 0x5d, 0xbf, 0xbd, 0xb9, 0x59, 0xcb, 0x21, 0x04, 0xb3, 0xf1, 0xef, 0xde, 0xd6, 0xfa, + 0xf6, 0xad, 0xde, 0xd6, 0x8d, 0x5a, 0x3e, 0xc5, 0xb6, 0x6f, 0xf7, 0x6f, 0x6c, 0x73, 0xac, 0x80, + 0xaa, 0x50, 0xda, 0xdc, 0x58, 0x33, 0xb7, 0x36, 0xcc, 0x9a, 0xd6, 0xd4, 0xbf, 0xf9, 0xa9, 0x95, + 0xeb, 0x9e, 0x7f, 0xfc, 0x4b, 0x2b, 0xf7, 0xf8, 0xa8, 0xa5, 0x3d, 0x39, 0x6a, 0x69, 0x4f, 0x8f, + 0x5a, 0xda, 0xb3, 0xa3, 0x96, 0xf6, 0xfd, 0xf3, 0x56, 0xee, 0xc9, 0xf3, 0x56, 0xee, 0xe9, 0xf3, + 0x56, 0xee, 0xf3, 0x92, 0xbc, 0x86, 0x3f, 0x02, 0x00, 0x00, 0xff, 0xff, 0xee, 0xcc, 0xbd, 0x37, + 0x99, 0x0d, 0x00, 0x00, } diff --git a/pkg/roachpb/metadata.proto b/pkg/roachpb/metadata.proto index 7d09c2927e74..65965262a57f 100644 --- a/pkg/roachpb/metadata.proto +++ b/pkg/roachpb/metadata.proto @@ -35,46 +35,48 @@ message ReplicationTarget { (gogoproto.customname) = "StoreID", (gogoproto.casttype) = "StoreID"]; } -// ReplicaType identifies which raft activities a replica participates in. -// In normal operation, Voter and Learner are the only used states. However, -// atomic replication changes require a transition through a "joint config"; -// in this joint config, the VoterOutgoing and VoterIncoming types are used -// as well to denote voters which are being removed and newly added by the -// change, respectively. +// ReplicaType identifies which raft activities a replica participates in. In +// normal operation, VOTER_FULL and LEARNER are the only used states. However, +// atomic replication changes require a transition through a "joint config"; in +// this joint config, the VOTER_OUTGOING and VOTER_INCOMING types are used as well +// to denote voters which are being removed and newly added by the change, +// respectively. // // All voter types indicate a replica that participates in all raft activities, // including voting for leadership and committing entries. Typically, this -// requires a majority of voters to reach a decision. In the joint config, +// requires a majority of voters to reach a decision. In a joint config, // two separate majorities are required: one from the set of replicas that -// have either type Voter or VoterOutgoing, as well as that of the set of -// types Voter and VoterIncoming. For example, when type Voter is assigned -// to replicas 1 and 2, while 3 is VoterOutgoing and 4 is VoterIncoming, then +// have either type VOTER or VOTER_OUTOING, as well as that of the set of +// types VOTER and VOTER_INCOMING . For example, when type VOTER_FULL is assigned +// to replicas 1 and 2, while 3 is VOTER_OUTGOING and 4 is VOTER_INCOMING, then // the two sets over which quorums need to be achieved are {1,2,3} and {1,2,4}. // Thus, {1,2} is a quorum of both, {1,3} is a quorum of the first but not the // second, {1,4} is a quorum of the second but not the first, and {3,4} is a // quorum of neither. enum ReplicaType { - // ReplicaType_VoterFull indicates a replica that is a voter both in the + option (gogoproto.goproto_enum_prefix) = false; + + // VOTER_FULL indicates a replica that is a voter both in the // incoming and outgoing set. - VoterFull = 0; - // ReplicaType_VoterIncoming indicates a voting replica that will be a - // VoterFull once the ongoing atomic replication change is finalized; that is, + VOTER_FULL = 0; + // VOTER_INCOMING indicates a voting replica that will be a + // VOTER_FULL once the ongoing atomic replication change is finalized; that is, // it is in the process of being added. In practice, this replica type should - // be treated like a VoterFull. - VoterIncoming = 2; - // ReplicaType_VoterOutgoing indicates a voting replica that will not be part + // be treated like a VOTER_FULL. + VOTER_INCOMING = 2; + // VOTER_OUTGOING indicates a voting replica that will not be part // of the descriptor once the ongoing atomic replication change is finalized; // that is, it is in the process of being removed. In practice, a replica of // this type should be treated accordingly and no work should be assigned to // it. - VoterOutgoing = 3; - // ReplicaType_Learner indicates a replica that applies committed entries, but + VOTER_OUTGOING = 3; + // LEARNER indicates a replica that applies committed entries, but // does not count towards the quorum(s). Learners do not vote for leadership // nor do their acknowledged log entries get taken into account for // determining the committed index. At the time of writing, learners in // CockroachDB are a short-term transient state: a replica being added and on - // its way to being a VOTER. - Learner = 1; + // its way to being a VOTER_FULL. + LEARNER = 1; } // ReplicaDescriptor describes a replica location by node ID diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index 5cc0aea73c2f..7d64c9151251 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -20,31 +20,31 @@ import ( "go.etcd.io/etcd/raft/tracker" ) -// ReplicaTypeVoterFull returns a ReplicaType_VoterFull pointer suitable for use in a +// ReplicaTypeVoterFull returns a VOTER_FULL pointer suitable for use in a // nullable proto field. func ReplicaTypeVoterFull() *ReplicaType { - t := ReplicaType_VoterFull + t := VOTER_FULL return &t } -// ReplicaTypeVoterIncoming returns a ReplicaType_VoterIncoming pointer suitable +// ReplicaTypeVoterIncoming returns a VOTER_INCOMING pointer suitable // for use in a nullable proto field. func ReplicaTypeVoterIncoming() *ReplicaType { - t := ReplicaType_VoterIncoming + t := VOTER_INCOMING return &t } -// ReplicaTypeVoterOutgoing returns a ReplicaType_VoterOutgoing pointer suitable +// ReplicaTypeVoterOutgoing returns a VOTER_OUTGOING pointer suitable // for use in a nullable proto field. func ReplicaTypeVoterOutgoing() *ReplicaType { - t := ReplicaType_VoterOutgoing + t := VOTER_OUTGOING return &t } -// ReplicaTypeLearner returns a ReplicaType_Learner pointer suitable for use in +// ReplicaTypeLearner returns a LEARNER pointer suitable for use in // a nullable proto field. func ReplicaTypeLearner() *ReplicaType { - t := ReplicaType_Learner + t := LEARNER return &t } @@ -106,7 +106,7 @@ func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { // save the alloc. fastpath := true for i := range d.wrapped { - if d.wrapped[i].GetType() != ReplicaType_VoterFull { + if d.wrapped[i].GetType() != VOTER_FULL { fastpath = false break } @@ -117,7 +117,7 @@ func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { voters := make([]ReplicaDescriptor, 0, len(d.wrapped)) for i := range d.wrapped { switch d.wrapped[i].GetType() { - case ReplicaType_VoterFull, ReplicaType_VoterIncoming: + case VOTER_FULL, VOTER_INCOMING: voters = append(voters, d.wrapped[i]) } } @@ -215,7 +215,7 @@ func (d ReplicaDescriptors) Learners() []ReplicaDescriptor { // save the alloc. var learners []ReplicaDescriptor for i := range d.wrapped { - if d.wrapped[i].GetType() == ReplicaType_Learner { + if d.wrapped[i].GetType() == LEARNER { if learners == nil { learners = make([]ReplicaDescriptor, 0, len(d.wrapped)-i) } @@ -273,12 +273,12 @@ func (d *ReplicaDescriptors) RemoveReplica( func (d ReplicaDescriptors) InAtomicReplicationChange() bool { for _, rDesc := range d.wrapped { switch rDesc.GetType() { - case ReplicaType_VoterFull: - case ReplicaType_VoterIncoming: + case VOTER_FULL: + case VOTER_INCOMING: return true - case ReplicaType_VoterOutgoing: + case VOTER_OUTGOING: return true - case ReplicaType_Learner: + case LEARNER: default: panic(fmt.Sprintf("unknown replica type %d", rDesc.GetType())) } @@ -290,25 +290,26 @@ func (d ReplicaDescriptors) InAtomicReplicationChange() bool { func (d ReplicaDescriptors) ConfState() raftpb.ConfState { var cs raftpb.ConfState joint := d.InAtomicReplicationChange() - // The incoming config is taken verbatim from the full voters when the config is not - // joint. If it is joint, slot the voters into the right category. - // We never need to populate LearnersNext because this would correspond to - // demoting a voter, which we don't do. If we wanted to add that, we'd add - // ReplicaType_VoterDemoting and populate both VotersOutgoing and LearnersNext from it. + // The incoming config is taken verbatim from the full voters when the + // config is not joint. If it is joint, slot the voters into the right + // category. We never need to populate LearnersNext because this would + // correspond to demoting a voter, which we don't do. If we wanted to add + // that, we'd add a replica type VOTER_DEMOTING and populate both + // VotersOutgoing and LearnersNext from it. for _, rep := range d.wrapped { id := uint64(rep.ReplicaID) typ := rep.GetType() switch typ { - case ReplicaType_VoterFull: + case VOTER_FULL: cs.Voters = append(cs.Voters, id) if joint { cs.VotersOutgoing = append(cs.VotersOutgoing, id) } - case ReplicaType_VoterIncoming: + case VOTER_INCOMING: cs.Voters = append(cs.Voters, id) - case ReplicaType_VoterOutgoing: + case VOTER_OUTGOING: cs.VotersOutgoing = append(cs.VotersOutgoing, id) - case ReplicaType_Learner: + case LEARNER: cs.Learners = append(cs.Learners, id) default: panic(fmt.Sprintf("unknown ReplicaType %d", typ)) @@ -324,7 +325,7 @@ func (d ReplicaDescriptors) CanMakeProgress(liveFunc func(descriptor ReplicaDesc voters := d.Voters() var c int // Take the fast path when there are only "current and future" voters, i.e. - // no learners and no voters of type VoterOutgoing. The config may be joint, + // no learners and no voters of type VOTER_OUTGOING. The config may be joint, // but the outgoing conf is subsumed by the incoming one. if n := len(d.wrapped); len(voters) == n { for _, rDesc := range voters { diff --git a/pkg/roachpb/metadata_replicas_test.go b/pkg/roachpb/metadata_replicas_test.go index 3249ead10025..61d66ac22f16 100644 --- a/pkg/roachpb/metadata_replicas_test.go +++ b/pkg/roachpb/metadata_replicas_test.go @@ -56,7 +56,7 @@ func TestVotersLearnersAll(t *testing.T) { for _, voter := range r.Voters() { typ := voter.GetType() switch typ { - case ReplicaType_VoterFull, ReplicaType_VoterIncoming: + case VOTER_FULL, VOTER_INCOMING: seen[voter] = struct{}{} default: assert.FailNow(t, "unexpectedly got a %s as Voter()", typ) @@ -64,18 +64,18 @@ func TestVotersLearnersAll(t *testing.T) { } for _, learner := range r.Learners() { seen[learner] = struct{}{} - assert.Equal(t, ReplicaType_Learner, learner.GetType()) + assert.Equal(t, LEARNER, learner.GetType()) } all := r.All() - // Make sure that VoterOutgoing is the only type that is skipped both + // Make sure that VOTER_OUTGOING is the only type that is skipped both // by Learners() and Voters() for _, rd := range all { typ := rd.GetType() if _, seen := seen[rd]; !seen { - assert.Equal(t, ReplicaType_VoterOutgoing, typ) + assert.Equal(t, VOTER_OUTGOING, typ) } else { - assert.NotEqual(t, ReplicaType_VoterOutgoing, typ) + assert.NotEqual(t, VOTER_OUTGOING, typ) } } assert.Equal(t, len(test), len(all)) @@ -128,10 +128,10 @@ func TestReplicaDescriptorsRemove(t *testing.T) { assert.Equal(t, lenBefore, len(r.All()), "testcase %d", i) } for _, voter := range r.Voters() { - assert.Equal(t, ReplicaType_VoterFull, voter.GetType(), "testcase %d", i) + assert.Equal(t, VOTER_FULL, voter.GetType(), "testcase %d", i) } for _, learner := range r.Learners() { - assert.Equal(t, ReplicaType_Learner, learner.GetType(), "testcase %d", i) + assert.Equal(t, LEARNER, learner.GetType(), "testcase %d", i) } } } diff --git a/pkg/storage/allocator_test.go b/pkg/storage/allocator_test.go index 388b63b08e5f..5b78ba103ffe 100644 --- a/pkg/storage/allocator_test.go +++ b/pkg/storage/allocator_test.go @@ -4784,7 +4784,7 @@ func TestAllocatorRemoveLearner(t *testing.T) { zone := config.ZoneConfig{ NumReplicas: proto.Int32(3), } - learnerType := roachpb.ReplicaType_Learner + learnerType := roachpb.LEARNER rangeWithLearnerDesc := roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { diff --git a/pkg/storage/batcheval/cmd_lease.go b/pkg/storage/batcheval/cmd_lease.go index 23b6d8a5bfcc..ba597922fda5 100644 --- a/pkg/storage/batcheval/cmd_lease.go +++ b/pkg/storage/batcheval/cmd_lease.go @@ -47,7 +47,7 @@ func checkCanReceiveLease(rec EvalContext) error { if !ok { return errors.AssertionFailedf( `could not find replica for store %s in %s`, rec.StoreID(), rec.Desc()) - } else if t := repDesc.GetType(); t != roachpb.ReplicaType_VoterFull { + } else if t := repDesc.GetType(); t != roachpb.VOTER_FULL { return errors.Errorf(`cannot transfer lease to replica of type %s`, t) } return nil diff --git a/pkg/storage/batcheval/cmd_lease_test.go b/pkg/storage/batcheval/cmd_lease_test.go index f6480f148a5e..a223c7f70cb9 100644 --- a/pkg/storage/batcheval/cmd_lease_test.go +++ b/pkg/storage/batcheval/cmd_lease_test.go @@ -129,9 +129,9 @@ func TestLeaseCommandLearnerReplica(t *testing.T) { // Learners are not allowed to become leaseholders for now, see the comments // in TransferLease and RequestLease. _, err := TransferLease(ctx, nil, cArgs, nil) - require.EqualError(t, err, `cannot transfer lease to replica of type Learner`) + require.EqualError(t, err, `cannot transfer lease to replica of type LEARNER`) cArgs.Args = &roachpb.RequestLeaseRequest{} _, err = RequestLease(ctx, nil, cArgs, nil) - require.EqualError(t, err, `cannot transfer lease to replica of type Learner`) + require.EqualError(t, err, `cannot transfer lease to replica of type LEARNER`) } diff --git a/pkg/storage/client_test.go b/pkg/storage/client_test.go index 158eb62a1893..7526de257297 100644 --- a/pkg/storage/client_test.go +++ b/pkg/storage/client_test.go @@ -1211,7 +1211,7 @@ func (m *multiTestContext) replicateRangeNonFatal(rangeID roachpb.RangeID, dests if e := expectedReplicaIDs[i]; repDesc.ReplicaID != e { return errors.Errorf("expected replica %s to have ID %d", repl, e) } - if t := repDesc.GetType(); t != roachpb.ReplicaType_VoterFull { + if t := repDesc.GetType(); t != roachpb.VOTER_FULL { return errors.Errorf("expected replica %s to be a voter was %s", repl, t) } if !repl.Desc().ContainsKey(startKey) { diff --git a/pkg/storage/merge_queue.go b/pkg/storage/merge_queue.go index 84baa46f2d6d..08557107cf2b 100644 --- a/pkg/storage/merge_queue.go +++ b/pkg/storage/merge_queue.go @@ -295,12 +295,12 @@ func (mq *mergeQueue) process( // Defensive sanity check that everything is now a voter. for i := range lhsReplicas { - if lhsReplicas[i].GetType() != roachpb.ReplicaType_VoterFull { + if lhsReplicas[i].GetType() != roachpb.VOTER_FULL { return errors.Errorf(`cannot merge non-voter replicas on lhs: %v`, lhsReplicas) } } for i := range rhsReplicas { - if rhsReplicas[i].GetType() != roachpb.ReplicaType_VoterFull { + if rhsReplicas[i].GetType() != roachpb.VOTER_FULL { return errors.Errorf(`cannot merge non-voter replicas on rhs: %v`, rhsReplicas) } } diff --git a/pkg/storage/raft_snapshot_queue.go b/pkg/storage/raft_snapshot_queue.go index 79d9b6facef7..0146493054bd 100644 --- a/pkg/storage/raft_snapshot_queue.go +++ b/pkg/storage/raft_snapshot_queue.go @@ -111,7 +111,7 @@ func (rq *raftSnapshotQueue) processRaftSnapshot( // that's adding it or it's been orphaned and it's about to be cleaned up by // the replicate queue. Either way, no point in also sending it a snapshot of // type RAFT. - if repDesc.GetType() == roachpb.ReplicaType_Learner { + if repDesc.GetType() == roachpb.LEARNER { if index := repl.getAndGCSnapshotLogTruncationConstraints(timeutil.Now()); index > 0 { // There is a snapshot being transferred. It's probably a LEARNER snap, so // bail for now and try again later. diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index be4cc79645b5..77071b82c3ad 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -859,7 +859,7 @@ func IsSnapshotError(err error) bool { // replica sets is required for decision making. Transitioning into this joint // configuration, the RangeDescriptor (which is the source of truth of // the replication configuration) is updated with corresponding replicas of -// type VoterIncoming and VoterOutgoing. +// type VOTER_INCOMING and VOTER_OUTGOING. // Immediately after committing this change, a second transition updates the // descriptor with and activates the final configuration. // @@ -1043,7 +1043,7 @@ func validateReplicationChanges( // trying to add it with the learner+snapshot+voter cycle and got // interrupted or else we hit a race between the replicate queue and // AdminChangeReplicas. - if rDesc.GetType() == roachpb.ReplicaType_Learner { + if rDesc.GetType() == roachpb.LEARNER { return errors.Errorf( "unable to add replica %v which is already present as a learner in %s", chg.Target, desc) } @@ -1125,7 +1125,7 @@ func (r *Replica) atomicReplicationChange( return nil, errors.Errorf("programming error: replica %v not found in %v", target, desc) } - if rDesc.GetType() != roachpb.ReplicaType_Learner { + if rDesc.GetType() != roachpb.LEARNER { return nil, errors.Errorf("programming error: cannot promote replica of type %s", rDesc.Type) } @@ -1184,7 +1184,7 @@ func (r *Replica) tryRollBackLearnerReplica( details string, ) { repDesc, ok := desc.GetReplicaDescriptor(target.StoreID) - if !ok || repDesc.GetType() != roachpb.ReplicaType_Learner { + if !ok || repDesc.GetType() != roachpb.LEARNER { // There's no learner to roll back. log.Event(ctx, "learner to roll back not found; skipping") return @@ -1324,14 +1324,14 @@ func execChangeReplicasTxn( case internalChangeTypeAddVoterViaPreemptiveSnap: // Legacy code. added = append(added, - updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_VoterFull)) + updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.VOTER_FULL)) case internalChangeTypeAddLearner: added = append(added, - updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_Learner)) + updatedDesc.AddReplica(chg.target.NodeID, chg.target.StoreID, roachpb.LEARNER)) case internalChangeTypePromoteLearner: - typ := roachpb.ReplicaType_VoterFull + typ := roachpb.VOTER_FULL if useJoint { - typ = roachpb.ReplicaType_VoterIncoming + typ = roachpb.VOTER_INCOMING } rDesc, ok := updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, typ) if !ok { @@ -1344,7 +1344,7 @@ func execChangeReplicasTxn( if !useJoint { rDesc, ok = updatedDesc.RemoveReplica(chg.target.NodeID, chg.target.StoreID) } else { - rDesc, ok = updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, roachpb.ReplicaType_VoterOutgoing) + rDesc, ok = updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, roachpb.VOTER_OUTGOING) } if !ok { return nil, errors.Errorf("cannot remove nonexistent target %v", chg.target) @@ -1362,10 +1362,10 @@ func execChangeReplicasTxn( // call RemoveReplica below. for _, rDesc := range updatedDesc.Replicas().DeepCopy().All() { switch rDesc.GetType() { - case roachpb.ReplicaType_VoterIncoming: - updatedDesc.SetReplicaType(rDesc.NodeID, rDesc.StoreID, roachpb.ReplicaType_VoterFull) + case roachpb.VOTER_INCOMING: + updatedDesc.SetReplicaType(rDesc.NodeID, rDesc.StoreID, roachpb.VOTER_FULL) isJoint = true - case roachpb.ReplicaType_VoterOutgoing: + case roachpb.VOTER_OUTGOING: updatedDesc.RemoveReplica(rDesc.NodeID, rDesc.StoreID) isJoint = true default: diff --git a/pkg/storage/replica_follower_read.go b/pkg/storage/replica_follower_read.go index 650952e0d247..807c85a383b0 100644 --- a/pkg/storage/replica_follower_read.go +++ b/pkg/storage/replica_follower_read.go @@ -45,7 +45,7 @@ func (r *Replica) canServeFollowerRead( if err != nil { return roachpb.NewError(err) } - if repDesc.GetType() == roachpb.ReplicaType_Learner { + if repDesc.GetType() == roachpb.LEARNER { log.Event(ctx, "learner replicas cannot serve follower reads") return pErr } diff --git a/pkg/storage/replica_learner_test.go b/pkg/storage/replica_learner_test.go index d3b5d4eea588..7a19a5bb97c5 100644 --- a/pkg/storage/replica_learner_test.go +++ b/pkg/storage/replica_learner_test.go @@ -592,8 +592,8 @@ func TestLearnerNoAcceptLease(t *testing.T) { desc := tc.LookupRangeOrFatal(t, scratchStartKey) err := tc.TransferRangeLease(desc, tc.Target(1)) - if !testutils.IsError(err, `cannot transfer lease to replica of type Learner`) { - t.Fatalf(`expected "cannot transfer lease to replica of type Learner" error got: %+v`, err) + if !testutils.IsError(err, `cannot transfer lease to replica of type LEARNER`) { + t.Fatalf(`expected "cannot transfer lease to replica of type LEARNER" error got: %+v`, err) } } diff --git a/pkg/storage/replica_range_lease.go b/pkg/storage/replica_range_lease.go index d662e51ec53e..4edd35d1df01 100644 --- a/pkg/storage/replica_range_lease.go +++ b/pkg/storage/replica_range_lease.go @@ -683,7 +683,7 @@ func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID // `r.mu.minLeaseProposedTS = status.Timestamp` line below will likely play // badly with that. This would be an issue even without learners, but // omitting this check would make it worse. Fixme. - if t := nextLeaseHolder.GetType(); t != roachpb.ReplicaType_VoterFull { + if t := nextLeaseHolder.GetType(); t != roachpb.VOTER_FULL { return nil, nil, errors.Errorf(`cannot transfer lease to replica of type %s`, t) }