From 01e99d013e34510a0f7fb2a02bf98cc9258bddef Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Thu, 7 Jan 2021 18:58:20 -0500 Subject: [PATCH] kvserver: support atomic promotions and demotions of non-voting replicas This PR teaches `AdminChangeReplicas` to atomically promote voters to non-voters, demote voters to non-voters or swap voters with non-voters via joint consensus. Fixes #58499 Informs #51943 Release note: None --- pkg/kv/kvserver/allocator.go | 24 +- pkg/kv/kvserver/allocator_scorer.go | 2 +- pkg/kv/kvserver/allocator_scorer_test.go | 3 +- pkg/kv/kvserver/client_lease_test.go | 2 +- pkg/kv/kvserver/client_replica_test.go | 38 +- pkg/kv/kvserver/raft.pb.go | 32 +- pkg/kv/kvserver/raft.proto | 2 +- pkg/kv/kvserver/replica_command.go | 369 +++++++++++++----- pkg/kv/kvserver/replica_command_test.go | 225 +++++++++++ pkg/kv/kvserver/replica_learner_test.go | 30 +- pkg/kv/kvserver/replica_proposal_buf_test.go | 2 +- pkg/kv/kvserver/replica_raftstorage.go | 7 +- pkg/kv/kvserver/replica_test.go | 10 +- pkg/kv/kvserver/store_rebalancer.go | 2 +- pkg/roachpb/data.go | 33 +- pkg/roachpb/data.pb.go | 70 ++-- pkg/roachpb/data.proto | 10 +- pkg/roachpb/data_test.go | 10 +- pkg/roachpb/metadata.pb.go | 289 +++++++------- pkg/roachpb/metadata.proto | 56 +-- pkg/roachpb/metadata_replicas.go | 15 +- pkg/roachpb/metadata_replicas_test.go | 2 +- .../serverutils/test_cluster_shim.go | 15 + pkg/testutils/testcluster/testcluster.go | 42 +- 24 files changed, 908 insertions(+), 382 deletions(-) diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index abbb0232200d..66583eac387d 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -735,7 +735,7 @@ func (a Allocator) simulateRemoveTarget( func (a Allocator) removeTarget( ctx context.Context, zone *zonepb.ZoneConfig, - candidates []roachpb.ReplicaDescriptor, + candidates []roachpb.ReplicationTarget, existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, targetType targetReplicaType, @@ -815,7 +815,14 @@ func (a Allocator) RemoveVoter( existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, ) (roachpb.ReplicaDescriptor, string, error) { - return a.removeTarget(ctx, zone, voterCandidates, existingVoters, existingNonVoters, voterTarget) + return a.removeTarget( + ctx, + zone, + roachpb.MakeReplicaSet(voterCandidates).ReplicationTargets(), + existingVoters, + existingNonVoters, + voterTarget, + ) } // RemoveNonVoter returns a suitable non-voting replica to remove from the @@ -830,7 +837,14 @@ func (a Allocator) RemoveNonVoter( existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, ) (roachpb.ReplicaDescriptor, string, error) { - return a.removeTarget(ctx, zone, nonVoterCandidates, existingVoters, existingNonVoters, nonVoterTarget) + return a.removeTarget( + ctx, + zone, + roachpb.MakeReplicaSet(nonVoterCandidates).ReplicationTargets(), + existingVoters, + existingNonVoters, + nonVoterTarget, + ) } // RebalanceTarget returns a suitable store for a rebalance target with @@ -1089,7 +1103,7 @@ func (a *Allocator) TransferLeaseTarget( // If the current leaseholder is not preferred, set checkTransferLeaseSource // to false to motivate the below logic to transfer the lease. existing = preferred - if !storeHasReplica(leaseStoreID, preferred) { + if !storeHasReplica(leaseStoreID, roachpb.MakeReplicaSet(preferred).ReplicationTargets()) { checkTransferLeaseSource = false } } @@ -1189,7 +1203,7 @@ func (a *Allocator) ShouldTransferLease( existing = preferred // If the current leaseholder isn't one of the preferred stores, then we // should try to transfer the lease. - if !storeHasReplica(leaseStoreID, existing) { + if !storeHasReplica(leaseStoreID, roachpb.MakeReplicaSet(existing).ReplicationTargets()) { return true } } diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 81f6f92ad2bf..0842aab29fb1 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -884,7 +884,7 @@ func nodeHasReplica(nodeID roachpb.NodeID, existing []roachpb.ReplicaDescriptor) // storeHasReplica returns true if the provided StoreID contains an entry in // the provided list of existing replicas. -func storeHasReplica(storeID roachpb.StoreID, existing []roachpb.ReplicaDescriptor) bool { +func storeHasReplica(storeID roachpb.StoreID, existing []roachpb.ReplicationTarget) bool { for _, r := range existing { if r.StoreID == storeID { return true diff --git a/pkg/kv/kvserver/allocator_scorer_test.go b/pkg/kv/kvserver/allocator_scorer_test.go index 80e90100676f..fe4c214b95f5 100644 --- a/pkg/kv/kvserver/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator_scorer_test.go @@ -418,7 +418,8 @@ func TestStoreHasReplica(t *testing.T) { existing = append(existing, roachpb.ReplicaDescriptor{StoreID: roachpb.StoreID(i)}) } for i := 1; i < 10; i++ { - if e, a := i%2 == 0, storeHasReplica(roachpb.StoreID(i), existing); e != a { + if e, a := i%2 == 0, + storeHasReplica(roachpb.StoreID(i), roachpb.MakeReplicaSet(existing).ReplicationTargets()); e != a { t.Errorf("StoreID %d expected to be %t, got %t", i, e, a) } } diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index 40fa64c6df9c..f4aba4633955 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -350,7 +350,7 @@ func TestCannotTransferLeaseToVoterOutgoing(t *testing.T) { // We have a sleep loop below to try to encourage the lease transfer // to make it past that sanity check prior to letting the change // of replicas proceed. - "cannot transfer lease to replica of type VOTER_DEMOTING", err.Error()) + "cannot transfer lease to replica of type VOTER_DEMOTING_LEARNER", err.Error()) }() // Try really hard to make sure that our request makes it past the // sanity check error to the evaluation error. diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index d453d9fbee53..c96e3a9a5331 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -2272,8 +2272,7 @@ func TestRandomConcurrentAdminChangeReplicasRequests(t *testing.T) { if rand.Intn(2) == 0 { op = roachpb.ADD_NON_VOTER } - _, err := db.AdminChangeReplicas( - ctx, key, rangeInfo.Desc, roachpb.MakeReplicationChanges(op, pickTargets()...)) + _, err := db.AdminChangeReplicas(ctx, key, rangeInfo.Desc, roachpb.MakeReplicationChanges(op, pickTargets()...)) return err } wg.Add(actors) @@ -2284,7 +2283,7 @@ func TestRandomConcurrentAdminChangeReplicasRequests(t *testing.T) { var gotSuccess bool for _, err := range errors { if err != nil { - assert.True(t, kvserver.IsRetriableReplicationChangeError(err), err) + require.Truef(t, kvserver.IsRetriableReplicationChangeError(err), "%s; desc: %v", err, rangeInfo.Desc) } else if gotSuccess { t.Error("expected only one success") } else { @@ -2293,6 +2292,39 @@ func TestRandomConcurrentAdminChangeReplicasRequests(t *testing.T) { } } +func TestChangeReplicasSwapVoterWithNonVoter(t *testing.T) { + defer leaktest.AfterTest(t)() + skip.UnderRace(t) + + const numNodes = 7 + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + }) + ctx := context.Background() + defer tc.Stopper().Stop(ctx) + + key := tc.ScratchRange(t) + // NB: The test cluster starts with firstVoter having a voting replica (and + // the lease) for all ranges. + firstVoter, secondVoter, nonVoter := tc.Target(0), tc.Target(1), tc.Target(3) + firstStore, err := tc.Server(0).GetStores().(*kvserver.Stores).GetStore(tc.Server(0).GetFirstStoreID()) + require.NoError(t, err) + firstRepl := firstStore.LookupReplica(roachpb.RKey(key)) + require.NotNil(t, firstRepl, `the first node in the TestCluster must have a replica for the ScratchRange`) + + // TODO(aayush): Trying to swap the last voting replica with a non-voter hits + // the safeguard inside Replica.propose() as the last voting replica is always + // the leaseholder. There are a bunch of subtleties around getting a + // leaseholder to remove itself without another voter to immediately transfer + // the lease to. Determine if/how this needs to be fixed. + tc.AddNonVotersOrFatal(t, key, nonVoter) + _, err = tc.SwapVoterWithNonVoter(key, firstVoter, nonVoter) + require.Regexp(t, "received invalid ChangeReplicasTrigger", err) + + tc.AddVotersOrFatal(t, key, secondVoter) + tc.SwapVoterWithNonVoterOrFatal(t, key, secondVoter, nonVoter) +} + // TestReplicaTombstone ensures that tombstones are written when we expect // them to be. Tombstones are laid down when replicas are removed. // Replicas are removed for several reasons: diff --git a/pkg/kv/kvserver/raft.pb.go b/pkg/kv/kvserver/raft.pb.go index 410d3bf903c1..bf1cb8d484bc 100644 --- a/pkg/kv/kvserver/raft.pb.go +++ b/pkg/kv/kvserver/raft.pb.go @@ -53,7 +53,7 @@ func (x SnapshotRequest_Priority) String() string { return proto.EnumName(SnapshotRequest_Priority_name, int32(x)) } func (SnapshotRequest_Priority) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{5, 0} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{5, 0} } type SnapshotRequest_Strategy int32 @@ -77,7 +77,7 @@ func (x SnapshotRequest_Strategy) String() string { return proto.EnumName(SnapshotRequest_Strategy_name, int32(x)) } func (SnapshotRequest_Strategy) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{5, 1} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{5, 1} } // Type is used for metrics collection on the receiver side. See @@ -92,7 +92,7 @@ const ( // for upreplication, before they're promoted to full voters. // // As of the time of writing, we only send this snapshot from the - // atomicReplicationChange after creating a new LEARNER replica. + // execReplicationChangesForVoters after creating a new LEARNER replica. SnapshotRequest_LEARNER_INITIAL SnapshotRequest_Type = 1 ) @@ -109,7 +109,7 @@ func (x SnapshotRequest_Type) String() string { return proto.EnumName(SnapshotRequest_Type_name, int32(x)) } func (SnapshotRequest_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{5, 2} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{5, 2} } type SnapshotResponse_Status int32 @@ -141,7 +141,7 @@ func (x SnapshotResponse_Status) String() string { return proto.EnumName(SnapshotResponse_Status_name, int32(x)) } func (SnapshotResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{6, 0} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{6, 0} } // RaftHeartbeat is a request that contains the barebones information for a @@ -163,7 +163,7 @@ func (m *RaftHeartbeat) Reset() { *m = RaftHeartbeat{} } func (m *RaftHeartbeat) String() string { return proto.CompactTextString(m) } func (*RaftHeartbeat) ProtoMessage() {} func (*RaftHeartbeat) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{0} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{0} } func (m *RaftHeartbeat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -239,7 +239,7 @@ func (m *RaftMessageRequest) Reset() { *m = RaftMessageRequest{} } func (m *RaftMessageRequest) String() string { return proto.CompactTextString(m) } func (*RaftMessageRequest) ProtoMessage() {} func (*RaftMessageRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{1} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{1} } func (m *RaftMessageRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -272,7 +272,7 @@ func (m *RaftMessageRequestBatch) Reset() { *m = RaftMessageRequestBatch func (m *RaftMessageRequestBatch) String() string { return proto.CompactTextString(m) } func (*RaftMessageRequestBatch) ProtoMessage() {} func (*RaftMessageRequestBatch) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{2} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{2} } func (m *RaftMessageRequestBatch) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -305,7 +305,7 @@ func (m *RaftMessageResponseUnion) Reset() { *m = RaftMessageResponseUni func (m *RaftMessageResponseUnion) String() string { return proto.CompactTextString(m) } func (*RaftMessageResponseUnion) ProtoMessage() {} func (*RaftMessageResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{3} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{3} } func (m *RaftMessageResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -348,7 +348,7 @@ func (m *RaftMessageResponse) Reset() { *m = RaftMessageResponse{} } func (m *RaftMessageResponse) String() string { return proto.CompactTextString(m) } func (*RaftMessageResponse) ProtoMessage() {} func (*RaftMessageResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{4} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{4} } func (m *RaftMessageResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -389,7 +389,7 @@ func (m *SnapshotRequest) Reset() { *m = SnapshotRequest{} } func (m *SnapshotRequest) String() string { return proto.CompactTextString(m) } func (*SnapshotRequest) ProtoMessage() {} func (*SnapshotRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{5} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{5} } func (m *SnapshotRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -451,7 +451,7 @@ func (m *SnapshotRequest_Header) Reset() { *m = SnapshotRequest_Header{} func (m *SnapshotRequest_Header) String() string { return proto.CompactTextString(m) } func (*SnapshotRequest_Header) ProtoMessage() {} func (*SnapshotRequest_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{5, 0} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{5, 0} } func (m *SnapshotRequest_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -485,7 +485,7 @@ func (m *SnapshotResponse) Reset() { *m = SnapshotResponse{} } func (m *SnapshotResponse) String() string { return proto.CompactTextString(m) } func (*SnapshotResponse) ProtoMessage() {} func (*SnapshotResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{6} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{6} } func (m *SnapshotResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -522,7 +522,7 @@ func (m *ConfChangeContext) Reset() { *m = ConfChangeContext{} } func (m *ConfChangeContext) String() string { return proto.CompactTextString(m) } func (*ConfChangeContext) ProtoMessage() {} func (*ConfChangeContext) Descriptor() ([]byte, []int) { - return fileDescriptor_raft_031413a263a2674e, []int{7} + return fileDescriptor_raft_6d6ea5455d4829c1, []int{7} } func (m *ConfChangeContext) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2856,9 +2856,9 @@ var ( ErrIntOverflowRaft = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("kv/kvserver/raft.proto", fileDescriptor_raft_031413a263a2674e) } +func init() { proto.RegisterFile("kv/kvserver/raft.proto", fileDescriptor_raft_6d6ea5455d4829c1) } -var fileDescriptor_raft_031413a263a2674e = []byte{ +var fileDescriptor_raft_6d6ea5455d4829c1 = []byte{ // 1296 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x57, 0x4f, 0x73, 0xdb, 0x44, 0x14, 0xb7, 0x12, 0xc5, 0x96, 0xd7, 0x71, 0x23, 0xb6, 0x21, 0x88, 0x4c, 0xb1, 0x83, 0xa6, 0x30, diff --git a/pkg/kv/kvserver/raft.proto b/pkg/kv/kvserver/raft.proto index 69891df1a96c..0abb9a1477a3 100644 --- a/pkg/kv/kvserver/raft.proto +++ b/pkg/kv/kvserver/raft.proto @@ -147,7 +147,7 @@ message SnapshotRequest { // for upreplication, before they're promoted to full voters. // // As of the time of writing, we only send this snapshot from the - // atomicReplicationChange after creating a new LEARNER replica. + // execReplicationChangesForVoters after creating a new LEARNER replica. LEARNER_INITIAL = 1; reserved 2; } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index be67327eba17..331887c9ecf4 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -914,7 +914,7 @@ func waitForReplicasInit( // 1. s1/1 s2/2 s3/3 (VOTER_FULL is implied) // 2. s1/1 s2/2 s3/3 s4/4LEARNER // 3. s1/1 s2/2 s3/3 s4/4LEARNER s5/5LEARNER -// 4. s1/1VOTER_DEMOTING s2/2VOTER_DEMOTING s3/3 s4/4VOTER_INCOMING s5/5VOTER_INCOMING +// 4. s1/1VOTER_DEMOTING_LEARNER s2/2VOTER_DEMOTING_LEARNER s3/3 s4/4VOTER_INCOMING s5/5VOTER_INCOMING // 5. s1/1LEARNER s2/2LEARNER s3/3 s4/4 s5/5 // 6. s2/2LEARNER s3/3 s4/4 s5/5 // 7. s3/3 s4/4 s5/5 @@ -998,41 +998,53 @@ func (r *Replica) changeReplicasImpl( if err := validateReplicationChanges(desc, chgs); err != nil { return nil, err } + targets := synthesizeTargetsByChangeType(chgs) - if adds := chgs.NonVoterAdditions(); len(adds) > 0 { - desc, err = addRaftLearners(ctx, r.store, desc, reason, details, adds, internalChangeTypeAddNonVoter) + // NB: As of the time of this writing,`AdminRelocateRange` will only execute + // replication changes one by one. Thus, the order in which we execute the + // changes we've synthesized doesn't matter that much. However, this + // limitation might be lifted soon (see #58752, for instance). + // + // We choose to execute changes in the following order: + // 1. Promotions / demotions / swaps between voters and non-voters + // 2. Voter additions + // 3. Voter removals + // 4. Non-voter additions + // 5. Non-voter removals + // + // This order is meant to be symmetric with how the allocator prioritizes + // these actions. Broadly speaking, we first want to add a missing voter (and + // promoting an existing non-voter, or swapping with one, is the fastest way + // to do that). Then, we consider rebalancing/removing voters. Finally, we + // handle non-voter additions & removals. + + // We perform promotions of non-voting replicas to voting replicas, and + // likewise, demotions of voting replicas to non-voting replicas. If both + // these types of operations are being applied to a (voter, non-voter) pair, + // it will execute an atomic swap using joint consensus. + // + // NB: Promotions and demotions of LEARNER replicas are handled implicitly + // during the addition or removal of voting replicas and are not handled + // here. + swaps := getInternalChangesForExplicitPromotionsAndDemotions(targets.voterDemotions, targets.nonVoterPromotions) + if len(swaps) > 0 { + desc, err = execChangeReplicasTxn(ctx, desc, reason, details, swaps, changeReplicasTxnArgs{ + db: r.store.DB(), + liveAndDeadReplicas: r.store.allocator.storePool.liveAndDeadReplicas, + logChange: r.store.logChange, + testForceJointConfig: r.store.TestingKnobs().ReplicationAlwaysUseJointConfig, + testAllowDangerousReplicationChanges: r.store.TestingKnobs().AllowDangerousReplicationChanges, + }) if err != nil { return nil, err } - // Queue the replica up into the raft snapshot queue so that the non-voters - // that were added receive their first snapshot relatively soon. See the - // comment block above ReplicaSet.NonVoters() for why we do this. - r.store.raftSnapshotQueue.AddAsync(ctx, r, raftSnapshotPriority) } - if removals := chgs.NonVoterRemovals(); len(removals) > 0 { - for _, rem := range removals { - iChgs := []internalReplicationChange{{target: rem, typ: internalChangeTypeRemove}} - var err error - desc, err = execChangeReplicasTxn(ctx, desc, reason, details, iChgs, - changeReplicasTxnArgs{ - db: r.store.DB(), - liveAndDeadReplicas: r.store.allocator.storePool.liveAndDeadReplicas, - logChange: r.store.logChange, - testForceJointConfig: r.store.TestingKnobs().ReplicationAlwaysUseJointConfig, - testAllowDangerousReplicationChanges: r.store.TestingKnobs().AllowDangerousReplicationChanges, - }) - if err != nil { - return nil, err - } - } - } - - if adds := chgs.VoterAdditions(); len(adds) > 0 { + if adds := targets.voterAdditions; len(adds) > 0 { // Lock learner snapshots even before we run the ConfChange txn to add them // to prevent a race with the raft snapshot queue trying to send it first. // Note that this lock needs to cover sending the snapshots which happens in - _ = r.atomicReplicationChange + _ = r.execReplicationChangesForVoters // which also has some more details on what's going on here. // // Also note that the lock only prevents the raft snapshot queue from @@ -1056,7 +1068,8 @@ 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.atomicReplicationChange(ctx, desc, priority, reason, details, chgs) + desc, err = r.execReplicationChangesForVoters(ctx, desc, priority, reason, details, + targets.voterAdditions, targets.voterRemovals) 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. @@ -1068,17 +1081,79 @@ func (r *Replica) changeReplicasImpl( } // Don't leave a learner replica lying around if we didn't succeed in // promoting it to a voter. - if targets := chgs.VoterAdditions(); len(targets) > 0 { - log.Infof(ctx, "could not promote %v to voter, rolling back: %v", targets, err) - for _, target := range targets { + if adds := targets.voterAdditions; len(adds) > 0 { + log.Infof(ctx, "could not promote %v to voter, rolling back: %v", adds, err) + for _, target := range adds { r.tryRollBackLearnerReplica(ctx, r.Desc(), target, reason, details) } } return nil, err } + + if adds := targets.nonVoterAdditions; len(adds) > 0 { + desc, err = addRaftLearners(ctx, r.store, desc, reason, details, adds, internalChangeTypeAddNonVoter) + if err != nil { + return nil, err + } + // Queue the replica up into the raft snapshot queue so that the non-voters + // that were added receive their first snapshot relatively soon. See the + // comment block above ReplicaSet.NonVoters() for why we do this. + r.store.raftSnapshotQueue.AddAsync(ctx, r, raftSnapshotPriority) + } + + if removals := targets.nonVoterRemovals; len(removals) > 0 { + for _, rem := range removals { + iChgs := []internalReplicationChange{{target: rem, typ: internalChangeTypeRemove}} + var err error + desc, err = execChangeReplicasTxn(ctx, desc, reason, details, iChgs, + changeReplicasTxnArgs{ + db: r.store.DB(), + liveAndDeadReplicas: r.store.allocator.storePool.liveAndDeadReplicas, + logChange: r.store.logChange, + testForceJointConfig: r.store.TestingKnobs().ReplicationAlwaysUseJointConfig, + testAllowDangerousReplicationChanges: r.store.TestingKnobs().AllowDangerousReplicationChanges, + }) + if err != nil { + return nil, err + } + } + } return desc, err } +type targetsForReplicationChanges struct { + voterDemotions, nonVoterPromotions []roachpb.ReplicationTarget + voterAdditions, voterRemovals []roachpb.ReplicationTarget + nonVoterAdditions, nonVoterRemovals []roachpb.ReplicationTarget +} + +// synthesizeTargetsByChangeType groups replication changes in the +// manner they are intended to be executed by AdminChangeReplicas. +// +// In particular, it coalesces ReplicationChanges of types ADD_VOTER and +// REMOVE_NON_VOTER on a given target as promotions of non-voters into voters +// and likewise, ADD_NON_VOTER and REMOVE_VOTER changes for a given target as +// demotions of voters into non-voters. The rest of the changes are handled +// distinctly and are thus segregated in the return result. +func synthesizeTargetsByChangeType( + chgs roachpb.ReplicationChanges, +) (result targetsForReplicationChanges) { + // Isolate the promotions to voters and the demotions to non-voters from the + // rest of the changes, since we want to handle these together and execute + // atomic swaps of voters <-> non-voters if possible. + result.voterDemotions = intersectTargets(chgs.VoterRemovals(), chgs.NonVoterAdditions()) + result.nonVoterPromotions = intersectTargets(chgs.NonVoterRemovals(), chgs.VoterAdditions()) + + // Synthesize the additions and removals that shouldn't get executed as + // promotions of non-voters or demotions of voters. + result.voterAdditions = subtractTargets(chgs.VoterAdditions(), chgs.NonVoterRemovals()) + result.voterRemovals = subtractTargets(chgs.VoterRemovals(), chgs.NonVoterAdditions()) + result.nonVoterAdditions = subtractTargets(chgs.NonVoterAdditions(), chgs.VoterRemovals()) + result.nonVoterRemovals = subtractTargets(chgs.NonVoterRemovals(), chgs.VoterAdditions()) + + return result +} + // 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. The @@ -1203,10 +1278,22 @@ func validateReplicationChanges( } } } - if _, ok := byStoreID[chg.Target.StoreID]; ok { - return fmt.Errorf("changes %+v refer to n%d and s%d twice", chgs, - chg.Target.NodeID, chg.Target.StoreID) + if prevChg, ok := byStoreID[chg.Target.StoreID]; ok { + isVoterDemotion := prevChg.ChangeType == roachpb.ADD_NON_VOTER && + chg.ChangeType == roachpb.REMOVE_VOTER + isNonVoterPromotion := prevChg.ChangeType == roachpb.ADD_VOTER && + chg.ChangeType == roachpb.REMOVE_NON_VOTER + + if !isNonVoterPromotion && !isVoterDemotion { + return fmt.Errorf("changes %+v refer to n%d and s%d twice", chgs, + chg.Target.NodeID, chg.Target.StoreID) + } } + // NB: In case we have an add-remove on the same store, the line below makes + // it such that the `ChangeType` in this map is the one that corresponds to + // the removal. This is subtle but important since the loop below ignores + // removals, and will consequently ignore add-remove pairs as well (which is + // intentional). byStoreID[chg.Target.StoreID] = chg } @@ -1254,26 +1341,30 @@ func validateReplicationChanges( continue } // Looks like we found a replica with the same store and node id. If the - // replica is already a learner, then either some previous leaseholder was - // 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.LEARNER { + // replica is a learner, then one of the following is true: + // 1. some previous leaseholder was trying to add it with the + // learner+snapshot+voter cycle and got interrupted. + // 2. we hit a race between the replicate queue and AdminChangeReplicas. + // 3. We're trying to swap a voting replica with a non-voting replica + // before the voting replica has been upreplicated and switched from + // LEARNER to VOTER_FULL. + switch t := rDesc.GetType(); t { + case roachpb.LEARNER: return errors.Mark(errors.Errorf( "unable to add replica %v which is already present as a learner in %s", chg.Target, desc), errMarkInvalidReplicationChange) - } - if rDesc.GetType() == roachpb.NON_VOTER { + case roachpb.NON_VOTER: return errors.Mark(errors.Errorf( "unable to add replica %v which is already present as a non-voter in %s", chg.Target, desc), errMarkInvalidReplicationChange) + case roachpb.VOTER_FULL, roachpb.VOTER_DEMOTING_NON_VOTER, roachpb.VOTER_INCOMING, + roachpb.VOTER_DEMOTING_LEARNER: + return errors.Mark(errors.Errorf( + "unable to add replica %v which is already present as a voter in %s", chg.Target, desc), + errMarkInvalidReplicationChange) + default: + panic(fmt.Sprintf("unknown replica type %v", t)) } - - // Otherwise, we already had a full voter replica. Can't add another to - // this store. - return errors.Mark( - errors.Errorf("unable to add replica %v which is already present in %s", chg.Target, desc), - errMarkInvalidReplicationChange) } for _, chg := range byStoreID { @@ -1366,37 +1457,35 @@ func (r *Replica) lockLearnerSnapshot( } } -// 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. -// Cluster version permitting, voter removals (from the replication changes) -// will preferably be carried out by first demoting to a learner instead of -// outright removal (this avoids a [raft-bug] that can lead to unavailability). -// All of this occurs in one atomic raft membership change which is carried out -// across two phases. 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, -// maybeLeaveAtomicReplicationChange can fix this, but it is the caller's job to -// do this when necessary. +// execReplicationChangesForVoters carries out the atomic membership change that +// finalizes the addition and/or removal of voting 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. Cluster version permitting, voter removals (from the replication +// changes) will preferably be carried out by first demoting to a learner +// instead of outright removal (this avoids a [raft-bug] that can lead to +// unavailability). All of this occurs in one atomic raft membership change +// which is carried out across two phases. 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, maybeLeaveAtomicReplicationChange can fix this, but it is the +// caller's job to do this when necessary. // // The atomic membership change is carried out chiefly via the construction of a // suitable ChangeReplicasTrigger, see prepareChangeReplicasTrigger for details. // -// Contrary to the name, *all* membership changes go through this method, even -// those that add/remove only a single voter, though the simple protocol is used -// when this is opportune. Notably, demotions can never use the simple protocol, -// even if only a single voter is being demoted, due to a (liftable) limitation -// in etcd/raft. +// When adding/removing only a single voter, joint consensus is not used. +// Notably, demotions must always use joint consensus, even if only a single +// voter is being demoted, due to a (liftable) limitation in etcd/raft. // // [raft-bug]: https://github.com/etcd-io/etcd/issues/11284 -func (r *Replica) atomicReplicationChange( +func (r *Replica) execReplicationChangesForVoters( ctx context.Context, desc *roachpb.RangeDescriptor, priority SnapshotRequest_Priority, reason kvserverpb.RangeLogEventReason, details string, - chgs roachpb.ReplicationChanges, + voterAdditions, voterRemovals []roachpb.ReplicationTarget, ) (*roachpb.RangeDescriptor, error) { // TODO(dan): We allow ranges with learner replicas to split, so in theory // this may want to detect that and retry, sending a snapshot and promoting @@ -1430,9 +1519,9 @@ func (r *Replica) atomicReplicationChange( "waited for %s and replication hasn't caught up with descriptor update", timeutil.Since(start)) } - iChgs := make([]internalReplicationChange, 0, len(chgs)) + iChgs := make([]internalReplicationChange, 0, len(voterAdditions)+len(voterRemovals)) - for _, target := range chgs.VoterAdditions() { + for _, target := range voterAdditions { 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. @@ -1469,16 +1558,16 @@ func (r *Replica) atomicReplicationChange( } } - if adds := chgs.VoterAdditions(); len(adds) > 0 { + if adds := voterAdditions; len(adds) > 0 { if fn := r.store.cfg.TestingKnobs.ReplicaAddStopAfterLearnerSnapshot; fn != nil && fn(adds) { return desc, nil } } - for _, target := range chgs.VoterRemovals() { + for _, target := range voterRemovals { typ := internalChangeTypeRemove if rDesc, ok := desc.GetReplicaDescriptor(target.StoreID); ok && rDesc.GetType() == roachpb.VOTER_FULL { - typ = internalChangeTypeDemoteVoter + typ = internalChangeTypeDemoteVoterToLearner } iChgs = append(iChgs, internalReplicationChange{target: target, typ: typ}) } @@ -1559,14 +1648,21 @@ const ( _ internalChangeType = iota + 1 internalChangeTypeAddLearner internalChangeTypeAddNonVoter + // NB: internalChangeTypePromote{Learner,Voter} are quite similar to each + // other. We only chose to differentiate them in order to be able to assert on + // the type of replica being promoted. See `prepareChangeReplicasTrigger`. internalChangeTypePromoteLearner - // internalChangeTypeDemoteVoter changes a voter to an ephemeral learner. This will - // necessarily go through joint consensus since it requires two individual - // changes (only one changes the quorum, so we could allow it in a simple - // change too, with some work here and upstream). Demotions are treated like - // removals throughout (i.e. they show up in `ChangeReplicasTrigger.Removed()`, - // but not in `.Added()`). - internalChangeTypeDemoteVoter + internalChangeTypePromoteNonVoter + // internalChangeTypeDemoteVoterToLearner changes a voter to an ephemeral + // learner. This will necessarily go through joint consensus since it requires + // two individual changes (only one changes the quorum, so we could allow it + // in a simple change too, with some work here and upstream). Demotions are + // treated like removals throughout (i.e. they show up in + // `ChangeReplicasTrigger.Removed()`, but not in `.Added()`). + internalChangeTypeDemoteVoterToLearner + // internalChangeTypeDemoteVoterToNonVoter demotes a voter to a non-voter. + // This, like the demotion to learner, will go through joint consensus. + internalChangeTypeDemoteVoterToNonVoter // NB: can't remove multiple learners at once (need to remove at least one // voter with them), see: // https://github.com/cockroachdb/cockroach/pull/40268 @@ -1588,7 +1684,9 @@ func (c internalReplicationChanges) leaveJoint() bool { return len(c) == 0 } func (c internalReplicationChanges) useJoint() bool { // NB: demotions require joint consensus because of limitations in etcd/raft. // These could be lifted, but it doesn't seem worth it. - return len(c) > 1 || c[0].typ == internalChangeTypeDemoteVoter + isDemotion := c[0].typ == internalChangeTypeDemoteVoterToNonVoter || + c[0].typ == internalChangeTypeDemoteVoterToLearner + return len(c) > 1 || isDemotion } func prepareChangeReplicasTrigger( @@ -1626,7 +1724,19 @@ func prepareChangeReplicasTrigger( } rDesc, prevTyp, ok := updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, typ) if !ok || prevTyp != roachpb.LEARNER { - return nil, errors.Errorf("cannot promote target %v which is missing as Learner", chg.target) + return nil, errors.Errorf("cannot promote target %v which is missing as LEARNER", + chg.target) + } + added = append(added, rDesc) + case internalChangeTypePromoteNonVoter: + typ := roachpb.VOTER_FULL + if useJoint { + typ = roachpb.VOTER_INCOMING + } + rDesc, prevTyp, ok := updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, typ) + if !ok || prevTyp != roachpb.NON_VOTER { + return nil, errors.Errorf("cannot promote target %v which is missing as NON_VOTER", + chg.target) } added = append(added, rDesc) case internalChangeTypeRemove: @@ -1642,15 +1752,30 @@ func prepareChangeReplicasTrigger( // NB: prevTyp is already known to be VOTER_FULL because of // !InAtomicReplicationChange() and the learner handling // above. We check it anyway. - return nil, errors.Errorf("cannot transition from %s to VOTER_OUTGOING", prevTyp) + return nil, errors.AssertionFailedf("cannot transition from %s to VOTER_OUTGOING", prevTyp) } else { rDesc, _, _ = updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, roachpb.VOTER_OUTGOING) } removed = append(removed, rDesc) - case internalChangeTypeDemoteVoter: + case internalChangeTypeDemoteVoterToLearner: // Demotion is similar to removal, except that a demotion // cannot apply to a learner, and that the resulting type is // different when entering a joint config. + rDesc, ok := updatedDesc.GetReplicaDescriptor(chg.target.StoreID) + if !ok { + return nil, errors.Errorf("target %s not found", chg.target) + } + if !useJoint { + // NB: this won't fire because cc.useJoint() is always true when + // there's a demotion. This is just a sanity check. + return nil, errors.AssertionFailedf("demotions require joint consensus") + } + if prevTyp := rDesc.GetType(); prevTyp != roachpb.VOTER_FULL { + return nil, errors.Errorf("cannot transition from %s to VOTER_DEMOTING_LEARNER", prevTyp) + } + rDesc, _, _ = updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, roachpb.VOTER_DEMOTING_LEARNER) + removed = append(removed, rDesc) + case internalChangeTypeDemoteVoterToNonVoter: rDesc, ok := updatedDesc.GetReplicaDescriptor(chg.target.StoreID) if !ok { return nil, errors.Errorf("target %s not found", chg.target) @@ -1661,9 +1786,9 @@ func prepareChangeReplicasTrigger( return nil, errors.Errorf("demotions require joint consensus") } if prevTyp := rDesc.GetType(); prevTyp != roachpb.VOTER_FULL { - return nil, errors.Errorf("cannot transition from %s to VOTER_DEMOTING", prevTyp) + return nil, errors.Errorf("cannot transition from %s to VOTER_DEMOTING_NON_VOTER", prevTyp) } - rDesc, _, _ = updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, roachpb.VOTER_DEMOTING) + rDesc, _, _ = updatedDesc.SetReplicaType(chg.target.NodeID, chg.target.StoreID, roachpb.VOTER_DEMOTING_NON_VOTER) removed = append(removed, rDesc) default: return nil, errors.Errorf("unsupported internal change type %d", chg.typ) @@ -1683,9 +1808,12 @@ func prepareChangeReplicasTrigger( case roachpb.VOTER_OUTGOING: updatedDesc.RemoveReplica(rDesc.NodeID, rDesc.StoreID) isJoint = true - case roachpb.VOTER_DEMOTING: + case roachpb.VOTER_DEMOTING_LEARNER: updatedDesc.SetReplicaType(rDesc.NodeID, rDesc.StoreID, roachpb.LEARNER) isJoint = true + case roachpb.VOTER_DEMOTING_NON_VOTER: + updatedDesc.SetReplicaType(rDesc.NodeID, rDesc.StoreID, roachpb.NON_VOTER) + isJoint = true default: } } @@ -1786,7 +1914,12 @@ func execChangeReplicasTxn( } if chgs.leaveJoint() && !desc.Replicas().InAtomicReplicationChange() { // Nothing to do. See comment in 'check' above for details. - returnDesc = desc + // + // NB: We return the `referenceDesc` and not the `desc` we just read from + // KV, as it is possible that we raced with another `ChangeReplicas` + // operation that has mutated the RangeDescriptor. In such a scenario, we + // want all future `check`s inside `execChangeReplicasImpl` to fail. + returnDesc = referenceDesc return nil } // Note that we are now using the descriptor from KV, not the one passed @@ -1902,6 +2035,27 @@ func execChangeReplicasTxn( return returnDesc, nil } +func getInternalChangesForExplicitPromotionsAndDemotions( + voterDemotions, nonVoterPromotions []roachpb.ReplicationTarget, +) []internalReplicationChange { + iChgs := make([]internalReplicationChange, len(voterDemotions)+len(nonVoterPromotions)) + for i := range voterDemotions { + iChgs[i] = internalReplicationChange{ + target: voterDemotions[i], + typ: internalChangeTypeDemoteVoterToNonVoter, + } + } + + for j := range nonVoterPromotions { + iChgs[j+len(voterDemotions)] = internalReplicationChange{ + target: nonVoterPromotions[j], + typ: internalChangeTypePromoteNonVoter, + } + } + + return iChgs +} + type logChangeFn func( ctx context.Context, txn *kv.Txn, @@ -2539,7 +2693,7 @@ func (s *Store) relocateReplicas( } type relocationArgs struct { - targetsToAdd, targetsToRemove []roachpb.ReplicaDescriptor + targetsToAdd, targetsToRemove []roachpb.ReplicationTarget addOp, removeOp roachpb.ReplicaChangeType relocationTargets []roachpb.ReplicationTarget targetType targetReplicaType @@ -2622,9 +2776,7 @@ func (s *Store) relocateOne( candidateTargets := args.targetsToAdd if args.targetType == voterTarget && storeHasReplica(args.relocationTargets[0].StoreID, candidateTargets) { - candidateTargets = []roachpb.ReplicaDescriptor{ - {NodeID: args.relocationTargets[0].NodeID, StoreID: args.relocationTargets[0].StoreID}, - } + candidateTargets = []roachpb.ReplicationTarget{args.relocationTargets[0]} } // The storeList's list of stores is used to constrain which stores the @@ -2753,9 +2905,14 @@ func (s *Store) relocateOne( return ops, transferTarget, nil } -// subtractTargets returns the set of replication targets in `left` but not in +// subtractTargets returns the set of replica descriptors in `left` but not in // `right` (i.e. left - right). -func subtractTargets(left, right []roachpb.ReplicationTarget) (diff []roachpb.ReplicaDescriptor) { +// +// TODO(aayush): Make this and the `intersectTargets()` method below +// (along with other utility methods operating on `ReplicationTarget`) operate +// over an interface that both `ReplicaDescriptor` and `ReplicationTarget` +// satisfy. +func subtractTargets(left, right []roachpb.ReplicationTarget) (diff []roachpb.ReplicationTarget) { for _, t := range left { found := false for _, replicaDesc := range right { @@ -2765,15 +2922,33 @@ func subtractTargets(left, right []roachpb.ReplicationTarget) (diff []roachpb.Re } } if !found { - diff = append(diff, roachpb.ReplicaDescriptor{ - NodeID: t.NodeID, - StoreID: t.StoreID, - }) + diff = append(diff, t) } } return diff } +// intersectTargets returns the set of replica descriptors in `left` and +// `right`. +func intersectTargets( + left, right []roachpb.ReplicationTarget, +) (intersection []roachpb.ReplicationTarget) { + isInLeft := func(id roachpb.StoreID) bool { + for _, r := range left { + if r.StoreID == id { + return true + } + } + return false + } + for i := range right { + if isInLeft(right[i].StoreID) { + intersection = append(intersection, right[i]) + } + } + return intersection +} + // adminScatter moves replicas and leaseholders for a selection of ranges. func (r *Replica) adminScatter( ctx context.Context, args roachpb.AdminScatterRequest, diff --git a/pkg/kv/kvserver/replica_command_test.go b/pkg/kv/kvserver/replica_command_test.go index a68ec5feec50..49aaf2ea4a3f 100644 --- a/pkg/kv/kvserver/replica_command_test.go +++ b/pkg/kv/kvserver/replica_command_test.go @@ -92,6 +92,7 @@ func TestRangeDescriptorUpdateProtoChangedAcrossVersions(t *testing.T) { } } +// TODO(aayush): This test has gotten unwieldy. It needs to be table-driven. func TestValidateReplicationChanges(t *testing.T) { defer leaktest.AfterTest(t)() @@ -235,4 +236,228 @@ func TestValidateReplicationChanges(t *testing.T) { {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, }) require.NoError(t, err) + + // Test Case 18: Adding a non-voter where we already have a voting replica + // without a simultaneous removal of that voter. + err = validateReplicationChanges(descSingle, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + }) + require.Regexp(t, "unable to add replica .* which is already present as a voter", err) + + // Test Case 19: Demoting a voter to a non-voter. + err = validateReplicationChanges(descSingle, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + }) + require.NoError(t, err) + + descWithOneVoterAndNonVoter := &roachpb.RangeDescriptor{ + InternalReplicas: []roachpb.ReplicaDescriptor{ + {NodeID: 1, StoreID: 1}, + {NodeID: 2, StoreID: 2, Type: roachpb.ReplicaTypeNonVoter()}, + }, + } + + // Test Case 20: Adding a voter on a store where we already have a non-voter. + err = validateReplicationChanges(descWithOneVoterAndNonVoter, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, + }) + require.Regexp(t, "unable to add replica .* which is already present as a non-voter", err) + + // Test Case 21: Promoting a non-voter to a voter. + err = validateReplicationChanges(descWithOneVoterAndNonVoter, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, + }) + require.NoError(t, err) + + // Test Case 22: Swapping a voter with a non-voter. + err = validateReplicationChanges(descWithOneVoterAndNonVoter, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, + }) + require.NoError(t, err) +} + +func TestSynthesizeTargetsByChangeType(t *testing.T) { + type testCase struct { + name string + changes []roachpb.ReplicationChange + expPromotions, expDemotions []int32 + expVoterAdditions, expVoterRemovals []int32 + expNonVoterAdditions, expNonVoterRemovals []int32 + } + + mkTarget := func(t int32) roachpb.ReplicationTarget { + return roachpb.ReplicationTarget{NodeID: roachpb.NodeID(t), StoreID: roachpb.StoreID(t)} + } + + mkTargetList := func(targets []int32) []roachpb.ReplicationTarget { + if len(targets) == 0 { + return nil + } + res := make([]roachpb.ReplicationTarget, len(targets)) + for i, t := range targets { + res[i] = mkTarget(t) + } + return res + } + + tests := []testCase{ + { + name: "simple voter addition", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, + }, + expVoterAdditions: []int32{2}, + }, + { + name: "simple voter removal", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(2)}, + }, + expVoterRemovals: []int32{2}, + }, + { + name: "simple non-voter addition", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(2)}, + }, + expNonVoterAdditions: []int32{2}, + }, + { + name: "simple non-voter removal", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, + }, + expNonVoterRemovals: []int32{2}, + }, + { + name: "promote non_voter to voter", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, + }, + expPromotions: []int32{2}, + }, + { + name: "demote voter to non_voter", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, + }, + expDemotions: []int32{1}, + }, + { + name: "swap voter with non_voter", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, + }, + expPromotions: []int32{2}, + expDemotions: []int32{1}, + }, + { + name: "swap with simple addition", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(4)}, + }, + expPromotions: []int32{2}, + expDemotions: []int32{1}, + expVoterAdditions: []int32{4}, + }, + { + name: "swap with simple removal", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(4)}, + }, + expPromotions: []int32{2}, + expDemotions: []int32{1}, + expVoterRemovals: []int32{4}, + }, + { + name: "swap with addition promotion", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(3)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(3)}, + }, + expPromotions: []int32{2, 3}, + expDemotions: []int32{1}, + }, + { + name: "swap with additional demotion", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(4)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(4)}, + }, + expPromotions: []int32{2}, + expDemotions: []int32{1, 4}, + }, + { + name: "two swaps", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(4)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(4)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(3)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(3)}, + }, + expPromotions: []int32{2, 3}, + expDemotions: []int32{1, 4}, + }, + { + name: "all at once", + changes: []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(3)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(3)}, + {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(4)}, + {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(5)}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(6)}, + }, + expPromotions: []int32{2, 3}, + expDemotions: []int32{1}, + expVoterAdditions: []int32{4}, + expNonVoterAdditions: []int32{5}, + expNonVoterRemovals: []int32{6}, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + result := synthesizeTargetsByChangeType(test.changes) + require.Equal(t, result.nonVoterPromotions, mkTargetList(test.expPromotions)) + require.Equal(t, result.voterDemotions, mkTargetList(test.expDemotions)) + require.Equal(t, result.voterAdditions, mkTargetList(test.expVoterAdditions)) + require.Equal(t, result.voterRemovals, mkTargetList(test.expVoterRemovals)) + require.Equal(t, result.nonVoterAdditions, mkTargetList(test.expNonVoterAdditions)) + require.Equal(t, result.nonVoterRemovals, mkTargetList(test.expNonVoterRemovals)) + }) + } } diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index 960968934d58..93c3b709c382 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -47,8 +47,8 @@ func predOutgoing(rDesc roachpb.ReplicaDescriptor) bool { return rDesc.GetType() == roachpb.VOTER_OUTGOING } -func predDemoting(rDesc roachpb.ReplicaDescriptor) bool { - return rDesc.GetType() == roachpb.VOTER_DEMOTING +func predDemotingToLearner(rDesc roachpb.ReplicaDescriptor) bool { + return rDesc.GetType() == roachpb.VOTER_DEMOTING_LEARNER } type replicationTestKnobs struct { @@ -768,7 +768,7 @@ func TestJointConfigLease(t *testing.T) { // it's asked to do. desc = tc.RemoveVotersOrFatal(t, k, tc.Target(1)) err = tc.TransferRangeLease(desc, tc.Target(1)) - exp = `cannot transfer lease to replica of type VOTER_DEMOTING` + exp = `cannot transfer lease to replica of type VOTER_DEMOTING_LEARNER` require.True(t, testutils.IsError(err, exp), err) } @@ -850,7 +850,7 @@ func TestLearnerAndVoterOutgoingFollowerRead(t *testing.T) { // Remove the voter and remain in joint config. scratchDesc = tc.RemoveVotersOrFatal(t, scratchStartKey, tc.Target(1)) require.True(t, scratchDesc.Replicas().InAtomicReplicationChange(), scratchDesc) - require.Len(t, scratchDesc.Replicas().FilterToDescriptors(predDemoting), 1) + require.Len(t, scratchDesc.Replicas().FilterToDescriptors(predDemotingToLearner), 1) // Can't serve follower read from the VOTER_OUTGOING. check() @@ -909,7 +909,7 @@ func TestLearnerOrJointConfigAdminRelocateRange(t *testing.T) { atomic.StoreInt64(<k.replicationAlwaysUseJointConfig, 1) desc := tc.RemoveVotersOrFatal(t, scratchStartKey, tc.Target(3)) require.True(t, desc.Replicas().InAtomicReplicationChange(), desc) - require.Len(t, desc.Replicas().FilterToDescriptors(predDemoting), 1) + require.Len(t, desc.Replicas().FilterToDescriptors(predDemotingToLearner), 1) atomic.StoreInt64(<k.replicaAddStopAfterJointConfig, 0) check([]roachpb.ReplicationTarget{tc.Target(0), tc.Target(1), tc.Target(2)}) } @@ -974,29 +974,29 @@ func TestLearnerAndJointConfigAdminMerge(t *testing.T) { // desc{1,2,3} = (VOTER_FULL, VOTER_INCOMING) (VOTER_FULL) (VOTER_FULL, VOTER_INCOMING) checkFails() - // Turn the incoming voters on desc1 and desc3 into VOTER_DEMOTINGs. - // desc{1,2,3} = (VOTER_FULL, VOTER_DEMOTING) (VOTER_FULL) (VOTER_FULL, VOTER_DEMOTING) + // Turn the incoming voters on desc1 and desc3 into VOTER_DEMOTING_LEARNERs. + // desc{1,2,3} = (VOTER_FULL, VOTER_DEMOTING_LEARNER) (VOTER_FULL) (VOTER_FULL, VOTER_DEMOTING_LEARNER) desc1 = tc.RemoveVotersOrFatal(t, desc1.StartKey.AsRawKey(), tc.Target(1)) - require.Len(t, desc1.Replicas().FilterToDescriptors(predDemoting), 1) + require.Len(t, desc1.Replicas().FilterToDescriptors(predDemotingToLearner), 1) desc3 = tc.RemoveVotersOrFatal(t, desc3.StartKey.AsRawKey(), tc.Target(1)) - require.Len(t, desc3.Replicas().FilterToDescriptors(predDemoting), 1) + require.Len(t, desc3.Replicas().FilterToDescriptors(predDemotingToLearner), 1) - // VOTER_DEMOTING on the lhs or rhs should fail. + // VOTER_DEMOTING_LEARNER on the lhs or rhs should fail. checkFails() // Add a VOTER_INCOMING to desc2 to make sure it actually excludes this type // of replicas from merges (rather than really just checking whether the // replica sets are equal). - // desc{1,2,3} = (VOTER_FULL, VOTER_DEMOTING) (VOTER_FULL, VOTER_INCOMING) (VOTER_FULL, VOTER_DEMOTING) + // desc{1,2,3} = (VOTER_FULL, VOTER_DEMOTING_LEARNER) (VOTER_FULL, VOTER_INCOMING) (VOTER_FULL, VOTER_DEMOTING_LEARNER) desc2 := tc.AddVotersOrFatal(t, splitKey1, tc.Target(1)) require.Len(t, desc2.Replicas().FilterToDescriptors(predIncoming), 1) checkFails() - // Ditto VOTER_DEMOTING. - // desc{1,2,3} = (VOTER_FULL, VOTER_DEMOTING) (VOTER_FULL, VOTER_DEMOTING) (VOTER_FULL, VOTER_DEMOTING) + // Ditto VOTER_DEMOTING_LEARNER. + // desc{1,2,3} = (VOTER_FULL, VOTER_DEMOTING_LEARNER) (VOTER_FULL, VOTER_DEMOTING_LEARNER) (VOTER_FULL, VOTER_DEMOTING_LEARNER) desc2 = tc.RemoveVotersOrFatal(t, desc2.StartKey.AsRawKey(), tc.Target(1)) - require.Len(t, desc2.Replicas().FilterToDescriptors(predDemoting), 1) + require.Len(t, desc2.Replicas().FilterToDescriptors(predDemotingToLearner), 1) checkFails() } @@ -1094,7 +1094,7 @@ func TestMergeQueueSeesLearnerOrJointConfig(t *testing.T) { desc = splitAndUnsplit() ltk.withStopAfterJointConfig(func() { descRight := tc.RemoveVotersOrFatal(t, desc.EndKey.AsRawKey(), tc.Target(1)) - require.Len(t, descRight.Replicas().FilterToDescriptors(predDemoting), 1, desc) + require.Len(t, descRight.Replicas().FilterToDescriptors(predDemotingToLearner), 1, desc) }) // This should transition out (i.e. remove the voter on s2 for the RHS) diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index 07b5b159bd15..8b7df4c5c981 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -574,7 +574,7 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) { leader: self + 1, // The leader type makes it ineligible to get the lease. Thus, the local // proposal will not be rejected. - leaderRepType: roachpb.VOTER_DEMOTING, + leaderRepType: roachpb.VOTER_DEMOTING_LEARNER, expRejection: false, }, { diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index f7498b1d1c41..85ea16017361 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -777,9 +777,10 @@ func (r *Replica) applySnapshot( // NB: A replica of type LEARNER can receive a non-initial snapshot (via // the snapshot queue) if we end up truncating the raft log before it // gets promoted to a voter. We count such snapshot applications as - // "applied by voters" here. - case roachpb.VOTER_FULL, roachpb.VOTER_INCOMING, roachpb.VOTER_DEMOTING, - roachpb.VOTER_OUTGOING, roachpb.LEARNER: + // "applied by voters" here, since the LEARNER will soon be promoted to + // a voting replica. + case roachpb.VOTER_FULL, roachpb.VOTER_INCOMING, roachpb.VOTER_DEMOTING_LEARNER, + roachpb.VOTER_OUTGOING, roachpb.LEARNER, roachpb.VOTER_DEMOTING_NON_VOTER: r.store.metrics.RangeSnapshotsAppliedByVoters.Inc(1) case roachpb.NON_VOTER: r.store.metrics.RangeSnapshotsAppliedByNonVoters.Inc(1) diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index ac468760c49f..e58322de39bf 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -12971,17 +12971,17 @@ func TestPrepareChangeReplicasTrigger(t *testing.T) { // Demoting two voters. mk( - "ENTER_JOINT(r2 l2 r3 l3) [(n200,s200):2VOTER_DEMOTING (n300,s300):3VOTER_DEMOTING]: after=[(n100,s100):1 (n200,s200):2VOTER_DEMOTING (n300,s300):3VOTER_DEMOTING] next=4", + "ENTER_JOINT(r2 l2 r3 l3) [(n200,s200):2VOTER_DEMOTING_LEARNER (n300,s300):3VOTER_DEMOTING_LEARNER]: after=[(n100,s100):1 (n200,s200):2VOTER_DEMOTING_LEARNER (n300,s300):3VOTER_DEMOTING_LEARNER] next=4", typOp{roachpb.VOTER_FULL, noop}, - typOp{roachpb.VOTER_FULL, internalChangeTypeDemoteVoter}, - typOp{roachpb.VOTER_FULL, internalChangeTypeDemoteVoter}, + typOp{roachpb.VOTER_FULL, internalChangeTypeDemoteVoterToLearner}, + typOp{roachpb.VOTER_FULL, internalChangeTypeDemoteVoterToLearner}, ), // Leave joint config entered via demotion. mk( "LEAVE_JOINT: after=[(n100,s100):1 (n200,s200):2LEARNER (n300,s300):3LEARNER] next=4", typOp{roachpb.VOTER_FULL, noop}, - typOp{roachpb.VOTER_DEMOTING, noop}, - typOp{roachpb.VOTER_DEMOTING, noop}, + typOp{roachpb.VOTER_DEMOTING_LEARNER, noop}, + typOp{roachpb.VOTER_DEMOTING_LEARNER, noop}, ), } diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index be49480c4918..f60ed2342bdd 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -433,7 +433,7 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( continue } - if len(preferred) > 0 && !storeHasReplica(candidate.StoreID, preferred) { + if len(preferred) > 0 && !storeHasReplica(candidate.StoreID, roachpb.MakeReplicaSet(preferred).ReplicationTargets()) { log.VEventf(ctx, 3, "s%d not a preferred leaseholder for r%d; preferred: %v", candidate.StoreID, desc.RangeID, preferred) continue diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index e4f91d32d8ae..20f67eb3f160 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1618,7 +1618,7 @@ func confChangeImpl( if err := checkExists(rDesc); err != nil { return nil, err } - case VOTER_DEMOTING: + case VOTER_DEMOTING_LEARNER, VOTER_DEMOTING_NON_VOTER: // If a voter is demoted through joint consensus, it will // be turned into a demoting voter first. if err := checkExists(rDesc); err != nil { @@ -1630,12 +1630,12 @@ func confChangeImpl( NodeID: uint64(rDesc.ReplicaID), }) case LEARNER: - // A learner could in theory show up in the descriptor if the - // removal was really a demotion and no joint consensus is used. - // But etcd/raft currently forces us to go through joint consensus - // when demoting, so demotions will always have a VOTER_DEMOTING - // instead. We must be straight-up removing a voter or learner, so - // the target should be gone from the descriptor at this point. + // A learner could in theory show up in the descriptor if the removal was + // really a demotion and no joint consensus is used. But etcd/raft + // currently forces us to go through joint consensus when demoting, so + // demotions will always have a VOTER_DEMOTING_LEARNER instead. We must be + // straight-up removing a voter or learner, so the target should be gone + // from the descriptor at this point. if err := checkNotExists(rDesc); err != nil { return nil, err } @@ -1672,17 +1672,12 @@ func confChangeImpl( // We're adding a voter, but will transition into a joint config // first. changeType = raftpb.ConfChangeAddNode - case LEARNER: - // We're adding a learner. - // Note that we're guaranteed by virtue of the upstream - // ChangeReplicas txn that this learner is not currently a voter. - // Demotions (i.e. transitioning from voter to learner) are not - // represented in `added`; they're handled in `removed` above. - changeType = raftpb.ConfChangeAddLearnerNode - case NON_VOTER: - // We're adding a non-voter. Like the case above, we're guaranteed that - // this learner is not a voter. Promotions of non-voters to voters and - // demotions vice-versa are not currently supported. + case LEARNER, NON_VOTER: + // We're adding a learner or non-voter. + // Note that we're guaranteed by virtue of the upstream ChangeReplicas txn + // that this learner/non-voter is not currently a voter. Demotions (i.e. + // transitioning from voter to learner/non-voter) are not represented in + // `added`; they're handled in `removed` above. changeType = raftpb.ConfChangeAddLearnerNode default: // A voter that is demoting was just removed and re-added in the @@ -1703,7 +1698,7 @@ func confChangeImpl( var enteringJoint bool for _, rDesc := range replicas { switch rDesc.GetType() { - case VOTER_INCOMING, VOTER_OUTGOING, VOTER_DEMOTING: + case VOTER_INCOMING, VOTER_OUTGOING, VOTER_DEMOTING_LEARNER, VOTER_DEMOTING_NON_VOTER: enteringJoint = true default: } diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 9e2b1c658a2f..0e0fb4a47579 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -98,7 +98,7 @@ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } func (ValueType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{0} + return fileDescriptor_data_91736d498bf42716, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -128,7 +128,7 @@ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{1} + return fileDescriptor_data_91736d498bf42716, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -180,7 +180,7 @@ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } func (TransactionStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{2} + return fileDescriptor_data_91736d498bf42716, []int{2} } // RangeClosedTimestampPolicy represents the policy used by the leaseholder of a @@ -216,7 +216,7 @@ func (x RangeClosedTimestampPolicy) String() string { return proto.EnumName(RangeClosedTimestampPolicy_name, int32(x)) } func (RangeClosedTimestampPolicy) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{3} + return fileDescriptor_data_91736d498bf42716, []int{3} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -233,7 +233,7 @@ type Span struct { func (m *Span) Reset() { *m = Span{} } func (*Span) ProtoMessage() {} func (*Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{0} + return fileDescriptor_data_91736d498bf42716, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -285,7 +285,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_9fe3b4298c3b6243, []int{1} + return fileDescriptor_data_91736d498bf42716, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -321,7 +321,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_9fe3b4298c3b6243, []int{2} + return fileDescriptor_data_91736d498bf42716, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -359,7 +359,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_9fe3b4298c3b6243, []int{3} + return fileDescriptor_data_91736d498bf42716, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -399,7 +399,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_9fe3b4298c3b6243, []int{4} + return fileDescriptor_data_91736d498bf42716, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -461,7 +461,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_9fe3b4298c3b6243, []int{5} + return fileDescriptor_data_91736d498bf42716, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -490,11 +490,11 @@ var xxx_messageInfo_MergeTrigger proto.InternalMessageInfo // 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 VOTER_OUTGOING or VOTER_DEMOTING (if they are -// to be turned into learners instead); as a caveat a single demotion already -// counts as two changes (and is tracked as a Removal() only). This joint -// configuration is left via another ChangeReplicasTrigger which does not -// specify any additions nor removals. +// "removed" replicas are of type VOTER_OUTGOING or VOTER_DEMOTING_{LEARNER, +// NON_VOTER} (if they are to be turned into learners or non-voters instead); as +// a caveat a single demotion already counts as two changes (and is tracked as a +// Removal() only). 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). @@ -531,7 +531,7 @@ type ChangeReplicasTrigger struct { func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } func (*ChangeReplicasTrigger) ProtoMessage() {} func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{6} + return fileDescriptor_data_91736d498bf42716, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -573,7 +573,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_9fe3b4298c3b6243, []int{7} + return fileDescriptor_data_91736d498bf42716, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -619,7 +619,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_9fe3b4298c3b6243, []int{8} + return fileDescriptor_data_91736d498bf42716, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -658,7 +658,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_9fe3b4298c3b6243, []int{9} + return fileDescriptor_data_91736d498bf42716, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -727,7 +727,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_9fe3b4298c3b6243, []int{10} + return fileDescriptor_data_91736d498bf42716, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -902,7 +902,7 @@ type Transaction struct { func (m *Transaction) Reset() { *m = Transaction{} } func (*Transaction) ProtoMessage() {} func (*Transaction) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{11} + return fileDescriptor_data_91736d498bf42716, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -953,7 +953,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_9fe3b4298c3b6243, []int{12} + return fileDescriptor_data_91736d498bf42716, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -993,7 +993,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_9fe3b4298c3b6243, []int{13} + return fileDescriptor_data_91736d498bf42716, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1030,7 +1030,7 @@ func (m *Intent_SingleKeySpan) Reset() { *m = Intent_SingleKeySpan{} } func (m *Intent_SingleKeySpan) String() string { return proto.CompactTextString(m) } func (*Intent_SingleKeySpan) ProtoMessage() {} func (*Intent_SingleKeySpan) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{13, 0} + return fileDescriptor_data_91736d498bf42716, []int{13, 0} } func (m *Intent_SingleKeySpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1067,7 +1067,7 @@ func (m *LockAcquisition) Reset() { *m = LockAcquisition{} } func (m *LockAcquisition) String() string { return proto.CompactTextString(m) } func (*LockAcquisition) ProtoMessage() {} func (*LockAcquisition) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{14} + return fileDescriptor_data_91736d498bf42716, []int{14} } func (m *LockAcquisition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1108,7 +1108,7 @@ func (m *LockUpdate) Reset() { *m = LockUpdate{} } func (m *LockUpdate) String() string { return proto.CompactTextString(m) } func (*LockUpdate) ProtoMessage() {} func (*LockUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{15} + return fileDescriptor_data_91736d498bf42716, []int{15} } func (m *LockUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1145,7 +1145,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_9fe3b4298c3b6243, []int{16} + return fileDescriptor_data_91736d498bf42716, []int{16} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1210,7 +1210,7 @@ type Lease struct { func (m *Lease) Reset() { *m = Lease{} } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{17} + return fileDescriptor_data_91736d498bf42716, []int{17} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1255,7 +1255,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_9fe3b4298c3b6243, []int{18} + return fileDescriptor_data_91736d498bf42716, []int{18} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1312,7 +1312,7 @@ func (m *LeafTxnInputState) Reset() { *m = LeafTxnInputState{} } func (m *LeafTxnInputState) String() string { return proto.CompactTextString(m) } func (*LeafTxnInputState) ProtoMessage() {} func (*LeafTxnInputState) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{19} + return fileDescriptor_data_91736d498bf42716, []int{19} } func (m *LeafTxnInputState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1365,7 +1365,7 @@ func (m *LeafTxnFinalState) Reset() { *m = LeafTxnFinalState{} } func (m *LeafTxnFinalState) String() string { return proto.CompactTextString(m) } func (*LeafTxnFinalState) ProtoMessage() {} func (*LeafTxnFinalState) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{20} + return fileDescriptor_data_91736d498bf42716, []int{20} } func (m *LeafTxnFinalState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1404,7 +1404,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} } func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) } func (*ClientRangeInfo) ProtoMessage() {} func (*ClientRangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{21} + return fileDescriptor_data_91736d498bf42716, []int{21} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1441,7 +1441,7 @@ type RangeInfo struct { func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_9fe3b4298c3b6243, []int{22} + return fileDescriptor_data_91736d498bf42716, []int{22} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7249,9 +7249,9 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_9fe3b4298c3b6243) } +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_91736d498bf42716) } -var fileDescriptor_data_9fe3b4298c3b6243 = []byte{ +var fileDescriptor_data_91736d498bf42716 = []byte{ // 2606 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0x4f, 0x70, 0x23, 0x47, 0xd5, 0xf7, 0x58, 0x23, 0x69, 0xf4, 0x2c, 0xc9, 0xe3, 0x5e, 0xdb, 0xab, 0x75, 0xbe, 0xcf, 0xda, diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 61fc48f443d6..d0789f67e8ff 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -181,11 +181,11 @@ enum ReplicaChangeType { // 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 VOTER_OUTGOING or VOTER_DEMOTING (if they are -// to be turned into learners instead); as a caveat a single demotion already -// counts as two changes (and is tracked as a Removal() only). This joint -// configuration is left via another ChangeReplicasTrigger which does not -// specify any additions nor removals. +// "removed" replicas are of type VOTER_OUTGOING or VOTER_DEMOTING_{LEARNER, +// NON_VOTER} (if they are to be turned into learners or non-voters instead); as +// a caveat a single demotion already counts as two changes (and is tracked as a +// Removal() only). This joint configuration is left via another +// ChangeReplicasTrigger which does not specify any additions nor removals. message ChangeReplicasTrigger { option (gogoproto.goproto_stringer) = false; diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 78f2436ee49a..2df0e013056b 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1697,7 +1697,7 @@ func TestChangeReplicasTrigger_String(t *testing.T) { vi := VOTER_INCOMING vo := VOTER_OUTGOING - vd := VOTER_DEMOTING + vd := VOTER_DEMOTING_LEARNER l := LEARNER repl1 := ReplicaDescriptor{NodeID: 1, StoreID: 2, ReplicaID: 3, Type: &vi} repl2 := ReplicaDescriptor{NodeID: 4, StoreID: 5, ReplicaID: 6, Type: &vo} @@ -1722,9 +1722,9 @@ func TestChangeReplicasTrigger_String(t *testing.T) { } act := crt.String() exp := "ENTER_JOINT(r6 r12 l12 v3) [(n1,s2):3VOTER_INCOMING], " + - "[(n4,s5):6VOTER_OUTGOING (n10,s11):12VOTER_DEMOTING]: " + + "[(n4,s5):6VOTER_OUTGOING (n10,s11):12VOTER_DEMOTING_LEARNER]: " + "after=[(n1,s2):3VOTER_INCOMING (n4,s5):6VOTER_OUTGOING (n7,s8):9LEARNER " + - "(n10,s11):12VOTER_DEMOTING] next=10" + "(n10,s11):12VOTER_DEMOTING_LEARNER] next=10" require.Equal(t, exp, act) crt.InternalRemovedReplicas = nil @@ -1893,14 +1893,14 @@ func TestChangeReplicasTrigger_ConfChange(t *testing.T) { ), del: sl( // Voter removals. - LEARNER, 2, VOTER_OUTGOING, 8, VOTER_DEMOTING, 9, + LEARNER, 2, VOTER_OUTGOING, 8, VOTER_DEMOTING_LEARNER, 9, ), repls: sl( // Replicas. VOTER_FULL, 1, VOTER_INCOMING, 6, // added VOTER_INCOMING, 3, // added - VOTER_DEMOTING, 9, // removing + VOTER_DEMOTING_LEARNER, 9, // removing LEARNER, 4, // added VOTER_OUTGOING, 8, // removing VOTER_FULL, 10, diff --git a/pkg/roachpb/metadata.pb.go b/pkg/roachpb/metadata.pb.go index dd1be83f483c..942a5578e222 100644 --- a/pkg/roachpb/metadata.pb.go +++ b/pkg/roachpb/metadata.pb.go @@ -25,25 +25,27 @@ var _ = math.Inf const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // 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_DEMOTING and VOTER_INCOMING types are used as -// well to denote voters which are being downgraded to learners and newly added -// by the change, respectively. A demoting voter is turning into a learner, -// which we prefer over a direct removal, which was used prior to v20.1 and -// uses the VOTER_OUTGOING type instead (see #42251). +// normal operation, VOTER_FULL, NON_VOTER, and LEARNER are the only used +// states. However, atomic replication changes require a transition through a +// "joint config"; in this joint config, the VOTER_DEMOTING_{LEARNER, NON_VOTER} +// and VOTER_INCOMING types are used as well to denote voters which are being +// downgraded to learners and newly added by the change, respectively. When +// being removed, a demoting voter is turning into a learner, which we prefer +// over a direct removal, which was used prior to v20.1 and uses the +// VOTER_OUTGOING type instead (see VersionChangeReplicasDemotion for details on +// why we're not doing that any more). // -// All voter types indicate a replica that participates in all raft activities, +// 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 a joint config, two // separate majorities are required: one from the set of replicas that have -// either type VOTER or VOTER_OUTOING or VOTER_DEMOTING, 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. +// either type VOTER or VOTER_OUTGOING or VOTER_DEMOTING_{LEARNER, NON_VOTER}, +// 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. type ReplicaType int32 const ( @@ -64,19 +66,19 @@ const ( // Note: We're not using VOTER_OUTGOING since 20.1. We're using VOTER_DEMOTING // instead. See #42251. VOTER_OUTGOING ReplicaType = 3 - // VOTER_DEMOTING indicates a voting replica that will become a learner once - // the ongoing atomic replication change is finalized; that is, it is in the - // process of being demoted. Since learners are currently short-lived, this - // replica is really being removed, with an intermediate step, and no work - // should be assigned to it. - VOTER_DEMOTING ReplicaType = 4 + // VOTER_DEMOTING_LEARNER indicates a voting replica that will become a + // learner once the ongoing atomic replication change is finalized; that is, + // it is in the process of being demoted. Since learners are currently + // short-lived, this replica is really being removed, with an intermediate + // step, and no work should be assigned to it. + VOTER_DEMOTING_LEARNER ReplicaType = 4 // LEARNER indicates a replica that applies committed entries, but does not // count towards the quorum(s). Candidates will not ask for (or take into // account) votes of (peers they consider) LEARNERs for leadership nor do // their acknowledged log entries get taken into account for determining the // committed index. Learners in CockroachDB are a short-term transient state: // a replica being added and on its way to being a VOTER_{FULL,INCOMING}, or a - // VOTER_DEMOTING being removed. + // VOTER_DEMOTING_LEARNER being removed. // // Note that once these replicas upreplicate after receiving their initial // snapshot, they will count towards the raft leader's quota pool and throttle @@ -96,23 +98,29 @@ const ( // via follower reads. See comment above ReplicaDescriptors.NonVoters() for // differences in how LEARNERs and NON_VOTERs are handled internally. NON_VOTER ReplicaType = 5 + // VOTER_DEMOTING_NON_VOTER indicates a voting replica in the outgoing group + // of a joint state, which will become a non-voter when the atomic replication + // change is finalized (i.e. when we exit the joint state). + VOTER_DEMOTING_NON_VOTER ReplicaType = 6 ) var ReplicaType_name = map[int32]string{ 0: "VOTER_FULL", 2: "VOTER_INCOMING", 3: "VOTER_OUTGOING", - 4: "VOTER_DEMOTING", + 4: "VOTER_DEMOTING_LEARNER", 1: "LEARNER", 5: "NON_VOTER", + 6: "VOTER_DEMOTING_NON_VOTER", } var ReplicaType_value = map[string]int32{ - "VOTER_FULL": 0, - "VOTER_INCOMING": 2, - "VOTER_OUTGOING": 3, - "VOTER_DEMOTING": 4, - "LEARNER": 1, - "NON_VOTER": 5, + "VOTER_FULL": 0, + "VOTER_INCOMING": 2, + "VOTER_OUTGOING": 3, + "VOTER_DEMOTING_LEARNER": 4, + "LEARNER": 1, + "NON_VOTER": 5, + "VOTER_DEMOTING_NON_VOTER": 6, } func (x ReplicaType) Enum() *ReplicaType { @@ -132,7 +140,7 @@ func (x *ReplicaType) UnmarshalJSON(data []byte) error { return nil } func (ReplicaType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{0} + return fileDescriptor_metadata_9a44b19332750f27, []int{0} } // Attributes specifies a list of arbitrary strings describing @@ -144,7 +152,7 @@ type Attributes struct { func (m *Attributes) Reset() { *m = Attributes{} } func (*Attributes) ProtoMessage() {} func (*Attributes) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{0} + return fileDescriptor_metadata_9a44b19332750f27, []int{0} } func (m *Attributes) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -170,6 +178,12 @@ func (m *Attributes) XXX_DiscardUnknown() { var xxx_messageInfo_Attributes proto.InternalMessageInfo // ReplicationTarget identifies a node/store pair. +// +// TODO(aayush): There are a bunch of usages of ReplicaDescriptor in allocator +// methods where we should really be using ReplicationTarget. We should instead +// have something like a `ReplicationTargetI` interface that both +// `ReplicaDescriptor` and `ReplicationTarget` implement and refactor our +// utility methods for these operate on the interface instead. type ReplicationTarget struct { NodeID NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,casttype=NodeID" json:"node_id"` StoreID StoreID `protobuf:"varint,2,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` @@ -178,7 +192,7 @@ type ReplicationTarget struct { func (m *ReplicationTarget) Reset() { *m = ReplicationTarget{} } func (*ReplicationTarget) ProtoMessage() {} func (*ReplicationTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{1} + return fileDescriptor_metadata_9a44b19332750f27, []int{1} } func (m *ReplicationTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -223,7 +237,7 @@ type ReplicaDescriptor struct { func (m *ReplicaDescriptor) Reset() { *m = ReplicaDescriptor{} } func (*ReplicaDescriptor) ProtoMessage() {} func (*ReplicaDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{2} + return fileDescriptor_metadata_9a44b19332750f27, []int{2} } func (m *ReplicaDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -258,7 +272,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_6698bc8ee9dbca95, []int{3} + return fileDescriptor_metadata_9a44b19332750f27, []int{3} } func (m *ReplicaIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -402,7 +416,7 @@ type RangeDescriptor struct { func (m *RangeDescriptor) Reset() { *m = RangeDescriptor{} } func (*RangeDescriptor) ProtoMessage() {} func (*RangeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{4} + return fileDescriptor_metadata_9a44b19332750f27, []int{4} } func (m *RangeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -441,7 +455,7 @@ type Percentiles struct { func (m *Percentiles) Reset() { *m = Percentiles{} } func (*Percentiles) ProtoMessage() {} func (*Percentiles) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{5} + return fileDescriptor_metadata_9a44b19332750f27, []int{5} } func (m *Percentiles) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -503,7 +517,7 @@ type StoreCapacity struct { func (m *StoreCapacity) Reset() { *m = StoreCapacity{} } func (*StoreCapacity) ProtoMessage() {} func (*StoreCapacity) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{6} + return fileDescriptor_metadata_9a44b19332750f27, []int{6} } func (m *StoreCapacity) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -548,7 +562,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_6698bc8ee9dbca95, []int{7} + return fileDescriptor_metadata_9a44b19332750f27, []int{7} } func (m *NodeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -584,7 +598,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_6698bc8ee9dbca95, []int{8} + return fileDescriptor_metadata_9a44b19332750f27, []int{8} } func (m *LocalityAddress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -622,7 +636,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_6698bc8ee9dbca95, []int{9} + return fileDescriptor_metadata_9a44b19332750f27, []int{9} } func (m *StoreDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -659,7 +673,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_6698bc8ee9dbca95, []int{10} + return fileDescriptor_metadata_9a44b19332750f27, []int{10} } func (m *StoreDeadReplicas) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -693,7 +707,7 @@ type Locality struct { func (m *Locality) Reset() { *m = Locality{} } func (*Locality) ProtoMessage() {} func (*Locality) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{11} + return fileDescriptor_metadata_9a44b19332750f27, []int{11} } func (m *Locality) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -729,7 +743,7 @@ type Tier struct { func (m *Tier) Reset() { *m = Tier{} } func (*Tier) ProtoMessage() {} func (*Tier) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{12} + return fileDescriptor_metadata_9a44b19332750f27, []int{12} } func (m *Tier) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -769,7 +783,7 @@ type Version struct { func (m *Version) Reset() { *m = Version{} } func (*Version) ProtoMessage() {} func (*Version) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_6698bc8ee9dbca95, []int{13} + return fileDescriptor_metadata_9a44b19332750f27, []int{13} } func (m *Version) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1675,7 +1689,7 @@ func NewPopulatedReplicaDescriptor(r randyMetadata, easy bool) *ReplicaDescripto this.ReplicaID *= -1 } if r.Intn(10) != 0 { - v1 := ReplicaType([]int32{0, 2, 3, 4, 1, 5}[r.Intn(6)]) + v1 := ReplicaType([]int32{0, 2, 3, 4, 1, 5, 6}[r.Intn(7)]) this.Type = &v1 } if !easy && r.Intn(10) != 0 { @@ -4180,98 +4194,99 @@ var ( ErrIntOverflowMetadata = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_6698bc8ee9dbca95) } +func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_9a44b19332750f27) } -var fileDescriptor_metadata_6698bc8ee9dbca95 = []byte{ - // 1440 bytes of a gzipped FileDescriptorProto +var fileDescriptor_metadata_9a44b19332750f27 = []byte{ + // 1454 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xcf, 0x6f, 0x1b, 0x45, - 0x1b, 0xf6, 0xc6, 0xeb, 0xd8, 0x7e, 0x1d, 0x27, 0xf6, 0xe8, 0xfb, 0x5a, 0xcb, 0x9f, 0x3e, 0xdb, - 0xf5, 0xf7, 0x55, 0xa4, 0x05, 0x25, 0x69, 0x50, 0x54, 0x35, 0x50, 0xa4, 0x38, 0x49, 0x8b, 0x69, - 0xe2, 0x84, 0x8d, 0x5b, 0x24, 0x2e, 0xab, 0xc9, 0xee, 0xe0, 0x2c, 0x5d, 0xef, 0x6e, 0x67, 0xc7, - 0x69, 0x2d, 0xae, 0x08, 0x90, 0x10, 0x12, 0x47, 0x24, 0x2e, 0x95, 0x10, 0x07, 0xfe, 0x03, 0xfe, - 0x84, 0x1e, 0x7b, 0xec, 0x29, 0x82, 0x54, 0x42, 0x9c, 0x39, 0xe6, 0x80, 0xd0, 0xcc, 0xce, 0xec, - 0xae, 0xd3, 0x14, 0xda, 0x22, 0x71, 0x5b, 0x3f, 0xef, 0xf3, 0xbc, 0x7e, 0xe7, 0x9d, 0xf7, 0xc7, - 0xc0, 0x39, 0xea, 0x63, 0xeb, 0x20, 0xd8, 0x5f, 0x1c, 0x12, 0x86, 0x6d, 0xcc, 0xf0, 0x42, 0x40, - 0x7d, 0xe6, 0xa3, 0xaa, 0xe5, 0x5b, 0x77, 0x85, 0x6d, 0x41, 0x32, 0xea, 0xf5, 0x11, 0x73, 0xdc, - 0xc5, 0x91, 0x47, 0x49, 0xe8, 0xbb, 0x87, 0xc4, 0x36, 0xb1, 0x6d, 0xd3, 0x88, 0x5e, 0xaf, 0x09, - 0xdb, 0x81, 0x6b, 0x2d, 0x32, 0x67, 0x48, 0x42, 0x86, 0x87, 0x81, 0xb4, 0xfc, 0x6b, 0xe0, 0x0f, - 0x7c, 0xf1, 0xb9, 0xc8, 0xbf, 0x22, 0xb4, 0xbd, 0x0e, 0xb0, 0xc6, 0x18, 0x75, 0xf6, 0x47, 0x8c, - 0x84, 0xe8, 0x75, 0xc8, 0x61, 0xc6, 0x68, 0x58, 0xd3, 0x5a, 0xd9, 0xf9, 0x62, 0xe7, 0xdf, 0xbf, - 0x1d, 0x35, 0xab, 0x63, 0x3c, 0x74, 0x57, 0xdb, 0x02, 0x7e, 0xe3, 0x23, 0xd7, 0xbf, 0xdf, 0x36, - 0x22, 0xce, 0x6a, 0xe1, 0x9b, 0x87, 0xcd, 0xcc, 0xaf, 0x0f, 0x9b, 0x5a, 0xfb, 0x73, 0x0d, 0xaa, - 0x06, 0x09, 0x5c, 0xc7, 0xc2, 0xcc, 0xf1, 0xbd, 0x3e, 0xa6, 0x03, 0xc2, 0xd0, 0x15, 0xc8, 0x7b, - 0xbe, 0x4d, 0x4c, 0xc7, 0xae, 0x69, 0x2d, 0x6d, 0x3e, 0xd7, 0xa9, 0x3d, 0x3a, 0x6a, 0x66, 0x8e, - 0x8f, 0x9a, 0xd3, 0x3d, 0xdf, 0x26, 0xdd, 0x8d, 0x93, 0xf8, 0xcb, 0x98, 0xe6, 0xc4, 0xae, 0x8d, - 0x56, 0xa0, 0x10, 0x32, 0x9f, 0x0a, 0xcd, 0x94, 0xd0, 0xd4, 0xa5, 0x26, 0xbf, 0xc7, 0x71, 0x21, - 0x52, 0x9f, 0x46, 0x5e, 0x70, 0xbb, 0x76, 0x2a, 0x92, 0xdf, 0x93, 0x48, 0x36, 0x48, 0x68, 0x51, - 0x27, 0x60, 0x3e, 0xfd, 0xe7, 0x22, 0x41, 0xd7, 0x01, 0x68, 0xf4, 0xf7, 0x5c, 0x98, 0x15, 0xc2, - 0x86, 0x14, 0x16, 0x65, 0x60, 0x42, 0x9a, 0xfc, 0x30, 0x8a, 0x52, 0xd1, 0xb5, 0xd1, 0x32, 0xe8, - 0x6c, 0x1c, 0x90, 0x9a, 0xde, 0xd2, 0xe6, 0x67, 0x97, 0x1b, 0x0b, 0xcf, 0xdc, 0xfd, 0x82, 0x94, - 0xf5, 0xc7, 0x01, 0x31, 0x04, 0x77, 0x75, 0x86, 0x1f, 0xfe, 0xc7, 0x87, 0x4d, 0x4d, 0x24, 0xe0, - 0x4b, 0x0d, 0x66, 0x94, 0x6b, 0x9b, 0x78, 0x8c, 0x1f, 0x84, 0x62, 0x6f, 0x10, 0x1f, 0x3e, 0x9b, - 0x1c, 0xc4, 0xe0, 0x78, 0x74, 0x10, 0xf9, 0x69, 0xe4, 0x05, 0xb7, 0x6b, 0xa3, 0x0d, 0xc8, 0xcb, - 0xb0, 0xc4, 0xf1, 0x4b, 0xcb, 0xff, 0x7f, 0x7e, 0x30, 0x49, 0xa6, 0x3b, 0x3a, 0xf7, 0x6d, 0x28, - 0x69, 0xfb, 0x33, 0x1d, 0xe6, 0x84, 0xeb, 0xd4, 0x65, 0xbc, 0x62, 0x40, 0x17, 0xa1, 0x18, 0x32, - 0x4c, 0x99, 0x79, 0x97, 0x8c, 0x45, 0x48, 0x33, 0x9d, 0xc2, 0xc9, 0x51, 0x53, 0x37, 0x6e, 0x91, - 0xb1, 0x51, 0x10, 0xa6, 0x5b, 0x64, 0x8c, 0x2e, 0x40, 0x9e, 0x78, 0xb6, 0x20, 0x65, 0x4f, 0x91, - 0xa6, 0x89, 0x67, 0x73, 0xca, 0x07, 0x50, 0x75, 0x3c, 0x46, 0xa8, 0x87, 0x5d, 0x53, 0x06, 0x1a, - 0xd6, 0xf4, 0x56, 0xf6, 0x25, 0x0f, 0x59, 0x51, 0x4e, 0x24, 0x21, 0x44, 0xef, 0xc1, 0x9c, 0x47, - 0x1e, 0x30, 0x33, 0x55, 0x01, 0x39, 0x51, 0x01, 0x6d, 0x79, 0xc0, 0x72, 0x8f, 0x3c, 0x60, 0xcf, - 0xa9, 0x82, 0xb2, 0x97, 0xb2, 0xd9, 0xe8, 0x2a, 0xc0, 0x80, 0x78, 0x84, 0x8a, 0x86, 0xaa, 0x4d, - 0x8b, 0x3c, 0x9d, 0xe7, 0x6e, 0x4e, 0x8e, 0x9a, 0x51, 0x4a, 0x6f, 0xc6, 0x66, 0x23, 0x45, 0x45, - 0x6f, 0x03, 0x84, 0xcc, 0xb1, 0xee, 0x8e, 0xcd, 0x7d, 0x87, 0xd5, 0xf2, 0xe2, 0xee, 0xfe, 0x9b, - 0x3a, 0x16, 0x9f, 0x0f, 0x0b, 0x07, 0xae, 0xb5, 0xd0, 0x57, 0xf3, 0xc1, 0x28, 0x46, 0x82, 0x8e, - 0xc3, 0xd0, 0xbb, 0xd0, 0xb2, 0x49, 0x40, 0x89, 0x85, 0x19, 0xb1, 0xcd, 0xc4, 0xad, 0x69, 0xf9, - 0xc3, 0x00, 0x53, 0xbc, 0xef, 0x92, 0x5a, 0xa1, 0xa5, 0xcd, 0x17, 0x8c, 0x46, 0xc2, 0x4b, 0x22, - 0x59, 0x8f, 0x59, 0x13, 0x65, 0x99, 0x69, 0x7f, 0xaf, 0x41, 0x69, 0x97, 0x50, 0x8b, 0x78, 0xcc, - 0x71, 0x49, 0x88, 0xce, 0x41, 0x36, 0xb8, 0xb2, 0x24, 0xee, 0x5f, 0x93, 0xf9, 0xe4, 0x80, 0xc0, - 0x97, 0x57, 0xc4, 0xfd, 0x26, 0xf8, 0xf2, 0x8a, 0xc0, 0x57, 0x96, 0xc4, 0x95, 0x26, 0xf8, 0x4a, - 0xc4, 0xbf, 0xba, 0x22, 0xfa, 0x25, 0xc1, 0xaf, 0x46, 0xfc, 0x6b, 0x4b, 0x22, 0xfd, 0x09, 0x7e, - 0x6d, 0x09, 0xd5, 0x40, 0x0f, 0xb6, 0xf1, 0x03, 0x91, 0x50, 0x65, 0x10, 0xc8, 0xaa, 0xce, 0xe3, - 0x6d, 0x9f, 0x64, 0xa1, 0x2c, 0x9a, 0x7a, 0x1d, 0x07, 0xd8, 0x72, 0xd8, 0x18, 0xb5, 0xa0, 0x60, - 0xc9, 0x6f, 0x59, 0xae, 0x91, 0x2a, 0x46, 0x51, 0x1b, 0x8a, 0xf8, 0x10, 0x3b, 0xae, 0x48, 0xce, - 0x54, 0x8a, 0x92, 0xc0, 0xe8, 0x22, 0x94, 0xa2, 0xa2, 0xb7, 0xfc, 0x91, 0xc7, 0xe4, 0x60, 0x88, - 0x58, 0x20, 0x0c, 0xeb, 0x1c, 0xe7, 0x34, 0x97, 0xe0, 0x50, 0xd1, 0xf4, 0x34, 0x4d, 0x18, 0x22, - 0xda, 0x12, 0x54, 0xef, 0x53, 0x87, 0x91, 0xd0, 0x0c, 0x08, 0x35, 0x43, 0x62, 0xf9, 0x9e, 0x3d, - 0x71, 0xd6, 0xb9, 0xc8, 0xbc, 0x4b, 0xe8, 0x9e, 0x30, 0xa2, 0x5d, 0xa8, 0xee, 0x8f, 0x95, 0x40, - 0x35, 0xf6, 0xb4, 0x28, 0x8e, 0xb3, 0xa6, 0x4c, 0xea, 0xaa, 0x94, 0x47, 0x21, 0xdf, 0x25, 0x54, - 0x96, 0x28, 0x32, 0x00, 0xa5, 0x62, 0x50, 0x2e, 0xf3, 0x2f, 0xe1, 0xb2, 0x12, 0x07, 0xa9, 0x7c, - 0xd6, 0x40, 0x1f, 0x85, 0xc4, 0x16, 0x15, 0xa6, 0x92, 0x28, 0x10, 0x74, 0x09, 0xca, 0xae, 0x3f, - 0x70, 0x2c, 0xec, 0x9a, 0x22, 0x90, 0x5a, 0x31, 0x45, 0x99, 0x91, 0xa6, 0x0e, 0xb7, 0xa0, 0x65, - 0x40, 0xf7, 0x46, 0x84, 0x3a, 0x93, 0xd9, 0x81, 0x54, 0x76, 0x2a, 0xd2, 0x1e, 0xa7, 0x47, 0x5e, - 0xfe, 0x2f, 0x3a, 0xcc, 0xf2, 0x35, 0xf0, 0xf7, 0x36, 0xc7, 0x3b, 0x90, 0xe7, 0xfb, 0x98, 0x84, - 0xa1, 0x9c, 0x9c, 0x8d, 0xd3, 0xdd, 0x77, 0x3b, 0xde, 0xdc, 0x6b, 0xb6, 0x1d, 0xcf, 0x4c, 0x29, - 0x42, 0xd7, 0xd4, 0x0e, 0xce, 0x3e, 0xd3, 0xbb, 0x2a, 0x97, 0xc9, 0xc6, 0x96, 0xe2, 0x48, 0x81, - 0xae, 0x43, 0xc1, 0xf5, 0x2d, 0xec, 0xf2, 0x5a, 0xd5, 0x85, 0xfa, 0x3f, 0x67, 0xa8, 0xb7, 0x24, - 0x45, 0x15, 0xb2, 0x92, 0xa0, 0x1b, 0x50, 0xde, 0x23, 0xf4, 0x90, 0xd0, 0x3b, 0x84, 0x86, 0x7c, - 0xec, 0xe4, 0x84, 0x8f, 0xfa, 0x19, 0x3e, 0x24, 0x43, 0xba, 0x98, 0x94, 0xa1, 0x0b, 0x50, 0xdc, - 0x1f, 0x39, 0xae, 0x6d, 0x32, 0x3c, 0x10, 0x45, 0x56, 0x54, 0x7f, 0x25, 0xe0, 0x3e, 0x1e, 0xa0, - 0xff, 0xf1, 0x29, 0x85, 0x29, 0x1f, 0x32, 0x38, 0x9a, 0x52, 0x71, 0xd3, 0x48, 0x7c, 0x8d, 0xa1, - 0x3d, 0xa8, 0xa8, 0xd8, 0x4c, 0x95, 0xd2, 0x82, 0x98, 0xd3, 0xed, 0x3f, 0x39, 0xd6, 0x5a, 0xc4, - 0x54, 0x75, 0xeb, 0x4e, 0xc2, 0xe8, 0x35, 0x98, 0xb1, 0xdc, 0x51, 0xc8, 0x08, 0x35, 0x3d, 0x3c, - 0x24, 0xa2, 0x90, 0x54, 0x7c, 0x25, 0x69, 0xe9, 0xe1, 0x21, 0x41, 0x7b, 0x50, 0x0a, 0xef, 0xb9, - 0xf1, 0x1f, 0xc3, 0x0b, 0xdd, 0x25, 0x92, 0xe5, 0x01, 0x7b, 0xef, 0x6f, 0xc9, 0x7f, 0x34, 0x20, - 0xbc, 0xe7, 0xca, 0xef, 0x55, 0x5d, 0x2c, 0xe9, 0x6f, 0x35, 0x98, 0x3b, 0x15, 0x6e, 0xba, 0x6c, - 0xb4, 0x57, 0x29, 0x9b, 0x0e, 0xef, 0x10, 0x99, 0x2c, 0xe6, 0x10, 0x2a, 0x8b, 0xef, 0xfc, 0x19, - 0x99, 0xea, 0x3b, 0x84, 0x26, 0xad, 0x13, 0x69, 0x38, 0x26, 0xa3, 0xfb, 0x74, 0x0a, 0xe6, 0xc4, - 0x0c, 0x9c, 0x5c, 0xda, 0xf1, 0x73, 0x48, 0x7b, 0xf1, 0xe7, 0x50, 0x5c, 0xcb, 0x53, 0x2f, 0x5d, - 0xcb, 0x6f, 0x81, 0xce, 0x1b, 0x4a, 0x76, 0xc1, 0x85, 0x33, 0x94, 0x93, 0xad, 0xaa, 0xc6, 0x05, - 0x17, 0xa1, 0x4e, 0x6a, 0x68, 0x47, 0x8d, 0xd0, 0x3a, 0xc3, 0xc1, 0xc4, 0xa0, 0x3f, 0x3d, 0xd6, - 0xdb, 0x5f, 0x69, 0x50, 0x95, 0x69, 0xc0, 0x76, 0xbc, 0xe3, 0x5f, 0x31, 0x11, 0x6b, 0x50, 0x88, - 0x9f, 0x1a, 0x53, 0xa2, 0x84, 0x9b, 0xcf, 0x7f, 0x6a, 0x88, 0x87, 0x9b, 0x8a, 0x47, 0xc9, 0xda, - 0x5d, 0x28, 0xa8, 0x9a, 0x41, 0x6f, 0x42, 0x8e, 0xdf, 0x71, 0xf4, 0x4e, 0xff, 0xcb, 0x4b, 0x8e, - 0xb8, 0xa9, 0x57, 0xf2, 0x06, 0xe8, 0xdc, 0xcc, 0xb7, 0x24, 0x7f, 0x28, 0x69, 0xa9, 0x16, 0xe0, - 0x00, 0xaa, 0x43, 0xee, 0x10, 0xbb, 0xa3, 0x68, 0x9b, 0x29, 0x4b, 0x04, 0xa5, 0xbc, 0xfc, 0xa0, - 0x41, 0x5e, 0xb5, 0xfc, 0x65, 0x28, 0x0e, 0xf1, 0xc7, 0x3e, 0x35, 0x0f, 0xb1, 0x2b, 0xf3, 0x52, - 0x96, 0x79, 0xc9, 0x6d, 0x73, 0x83, 0x51, 0x10, 0xf6, 0x3b, 0xd8, 0x15, 0x5c, 0xc7, 0x93, 0xdc, - 0xa9, 0x53, 0x5c, 0x6e, 0x30, 0x0a, 0xc2, 0xce, 0xb9, 0x75, 0xc8, 0x05, 0x98, 0x59, 0x07, 0x13, - 0x1b, 0x33, 0x82, 0xf8, 0x66, 0x56, 0x4f, 0xb0, 0x89, 0x4d, 0x19, 0xa3, 0x49, 0xac, 0x97, 0x3f, - 0x81, 0x52, 0xea, 0xe5, 0x8c, 0x66, 0x01, 0xee, 0xec, 0xf4, 0x37, 0x0d, 0xf3, 0xc6, 0xed, 0xad, - 0xad, 0x4a, 0x06, 0x21, 0x98, 0x8d, 0x7e, 0x77, 0x7b, 0xeb, 0x3b, 0xdb, 0xdd, 0xde, 0xcd, 0xca, - 0x54, 0x82, 0xed, 0xdc, 0xee, 0xdf, 0xdc, 0xe1, 0x58, 0x36, 0xc1, 0x36, 0x36, 0xb7, 0x77, 0xfa, - 0x1c, 0xd3, 0x51, 0x09, 0xf2, 0x5b, 0x9b, 0x6b, 0x46, 0x6f, 0xd3, 0xa8, 0x68, 0xa8, 0x0c, 0xc5, - 0xde, 0x4e, 0xcf, 0x14, 0xa4, 0x4a, 0xae, 0xae, 0x7f, 0xf1, 0x5d, 0x23, 0xd3, 0xb9, 0xf4, 0xe8, - 0xe7, 0x46, 0xe6, 0xd1, 0x71, 0x43, 0x7b, 0x7c, 0xdc, 0xd0, 0x9e, 0x1c, 0x37, 0xb4, 0x9f, 0x8e, - 0x1b, 0xda, 0xd7, 0x4f, 0x1b, 0x99, 0xc7, 0x4f, 0x1b, 0x99, 0x27, 0x4f, 0x1b, 0x99, 0x0f, 0xf3, - 0xf2, 0xd6, 0xfe, 0x08, 0x00, 0x00, 0xff, 0xff, 0x31, 0x39, 0x15, 0xc6, 0x06, 0x0e, 0x00, 0x00, + 0x1b, 0xf6, 0xc6, 0xeb, 0xd8, 0x7e, 0x13, 0x27, 0xf6, 0xe8, 0xfb, 0x5a, 0xcb, 0xdf, 0x87, 0xed, + 0x1a, 0x2a, 0xd2, 0x82, 0x92, 0x34, 0x28, 0xaa, 0x1a, 0x28, 0x52, 0x9c, 0xa4, 0xc1, 0x34, 0x71, + 0xc2, 0xc6, 0x2d, 0x12, 0x97, 0xd5, 0x64, 0x77, 0x70, 0x96, 0xae, 0x77, 0xb7, 0xb3, 0xe3, 0xb4, + 0xbe, 0x23, 0x40, 0x42, 0x48, 0x1c, 0x91, 0xb8, 0x14, 0x21, 0x0e, 0xfc, 0x07, 0xfc, 0x09, 0x3d, + 0xf6, 0xd8, 0x53, 0x04, 0xa9, 0x84, 0x38, 0x73, 0xcc, 0x01, 0xa1, 0x99, 0x9d, 0xd9, 0x5d, 0xa7, + 0x29, 0xb4, 0x45, 0xe2, 0xb6, 0x7e, 0xde, 0xe7, 0x79, 0xfd, 0xce, 0x3b, 0xef, 0x8f, 0x81, 0x73, + 0xd4, 0xc7, 0xd6, 0x41, 0xb0, 0xbf, 0x30, 0x20, 0x0c, 0xdb, 0x98, 0xe1, 0xf9, 0x80, 0xfa, 0xcc, + 0x47, 0x15, 0xcb, 0xb7, 0xee, 0x08, 0xdb, 0xbc, 0x64, 0xd4, 0x6a, 0x43, 0xe6, 0xb8, 0x0b, 0x43, + 0x8f, 0x92, 0xd0, 0x77, 0x0f, 0x89, 0x6d, 0x62, 0xdb, 0xa6, 0x11, 0xbd, 0x56, 0x15, 0xb6, 0x03, + 0xd7, 0x5a, 0x60, 0xce, 0x80, 0x84, 0x0c, 0x0f, 0x02, 0x69, 0xf9, 0x4f, 0xdf, 0xef, 0xfb, 0xe2, + 0x73, 0x81, 0x7f, 0x45, 0x68, 0x6b, 0x0d, 0x60, 0x95, 0x31, 0xea, 0xec, 0x0f, 0x19, 0x09, 0xd1, + 0x1b, 0x90, 0xc3, 0x8c, 0xd1, 0xb0, 0xaa, 0x35, 0xb3, 0x73, 0xc5, 0xf6, 0x7f, 0x7f, 0x3f, 0x6a, + 0x54, 0x46, 0x78, 0xe0, 0xae, 0xb4, 0x04, 0xfc, 0xe6, 0xc7, 0xae, 0x7f, 0xaf, 0x65, 0x44, 0x9c, + 0x95, 0xc2, 0x37, 0x0f, 0x1a, 0x99, 0xdf, 0x1e, 0x34, 0xb4, 0xd6, 0xe7, 0x1a, 0x54, 0x0c, 0x12, + 0xb8, 0x8e, 0x85, 0x99, 0xe3, 0x7b, 0x3d, 0x4c, 0xfb, 0x84, 0xa1, 0x2b, 0x90, 0xf7, 0x7c, 0x9b, + 0x98, 0x8e, 0x5d, 0xd5, 0x9a, 0xda, 0x5c, 0xae, 0x5d, 0x7d, 0x78, 0xd4, 0xc8, 0x1c, 0x1f, 0x35, + 0x26, 0xbb, 0xbe, 0x4d, 0x3a, 0xeb, 0x27, 0xf1, 0x97, 0x31, 0xc9, 0x89, 0x1d, 0x1b, 0x2d, 0x43, + 0x21, 0x64, 0x3e, 0x15, 0x9a, 0x09, 0xa1, 0xa9, 0x49, 0x4d, 0x7e, 0x8f, 0xe3, 0x42, 0xa4, 0x3e, + 0x8d, 0xbc, 0xe0, 0x76, 0xec, 0x54, 0x24, 0x7f, 0x24, 0x91, 0xac, 0x93, 0xd0, 0xa2, 0x4e, 0xc0, + 0x7c, 0xfa, 0xef, 0x45, 0x82, 0xae, 0x03, 0xd0, 0xe8, 0xef, 0xb9, 0x30, 0x2b, 0x84, 0x75, 0x29, + 0x2c, 0xca, 0xc0, 0x84, 0x34, 0xf9, 0x61, 0x14, 0xa5, 0xa2, 0x63, 0xa3, 0x25, 0xd0, 0xd9, 0x28, + 0x20, 0x55, 0xbd, 0xa9, 0xcd, 0xcd, 0x2c, 0xd5, 0xe7, 0x9f, 0xba, 0xfb, 0x79, 0x29, 0xeb, 0x8d, + 0x02, 0x62, 0x08, 0xee, 0xca, 0x34, 0x3f, 0xfc, 0x4f, 0x0f, 0x1a, 0x9a, 0x48, 0xc0, 0x97, 0x1a, + 0x4c, 0x2b, 0xd7, 0x36, 0xf1, 0x18, 0x3f, 0x08, 0xc5, 0x5e, 0x3f, 0x3e, 0x7c, 0x36, 0x39, 0x88, + 0xc1, 0xf1, 0xe8, 0x20, 0xf2, 0xd3, 0xc8, 0x0b, 0x6e, 0xc7, 0x46, 0xeb, 0x90, 0x97, 0x61, 0x89, + 0xe3, 0x4f, 0x2d, 0xbd, 0xf6, 0xec, 0x60, 0x92, 0x4c, 0xb7, 0x75, 0xee, 0xdb, 0x50, 0xd2, 0xd6, + 0x67, 0x3a, 0xcc, 0x0a, 0xd7, 0xa9, 0xcb, 0x78, 0xc9, 0x80, 0x2e, 0x42, 0x31, 0x64, 0x98, 0x32, + 0xf3, 0x0e, 0x19, 0x89, 0x90, 0xa6, 0xdb, 0x85, 0x93, 0xa3, 0x86, 0x6e, 0xdc, 0x24, 0x23, 0xa3, + 0x20, 0x4c, 0x37, 0xc9, 0x08, 0x5d, 0x80, 0x3c, 0xf1, 0x6c, 0x41, 0xca, 0x9e, 0x22, 0x4d, 0x12, + 0xcf, 0xe6, 0x94, 0x0f, 0xa1, 0xe2, 0x78, 0x8c, 0x50, 0x0f, 0xbb, 0xa6, 0x0c, 0x34, 0xac, 0xea, + 0xcd, 0xec, 0x0b, 0x1e, 0xb2, 0xac, 0x9c, 0x48, 0x42, 0x88, 0xde, 0x87, 0x59, 0x8f, 0xdc, 0x67, + 0x66, 0xaa, 0x02, 0x72, 0xa2, 0x02, 0x5a, 0xf2, 0x80, 0xa5, 0x2e, 0xb9, 0xcf, 0x9e, 0x51, 0x05, + 0x25, 0x2f, 0x65, 0xb3, 0xd1, 0x55, 0x80, 0x3e, 0xf1, 0x08, 0x15, 0x0d, 0x55, 0x9d, 0x14, 0x79, + 0x3a, 0xcf, 0xdd, 0x9c, 0x1c, 0x35, 0xa2, 0x94, 0x6e, 0xc6, 0x66, 0x23, 0x45, 0x45, 0xef, 0x00, + 0x84, 0xcc, 0xb1, 0xee, 0x8c, 0xcc, 0x7d, 0x87, 0x55, 0xf3, 0xe2, 0xee, 0x5e, 0x49, 0x1d, 0x8b, + 0xcf, 0x87, 0xf9, 0x03, 0xd7, 0x9a, 0xef, 0xa9, 0xf9, 0x60, 0x14, 0x23, 0x41, 0xdb, 0x61, 0xe8, + 0x3d, 0x68, 0xda, 0x24, 0xa0, 0xc4, 0xc2, 0x8c, 0xd8, 0x66, 0xe2, 0xd6, 0xb4, 0xfc, 0x41, 0x80, + 0x29, 0xde, 0x77, 0x49, 0xb5, 0xd0, 0xd4, 0xe6, 0x0a, 0x46, 0x3d, 0xe1, 0x25, 0x91, 0xac, 0xc5, + 0xac, 0xb1, 0xb2, 0xcc, 0xb4, 0x7e, 0xd0, 0x60, 0x6a, 0x97, 0x50, 0x8b, 0x78, 0xcc, 0x71, 0x49, + 0x88, 0xce, 0x41, 0x36, 0xb8, 0xb2, 0x28, 0xee, 0x5f, 0x93, 0xf9, 0xe4, 0x80, 0xc0, 0x97, 0x96, + 0xc5, 0xfd, 0x26, 0xf8, 0xd2, 0xb2, 0xc0, 0x97, 0x17, 0xc5, 0x95, 0x26, 0xf8, 0x72, 0xc4, 0xbf, + 0xba, 0x2c, 0xfa, 0x25, 0xc1, 0xaf, 0x46, 0xfc, 0x6b, 0x8b, 0x22, 0xfd, 0x09, 0x7e, 0x6d, 0x11, + 0x55, 0x41, 0x0f, 0xb6, 0xf1, 0x7d, 0x91, 0x50, 0x65, 0x10, 0xc8, 0x8a, 0xce, 0xe3, 0x6d, 0x9d, + 0x64, 0xa1, 0x24, 0x9a, 0x7a, 0x0d, 0x07, 0xd8, 0x72, 0xd8, 0x08, 0x35, 0xa1, 0x60, 0xc9, 0x6f, + 0x59, 0xae, 0x91, 0x2a, 0x46, 0x51, 0x0b, 0x8a, 0xf8, 0x10, 0x3b, 0xae, 0x48, 0xce, 0x44, 0x8a, + 0x92, 0xc0, 0xe8, 0x22, 0x4c, 0x45, 0x45, 0x6f, 0xf9, 0x43, 0x8f, 0xc9, 0xc1, 0x10, 0xb1, 0x40, + 0x18, 0xd6, 0x38, 0xce, 0x69, 0x2e, 0xc1, 0xa1, 0xa2, 0xe9, 0x69, 0x9a, 0x30, 0x44, 0xb4, 0x45, + 0xa8, 0xdc, 0xa3, 0x0e, 0x23, 0xa1, 0x19, 0x10, 0x6a, 0x86, 0xc4, 0xf2, 0x3d, 0x7b, 0xec, 0xac, + 0xb3, 0x91, 0x79, 0x97, 0xd0, 0x3d, 0x61, 0x44, 0xbb, 0x50, 0xd9, 0x1f, 0x29, 0x81, 0x6a, 0xec, + 0x49, 0x51, 0x1c, 0x67, 0x4d, 0x99, 0xd4, 0x55, 0x29, 0x8f, 0x42, 0xbe, 0x4b, 0xa8, 0x2c, 0x51, + 0x64, 0x00, 0x4a, 0xc5, 0xa0, 0x5c, 0xe6, 0x5f, 0xc0, 0x65, 0x39, 0x0e, 0x52, 0xf9, 0xac, 0x82, + 0x3e, 0x0c, 0x89, 0x2d, 0x2a, 0x4c, 0x25, 0x51, 0x20, 0xe8, 0x12, 0x94, 0x5c, 0xbf, 0xef, 0x58, + 0xd8, 0x35, 0x45, 0x20, 0xd5, 0x62, 0x8a, 0x32, 0x2d, 0x4d, 0x6d, 0x6e, 0x41, 0x4b, 0x80, 0xee, + 0x0e, 0x09, 0x75, 0xc6, 0xb3, 0x03, 0xa9, 0xec, 0x94, 0xa5, 0x3d, 0x4e, 0x8f, 0xbc, 0xfc, 0x5f, + 0x75, 0x98, 0xe1, 0x6b, 0xe0, 0x9f, 0x6d, 0x8e, 0x77, 0x21, 0xcf, 0xf7, 0x31, 0x09, 0x43, 0x39, + 0x39, 0xeb, 0xa7, 0xbb, 0xef, 0x56, 0xbc, 0xb9, 0x57, 0x6d, 0x3b, 0x9e, 0x99, 0x52, 0x84, 0xae, + 0xa9, 0x1d, 0x9c, 0x7d, 0xaa, 0x77, 0x55, 0x2e, 0x93, 0x8d, 0x2d, 0xc5, 0x91, 0x02, 0x5d, 0x87, + 0x82, 0xeb, 0x5b, 0xd8, 0xe5, 0xb5, 0xaa, 0x0b, 0xf5, 0xff, 0xce, 0x50, 0x6f, 0x49, 0x8a, 0x2a, + 0x64, 0x25, 0x41, 0x37, 0xa0, 0xb4, 0x47, 0xe8, 0x21, 0xa1, 0xb7, 0x09, 0x0d, 0xf9, 0xd8, 0xc9, + 0x09, 0x1f, 0xb5, 0x33, 0x7c, 0x48, 0x86, 0x74, 0x31, 0x2e, 0x43, 0x17, 0xa0, 0xb8, 0x3f, 0x74, + 0x5c, 0xdb, 0x64, 0xb8, 0x2f, 0x8a, 0xac, 0xa8, 0xfe, 0x4a, 0xc0, 0x3d, 0xdc, 0x47, 0xaf, 0xf2, + 0x29, 0x85, 0x29, 0x1f, 0x32, 0x38, 0x9a, 0x52, 0x71, 0xd3, 0x48, 0x7c, 0x95, 0xa1, 0x3d, 0x28, + 0xab, 0xd8, 0x4c, 0x95, 0xd2, 0x82, 0x98, 0xd3, 0xad, 0xbf, 0x38, 0xd6, 0x6a, 0xc4, 0x54, 0x75, + 0xeb, 0x8e, 0xc3, 0xe8, 0x75, 0x98, 0xb6, 0xdc, 0x61, 0xc8, 0x08, 0x35, 0x3d, 0x3c, 0x20, 0xa2, + 0x90, 0x54, 0x7c, 0x53, 0xd2, 0xd2, 0xc5, 0x03, 0x82, 0xf6, 0x60, 0x2a, 0xbc, 0xeb, 0xc6, 0x7f, + 0x0c, 0xcf, 0x75, 0x97, 0x48, 0x96, 0x07, 0xec, 0x7d, 0xb0, 0x25, 0xff, 0xd1, 0x80, 0xf0, 0xae, + 0x2b, 0xbf, 0x57, 0x74, 0xb1, 0xa4, 0xbf, 0xd5, 0x60, 0xf6, 0x54, 0xb8, 0xe9, 0xb2, 0xd1, 0x5e, + 0xa6, 0x6c, 0xda, 0xbc, 0x43, 0x64, 0xb2, 0x98, 0x43, 0xa8, 0x2c, 0xbe, 0xf3, 0x67, 0x64, 0xaa, + 0xe7, 0x10, 0x9a, 0xb4, 0x4e, 0xa4, 0xe1, 0x98, 0x8c, 0xee, 0xd3, 0x09, 0x98, 0x15, 0x33, 0x70, + 0x7c, 0x69, 0xc7, 0xcf, 0x21, 0xed, 0xf9, 0x9f, 0x43, 0x71, 0x2d, 0x4f, 0xbc, 0x70, 0x2d, 0xbf, + 0x0d, 0x3a, 0x6f, 0x28, 0xd9, 0x05, 0x17, 0xce, 0x50, 0x8e, 0xb7, 0xaa, 0x1a, 0x17, 0x5c, 0x84, + 0xda, 0xa9, 0xa1, 0x1d, 0x35, 0x42, 0xf3, 0x0c, 0x07, 0x63, 0x83, 0xfe, 0xf4, 0x58, 0x6f, 0x7d, + 0xa5, 0x41, 0x45, 0xa6, 0x01, 0xdb, 0xf1, 0x8e, 0x7f, 0xc9, 0x44, 0xac, 0x42, 0x21, 0x7e, 0x6a, + 0x4c, 0x88, 0x12, 0x6e, 0x3c, 0xfb, 0xa9, 0x21, 0x1e, 0x6e, 0x2a, 0x1e, 0x25, 0x6b, 0x75, 0xa0, + 0xa0, 0x6a, 0x06, 0xbd, 0x05, 0x39, 0x7e, 0xc7, 0xd1, 0x3b, 0xfd, 0x6f, 0x2f, 0x39, 0xe2, 0xa6, + 0x5e, 0xc9, 0xeb, 0xa0, 0x73, 0x33, 0xdf, 0x92, 0xfc, 0xa1, 0xa4, 0xa5, 0x5a, 0x80, 0x03, 0xa8, + 0x06, 0xb9, 0x43, 0xec, 0x0e, 0xa3, 0x6d, 0xa6, 0x2c, 0x11, 0x94, 0xf2, 0xf2, 0xa3, 0x06, 0x79, + 0xd5, 0xf2, 0x97, 0xa1, 0x38, 0xc0, 0x9f, 0xf8, 0xd4, 0x3c, 0xc4, 0xae, 0xcc, 0x4b, 0x49, 0xe6, + 0x25, 0xb7, 0xcd, 0x0d, 0x46, 0x41, 0xd8, 0x6f, 0x63, 0x57, 0x70, 0x1d, 0x4f, 0x72, 0x27, 0x4e, + 0x71, 0xb9, 0xc1, 0x28, 0x08, 0x3b, 0xe7, 0xd6, 0x20, 0x17, 0x60, 0x66, 0x1d, 0x8c, 0x6d, 0xcc, + 0x08, 0xe2, 0x9b, 0x59, 0x3d, 0xc1, 0xc6, 0x36, 0x65, 0x8c, 0x26, 0xb1, 0x5e, 0xfe, 0x4e, 0x83, + 0xa9, 0xd4, 0xd3, 0x19, 0xcd, 0x00, 0xdc, 0xde, 0xe9, 0x6d, 0x18, 0xe6, 0x8d, 0x5b, 0x5b, 0x5b, + 0xe5, 0x0c, 0x42, 0x30, 0x13, 0xfd, 0xee, 0x74, 0xd7, 0x76, 0xb6, 0x3b, 0xdd, 0xcd, 0xf2, 0x44, + 0x82, 0xed, 0xdc, 0xea, 0x6d, 0xee, 0x70, 0x2c, 0x8b, 0x6a, 0x70, 0x2e, 0xc2, 0xd6, 0x37, 0xb6, + 0x77, 0x7a, 0x9d, 0xee, 0xa6, 0xb9, 0xb5, 0xb1, 0x6a, 0x74, 0x37, 0x8c, 0xb2, 0x8e, 0xa6, 0x20, + 0xaf, 0x7e, 0x68, 0xa8, 0x04, 0xc5, 0xee, 0x4e, 0xd7, 0x14, 0xe4, 0x72, 0x0e, 0xfd, 0x1f, 0xaa, + 0xa7, 0x74, 0x89, 0x75, 0xb2, 0xa6, 0x7f, 0xf1, 0x7d, 0x3d, 0xd3, 0xbe, 0xf4, 0xf0, 0x97, 0x7a, + 0xe6, 0xe1, 0x71, 0x5d, 0x7b, 0x74, 0x5c, 0xd7, 0x1e, 0x1f, 0xd7, 0xb5, 0x9f, 0x8f, 0xeb, 0xda, + 0xd7, 0x4f, 0xea, 0x99, 0x47, 0x4f, 0xea, 0x99, 0xc7, 0x4f, 0xea, 0x99, 0x8f, 0xf2, 0xf2, 0x72, + 0xff, 0x0c, 0x00, 0x00, 0xff, 0xff, 0x8c, 0x71, 0x3d, 0xa2, 0x2d, 0x0e, 0x00, 0x00, } diff --git a/pkg/roachpb/metadata.proto b/pkg/roachpb/metadata.proto index 31ec796673a9..20a32cc6215e 100644 --- a/pkg/roachpb/metadata.proto +++ b/pkg/roachpb/metadata.proto @@ -26,6 +26,12 @@ message Attributes { } // ReplicationTarget identifies a node/store pair. +// +// TODO(aayush): There are a bunch of usages of ReplicaDescriptor in allocator +// methods where we should really be using ReplicationTarget. We should instead +// have something like a `ReplicationTargetI` interface that both +// `ReplicaDescriptor` and `ReplicationTarget` implement and refactor our +// utility methods for these operate on the interface instead. message ReplicationTarget { option (gogoproto.goproto_stringer) = false; option (gogoproto.equal) = true; @@ -37,25 +43,27 @@ message ReplicationTarget { } // 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_DEMOTING and VOTER_INCOMING types are used as -// well to denote voters which are being downgraded to learners and newly added -// by the change, respectively. A demoting voter is turning into a learner, -// which we prefer over a direct removal, which was used prior to v20.1 and -// uses the VOTER_OUTGOING type instead (see #42251). +// normal operation, VOTER_FULL, NON_VOTER, and LEARNER are the only used +// states. However, atomic replication changes require a transition through a +// "joint config"; in this joint config, the VOTER_DEMOTING_{LEARNER, NON_VOTER} +// and VOTER_INCOMING types are used as well to denote voters which are being +// downgraded to learners and newly added by the change, respectively. When +// being removed, a demoting voter is turning into a learner, which we prefer +// over a direct removal, which was used prior to v20.1 and uses the +// VOTER_OUTGOING type instead (see VersionChangeReplicasDemotion for details on +// why we're not doing that any more). // -// All voter types indicate a replica that participates in all raft activities, +// 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 a joint config, two // separate majorities are required: one from the set of replicas that have -// either type VOTER or VOTER_OUTOING or VOTER_DEMOTING, 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. +// either type VOTER or VOTER_OUTGOING or VOTER_DEMOTING_{LEARNER, NON_VOTER}, +// 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 { option (gogoproto.goproto_enum_prefix) = false; @@ -76,19 +84,19 @@ enum ReplicaType { // Note: We're not using VOTER_OUTGOING since 20.1. We're using VOTER_DEMOTING // instead. See #42251. VOTER_OUTGOING = 3; - // VOTER_DEMOTING indicates a voting replica that will become a learner once - // the ongoing atomic replication change is finalized; that is, it is in the - // process of being demoted. Since learners are currently short-lived, this - // replica is really being removed, with an intermediate step, and no work - // should be assigned to it. - VOTER_DEMOTING = 4; + // VOTER_DEMOTING_LEARNER indicates a voting replica that will become a + // learner once the ongoing atomic replication change is finalized; that is, + // it is in the process of being demoted. Since learners are currently + // short-lived, this replica is really being removed, with an intermediate + // step, and no work should be assigned to it. + VOTER_DEMOTING_LEARNER = 4; // LEARNER indicates a replica that applies committed entries, but does not // count towards the quorum(s). Candidates will not ask for (or take into // account) votes of (peers they consider) LEARNERs for leadership nor do // their acknowledged log entries get taken into account for determining the // committed index. Learners in CockroachDB are a short-term transient state: // a replica being added and on its way to being a VOTER_{FULL,INCOMING}, or a - // VOTER_DEMOTING being removed. + // VOTER_DEMOTING_LEARNER being removed. // // Note that once these replicas upreplicate after receiving their initial // snapshot, they will count towards the raft leader's quota pool and throttle @@ -108,6 +116,10 @@ enum ReplicaType { // via follower reads. See comment above ReplicaDescriptors.NonVoters() for // differences in how LEARNERs and NON_VOTERs are handled internally. NON_VOTER = 5; + // VOTER_DEMOTING_NON_VOTER indicates a voting replica in the outgoing group + // of a joint state, which will become a non-voter when the atomic replication + // change is finalized (i.e. when we exit the joint state). + VOTER_DEMOTING_NON_VOTER = 6; } // ReplicaDescriptor describes a replica location by node ID diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index fa11c8f95111..cae6035c4c2d 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -39,10 +39,10 @@ func ReplicaTypeVoterOutgoing() *ReplicaType { return &t } -// ReplicaTypeVoterDemoting returns a VOTER_DEMOTING pointer suitable -// for use in a nullable proto field. -func ReplicaTypeVoterDemoting() *ReplicaType { - t := VOTER_DEMOTING +// ReplicaTypeVoterDemotingLearner returns a VOTER_DEMOTING_LEARNER pointer +// suitable for use in a nullable proto field. +func ReplicaTypeVoterDemotingLearner() *ReplicaType { + t := VOTER_DEMOTING_LEARNER return &t } @@ -376,7 +376,8 @@ func (d *ReplicaSet) RemoveReplica(nodeID NodeID, storeID StoreID) (ReplicaDescr func (d ReplicaSet) InAtomicReplicationChange() bool { for _, rDesc := range d.wrapped { switch rDesc.GetType() { - case VOTER_INCOMING, VOTER_OUTGOING, VOTER_DEMOTING: + case VOTER_INCOMING, VOTER_OUTGOING, VOTER_DEMOTING_LEARNER, + VOTER_DEMOTING_NON_VOTER: return true case VOTER_FULL, LEARNER, NON_VOTER: default: @@ -406,7 +407,7 @@ func (d ReplicaSet) ConfState() raftpb.ConfState { cs.Voters = append(cs.Voters, id) case VOTER_OUTGOING: cs.VotersOutgoing = append(cs.VotersOutgoing, id) - case VOTER_DEMOTING: + case VOTER_DEMOTING_LEARNER, VOTER_DEMOTING_NON_VOTER: cs.VotersOutgoing = append(cs.VotersOutgoing, id) cs.LearnersNext = append(cs.LearnersNext, id) case LEARNER: @@ -426,7 +427,7 @@ func (d ReplicaSet) ConfState() raftpb.ConfState { func (d ReplicaSet) CanMakeProgress(liveFunc func(descriptor ReplicaDescriptor) bool) bool { isVoterOldConfig := func(rDesc ReplicaDescriptor) bool { switch rDesc.GetType() { - case VOTER_FULL, VOTER_OUTGOING, VOTER_DEMOTING: + case VOTER_FULL, VOTER_OUTGOING, VOTER_DEMOTING_LEARNER, VOTER_DEMOTING_NON_VOTER: return true default: return false diff --git a/pkg/roachpb/metadata_replicas_test.go b/pkg/roachpb/metadata_replicas_test.go index 8fac0c4dd3ff..541419f5ea44 100644 --- a/pkg/roachpb/metadata_replicas_test.go +++ b/pkg/roachpb/metadata_replicas_test.go @@ -41,7 +41,7 @@ var vn = (*ReplicaType)(nil) // should be treated like VoterFull var v = ReplicaTypeVoterFull() var vi = ReplicaTypeVoterIncoming() var vo = ReplicaTypeVoterOutgoing() -var vd = ReplicaTypeVoterDemoting() +var vd = ReplicaTypeVoterDemotingLearner() var l = ReplicaTypeLearner() func TestVotersLearnersAll(t *testing.T) { diff --git a/pkg/testutils/serverutils/test_cluster_shim.go b/pkg/testutils/serverutils/test_cluster_shim.go index db2a8e062d87..c8bc71157c69 100644 --- a/pkg/testutils/serverutils/test_cluster_shim.go +++ b/pkg/testutils/serverutils/test_cluster_shim.go @@ -104,6 +104,21 @@ type TestClusterInterface interface { t testing.TB, startKey roachpb.Key, targets ...roachpb.ReplicationTarget, ) roachpb.RangeDescriptor + // SwapVoterWithNonVoter atomically "swaps" the voting replica located on + // `voterTarget` with the non-voting replica located on `nonVoterTarget`. A + // sequence of ReplicationChanges is considered to have "swapped" a voter on + // s1 with a non-voter on s2 iff the resulting state after the execution of + // these changes is such that s1 has a non-voter and s2 has a voter. + SwapVoterWithNonVoter( + startKey roachpb.Key, voterTarget, nonVoterTarget roachpb.ReplicationTarget, + ) (*roachpb.RangeDescriptor, error) + + // SwapVoterWithNonVoterOrFatal is the same as SwapVoterWithNonVoter but will + // fatal if it fails. + SwapVoterWithNonVoterOrFatal( + t *testing.T, startKey roachpb.Key, voterTarget, nonVoterTarget roachpb.ReplicationTarget, + ) *roachpb.RangeDescriptor + // FindRangeLeaseHolder returns the current lease holder for the given range. // In particular, it returns one particular node's (the hint, if specified) view // of the current lease. diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index 9fa3cb4cc09e..3ba606e84831 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" + "github.com/stretchr/testify/require" "go.etcd.io/etcd/raft/v3" ) @@ -785,7 +786,46 @@ func (tc *TestCluster) RemoveNonVotersOrFatal( return desc } -// TransferRangeLease is part of the TestClusterInterface. +func (tc *TestCluster) SwapVoterWithNonVoter( + startKey roachpb.Key, voterTarget, nonVoterTarget roachpb.ReplicationTarget, +) (*roachpb.RangeDescriptor, error) { + ctx := context.Background() + key := keys.MustAddr(startKey) + var beforeDesc roachpb.RangeDescriptor + if err := tc.Servers[0].DB().GetProto( + ctx, keys.RangeDescriptorKey(key), &beforeDesc, + ); err != nil { + return nil, errors.Wrap(err, "range descriptor lookup error") + } + changes := []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_VOTER, Target: nonVoterTarget}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: nonVoterTarget}, + {ChangeType: roachpb.ADD_NON_VOTER, Target: voterTarget}, + {ChangeType: roachpb.REMOVE_VOTER, Target: voterTarget}, + } + + return tc.Servers[0].DB().AdminChangeReplicas(ctx, key, beforeDesc, changes) +} + +// SwapVoterWithNonVoterOrFatal is part of TestClusterInterface. +func (tc *TestCluster) SwapVoterWithNonVoterOrFatal( + t *testing.T, startKey roachpb.Key, voterTarget, nonVoterTarget roachpb.ReplicationTarget, +) *roachpb.RangeDescriptor { + afterDesc, err := tc.SwapVoterWithNonVoter(startKey, voterTarget, nonVoterTarget) + + // Verify that the swap actually worked. + require.NoError(t, err) + replDesc, ok := afterDesc.GetReplicaDescriptor(voterTarget.StoreID) + require.True(t, ok) + require.Equal(t, roachpb.NON_VOTER, replDesc.GetType()) + replDesc, ok = afterDesc.GetReplicaDescriptor(nonVoterTarget.StoreID) + require.True(t, ok) + require.Equal(t, roachpb.VOTER_FULL, replDesc.GetType()) + + return afterDesc +} + +// TransferRangeLease is part of the TestServerInterface. func (tc *TestCluster) TransferRangeLease( rangeDesc roachpb.RangeDescriptor, dest roachpb.ReplicationTarget, ) error {