From 7c08e57224e8437187d357f8e739930a8fb9a632 Mon Sep 17 00:00:00 2001 From: aayush Date: Wed, 24 Mar 2021 22:32:00 -0400 Subject: [PATCH 1/3] kvserver: remove rotten comment inside `AdminRelocateRange` Release note: None --- pkg/kv/kvserver/replica_command.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 8cdccd97b0fc..f9c93635b865 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -2860,11 +2860,6 @@ func (s *Store) relocateOne( // allocator about this because we want to respect the constraints. For // example, it would be unfortunate if we put two replicas into the same zone // despite having a locality- preserving option available. - // - // TODO(radu): we can't have multiple replicas on different stores on the - // same node, and this code doesn't do anything to specifically avoid that - // case (although the allocator will avoid even trying to send snapshots to - // such stores), so it could cause some failures. args := getRelocationArgs() existingVoters := desc.Replicas().VoterDescriptors() existingNonVoters := desc.Replicas().NonVoterDescriptors() From fb5167146bb31dc215882fdfec2b1851590bb5f4 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Thu, 25 Mar 2021 20:22:46 -0400 Subject: [PATCH 2/3] kvserver: update `AdminRelocateRange` to leverage explicit swaps of voters to non-voters This commit updates `AdminRelocateRange` to use explicit atomic swaps of voting replicas with non-voting replicas, that #58627 initially added support for. The patch does so by generalizing behavior that's already exercised by the `replicateQueue` when it decides to rebalance replicas. See #61239. This allows us, in the next commit, to remove bespoke relocation logic that's used by the `mergeQueue` to align replica sets for the sake of a range merge. Release note: None --- pkg/kv/kvserver/allocator.go | 33 +- pkg/kv/kvserver/client_relocate_range_test.go | 104 +++++- pkg/kv/kvserver/replica_command.go | 302 ++++++++++++------ pkg/kv/kvserver/replicate_queue.go | 58 ++-- pkg/kv/kvserver/testing_knobs.go | 4 + 5 files changed, 362 insertions(+), 139 deletions(-) diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index 45df126e456d..ace9bc8836d0 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -184,14 +184,43 @@ const ( nonVoterTarget ) +// AddChangeType returns the roachpb.ReplicaChangeType corresponding to the +// given targetReplicaType. +// +// TODO(aayush): Clean up usages of ADD_{NON_}VOTER. Use +// targetReplicaType.{Add,Remove}ChangeType methods wherever possible. +func (t targetReplicaType) AddChangeType() roachpb.ReplicaChangeType { + switch t { + case voterTarget: + return roachpb.ADD_VOTER + case nonVoterTarget: + return roachpb.ADD_NON_VOTER + default: + panic(fmt.Sprintf("unknown targetReplicaType %d", t)) + } +} + +// RemoveChangeType returns the roachpb.ReplicaChangeType corresponding to the +// given targetReplicaType. +func (t targetReplicaType) RemoveChangeType() roachpb.ReplicaChangeType { + switch t { + case voterTarget: + return roachpb.REMOVE_VOTER + case nonVoterTarget: + return roachpb.REMOVE_NON_VOTER + default: + panic(fmt.Sprintf("unknown targetReplicaType %d", t)) + } +} + func (t targetReplicaType) String() string { - switch typ := t; typ { + switch t { case voterTarget: return "voter" case nonVoterTarget: return "non-voter" default: - panic(fmt.Sprintf("unknown targetReplicaType %d", typ)) + panic(fmt.Sprintf("unknown targetReplicaType %d", t)) } } diff --git a/pkg/kv/kvserver/client_relocate_range_test.go b/pkg/kv/kvserver/client_relocate_range_test.go index 5fe7e4c8a99d..186a66395dc7 100644 --- a/pkg/kv/kvserver/client_relocate_range_test.go +++ b/pkg/kv/kvserver/client_relocate_range_test.go @@ -12,8 +12,10 @@ package kvserver_test import ( "context" + "math/rand" "sort" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" @@ -21,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -34,12 +37,16 @@ func relocateAndCheck( voterTargets []roachpb.ReplicationTarget, nonVoterTargets []roachpb.ReplicationTarget, ) (retries int) { + every := log.Every(1 * time.Second) testutils.SucceedsSoon(t, func() error { err := tc.Servers[0].DB(). AdminRelocateRange( context.Background(), startKey.AsRawKey(), voterTargets, nonVoterTargets, ) if err != nil { + if every.ShouldLog() { + log.Infof(context.Background(), "AdminRelocateRange failed with error: %s", err) + } retries++ } return err @@ -96,6 +103,25 @@ func requireLeaseAt( }) } +func usesAtomicReplicationChange(ops []roachpb.ReplicationChange) bool { + // There are 4 sets of operations that are executed atomically: + // 1. Voter rebalances (ADD_VOTER, REMOVE_VOTER) + // 2. Non-voter promoted to voter (ADD_VOTER, REMOVE_NON_VOTER) + // 3. Voter demoted to non-voter (ADD_NON_VOTER, REMOVE_VOTER) + // 4. Voter swapped with non-voter (ADD_VOTER, REMOVE_NON_VOTER, + // ADD_NON_VOTER, REMOVE_VOTER) + if len(ops) >= 2 { + if ops[0].ChangeType == roachpb.ADD_VOTER && ops[1].ChangeType.IsRemoval() { + return true + } + } + if len(ops) == 2 && + ops[0].ChangeType == roachpb.ADD_NON_VOTER && ops[1].ChangeType == roachpb.REMOVE_VOTER { + return true + } + return false +} + func TestAdminRelocateRange(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -118,7 +144,7 @@ func TestAdminRelocateRange(t *testing.T) { if ic.err != nil { continue } - if len(ic.ops) == 2 && ic.ops[0].ChangeType == roachpb.ADD_VOTER && ic.ops[1].ChangeType == roachpb.REMOVE_VOTER { + if usesAtomicReplicationChange(ic.ops) { actAtomic++ } else { actSingle += len(ic.ops) @@ -206,10 +232,7 @@ func TestAdminRelocateRange(t *testing.T) { }) } - // Relocation of non-voting replicas is not done atomically under any - // scenario. - // TODO(aayush): Update this comment and test once we support atomic swaps of - // more than 1 non-voter at a time. + // Simple non-voter relocations. { requireNumAtomic(0, 2, func() int { return relocateAndCheck(t, tc, k, tc.Targets(2, 4), tc.Targets(1, 3)) @@ -223,4 +246,75 @@ func TestAdminRelocateRange(t *testing.T) { return relocateAndCheck(t, tc, k, tc.Targets(2, 4), tc.Targets(0, 3)) }) } + + // Relocation scenarios that require swapping of voters with non-voters. + { + // Single swap of voter and non-voter. + requireNumAtomic(1, 0, func() int { + return relocateAndCheck(t, tc, k, tc.Targets(0, 4), tc.Targets(2, 3)) + }) + // Multiple swaps. + requireNumAtomic(2, 0, func() int { + return relocateAndCheck(t, tc, k, tc.Targets(2, 3), tc.Targets(0, 4)) + }) + // Single promotion of non-voter to a voter. + requireNumAtomic(1, 0, func() int { + return relocateAndCheck(t, tc, k, tc.Targets(2, 3, 4), tc.Targets(0)) + }) + // Single demotion of voter to a non-voter. + requireNumAtomic(1, 0, func() int { + return relocateAndCheck(t, tc, k, tc.Targets(2, 4), tc.Targets(0, 3)) + }) + } +} + +// TestAdminRelocateRangeRandom runs a series of random relocations on a scratch +// range and checks to ensure that the relocations were successfully executed. +func TestAdminRelocateRangeRandom(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + args := base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + DontIgnoreFailureToTransferLease: true, + }, + NodeLiveness: kvserver.NodeLivenessTestingKnobs{ + // Use a long liveness duration to avoid flakiness under stress on the + // lease check performed by `relocateAndCheck`. + LivenessDuration: 20 * time.Second, + }, + }, + }, + } + numNodes, numIterations := 5, 10 + if util.RaceEnabled { + numNodes, numIterations = 3, 1 + } + + randomRelocationTargets := func() (voterTargets, nonVoterTargets []int) { + targets := make([]int, numNodes) + for i := 0; i < numNodes; i++ { + targets[i] = i + } + numVoters := 1 + rand.Intn(numNodes) // Need at least one voter. + rand.Shuffle(numNodes, func(i, j int) { + targets[i], targets[j] = targets[j], targets[i] + }) + + return targets[:numVoters], targets[numVoters:] + } + + tc := testcluster.StartTestCluster(t, numNodes, args) + defer tc.Stopper().Stop(ctx) + + k := keys.MustAddr(tc.ScratchRange(t)) + for i := 0; i < numIterations; i++ { + voters, nonVoters := randomRelocationTargets() + relocateAndCheck(t, tc, k, tc.Targets(voters...), tc.Targets(nonVoters...)) + } } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index f9c93635b865..5e5afd260fb3 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -2728,7 +2728,7 @@ func (s *Store) relocateReplicas( } return false } - transferLease := func(target roachpb.ReplicationTarget) { + transferLease := func(target roachpb.ReplicationTarget) error { // TODO(tbg): we ignore errors here, but it seems that in practice these // transfers "always work". Some of them are essential (we can't remove // the leaseholder so we'll fail there later if this fails), so it @@ -2738,7 +2738,11 @@ func (s *Store) relocateReplicas( ctx, startKey, target.StoreID, ); err != nil { log.Warningf(ctx, "while transferring lease: %+v", err) + if s.TestingKnobs().DontIgnoreFailureToTransferLease { + return err + } } + return nil } every := log.Every(time.Minute) @@ -2756,7 +2760,9 @@ func (s *Store) relocateReplicas( // NB: we may need to transfer even if there are no ops, to make // sure the attempt is made to make the first target the final // leaseholder. - transferLease(*leaseTarget) + if err := transferLease(*leaseTarget); err != nil { + return rangeDesc, err + } } if len(ops) == 0 { // Done. @@ -2766,11 +2772,6 @@ func (s *Store) relocateReplicas( fn(ops, leaseTarget, err) } - // Make sure we don't issue anything but singles and swaps before - // this migration is gone (for it doesn't support anything else). - if len(ops) > 2 { - log.Fatalf(ctx, "received more than 2 ops: %+v", ops) - } opss := [][]roachpb.ReplicationChange{ops} success := true for _, ops := range opss { @@ -2796,10 +2797,43 @@ func (s *Store) relocateReplicas( } type relocationArgs struct { - targetsToAdd, targetsToRemove []roachpb.ReplicationTarget - addOp, removeOp roachpb.ReplicaChangeType - relocationTargets []roachpb.ReplicationTarget - targetType targetReplicaType + votersToAdd, votersToRemove []roachpb.ReplicationTarget + nonVotersToAdd, nonVotersToRemove []roachpb.ReplicationTarget + finalVoterTargets, finalNonVoterTargets []roachpb.ReplicationTarget + targetType targetReplicaType +} + +func (r *relocationArgs) targetsToAdd() []roachpb.ReplicationTarget { + switch r.targetType { + case voterTarget: + return r.votersToAdd + case nonVoterTarget: + return r.nonVotersToAdd + default: + panic(fmt.Sprintf("unknown targetReplicaType: %s", r.targetType)) + } +} + +func (r *relocationArgs) targetsToRemove() []roachpb.ReplicationTarget { + switch r.targetType { + case voterTarget: + return r.votersToRemove + case nonVoterTarget: + return r.nonVotersToRemove + default: + panic(fmt.Sprintf("unknown targetReplicaType: %s", r.targetType)) + } +} + +func (r *relocationArgs) finalRelocationTargets() []roachpb.ReplicationTarget { + switch r.targetType { + case voterTarget: + return r.finalVoterTargets + case nonVoterTarget: + return r.finalNonVoterTargets + default: + panic(fmt.Sprintf("unknown targetReplicaType: %s", r.targetType)) + } } func (s *Store) relocateOne( @@ -2826,55 +2860,26 @@ func (s *Store) relocateOne( storeList, _, _ := s.allocator.storePool.getStoreList(storeFilterNone) storeMap := storeListToMap(storeList) - getRelocationArgs := func() relocationArgs { - votersToAdd := subtractTargets(voterTargets, desc.Replicas().Voters().ReplicationTargets()) - votersToRemove := subtractTargets(desc.Replicas().Voters().ReplicationTargets(), voterTargets) - // If there are no voters to relocate, we relocate the non-voters. - // - // NB: This means that non-voters are handled after all voters have been - // relocated since relocateOne is expected to be called repeatedly until - // there are no more replicas to relocate. - if len(votersToAdd) == 0 && len(votersToRemove) == 0 { - nonVotersToAdd := subtractTargets(nonVoterTargets, desc.Replicas().NonVoters().ReplicationTargets()) - nonVotersToRemove := subtractTargets(desc.Replicas().NonVoters().ReplicationTargets(), nonVoterTargets) - return relocationArgs{ - targetsToAdd: nonVotersToAdd, - targetsToRemove: nonVotersToRemove, - addOp: roachpb.ADD_NON_VOTER, - removeOp: roachpb.REMOVE_NON_VOTER, - relocationTargets: nonVoterTargets, - targetType: nonVoterTarget, - } - } - return relocationArgs{ - targetsToAdd: votersToAdd, - targetsToRemove: votersToRemove, - addOp: roachpb.ADD_VOTER, - removeOp: roachpb.REMOVE_VOTER, - relocationTargets: voterTargets, - targetType: voterTarget, - } - } - // Compute which replica to add and/or remove, respectively. We then ask the // allocator about this because we want to respect the constraints. For // example, it would be unfortunate if we put two replicas into the same zone // despite having a locality- preserving option available. - args := getRelocationArgs() + args := getRelocationArgs(desc, voterTargets, nonVoterTargets) existingVoters := desc.Replicas().VoterDescriptors() existingNonVoters := desc.Replicas().NonVoterDescriptors() existingReplicas := desc.Replicas().Descriptors() - var ops roachpb.ReplicationChanges - if len(args.targetsToAdd) > 0 { + var additionTarget, removalTarget roachpb.ReplicationTarget + var shouldAdd, shouldRemove, canPromoteNonVoter, canDemoteVoter bool + if len(args.targetsToAdd()) > 0 { // Each iteration, pick the most desirable replica to add. However, // prefer the first target because it's the one that should hold the // lease in the end; it helps to add it early so that the lease doesn't // have to move too much. - candidateTargets := args.targetsToAdd + candidateTargets := args.targetsToAdd() if args.targetType == voterTarget && - storeHasReplica(args.relocationTargets[0].StoreID, candidateTargets) { - candidateTargets = []roachpb.ReplicationTarget{args.relocationTargets[0]} + storeHasReplica(args.finalRelocationTargets()[0].StoreID, candidateTargets) { + candidateTargets = []roachpb.ReplicationTarget{args.finalRelocationTargets()[0]} } // The storeList's list of stores is used to constrain which stores the @@ -2898,39 +2903,60 @@ func (s *Store) relocateOne( existingVoters, existingNonVoters, s.allocator.scorerOptions(), - args.targetType) + args.targetType, + ) if targetStore == nil { - return nil, nil, fmt.Errorf("none of the remaining %ss %v are legal additions to %v", - args.targetType, args.targetsToAdd, desc.Replicas()) + return nil, nil, fmt.Errorf( + "none of the remaining %ss %v are legal additions to %v", + args.targetType, args.targetsToAdd(), desc.Replicas(), + ) } - target := roachpb.ReplicationTarget{ + additionTarget = roachpb.ReplicationTarget{ NodeID: targetStore.Node.NodeID, StoreID: targetStore.StoreID, } - ops = append(ops, roachpb.MakeReplicationChanges(args.addOp, target)...) - // Pretend the replica is already there so that the removal logic below will - // take it into account when deciding which replica to remove. - if args.targetType == nonVoterTarget { - existingNonVoters = append(existingNonVoters, roachpb.ReplicaDescriptor{ - NodeID: target.NodeID, - StoreID: target.StoreID, - ReplicaID: desc.NextReplicaID, - Type: roachpb.ReplicaTypeNonVoter(), - }) + // Pretend the new replica is already there so that the removal logic below + // will take it into account when deciding which replica to remove. + if args.targetType == voterTarget { + existingVoters = append( + existingVoters, roachpb.ReplicaDescriptor{ + NodeID: additionTarget.NodeID, + StoreID: additionTarget.StoreID, + ReplicaID: desc.NextReplicaID, + Type: roachpb.ReplicaTypeVoterFull(), + }, + ) + // When we're relocating voting replicas, `additionTarget` is allowed to + // be holding a non-voter. If that is the case, we want to promote that + // non-voter instead of removing it and then adding a new voter. + for i, nonVoter := range existingNonVoters { + if nonVoter.StoreID == additionTarget.StoreID { + canPromoteNonVoter = true + + // If can perform a promotion then we want that non-voter to be gone + // from `existingNonVoters`. + existingNonVoters[i] = existingNonVoters[len(existingNonVoters)-1] + existingNonVoters = existingNonVoters[:len(existingNonVoters)-1] + break + } + } } else { - existingVoters = append(existingVoters, roachpb.ReplicaDescriptor{ - NodeID: target.NodeID, - StoreID: target.StoreID, - ReplicaID: desc.NextReplicaID, - Type: roachpb.ReplicaTypeVoterFull(), - }) + existingNonVoters = append( + existingNonVoters, roachpb.ReplicaDescriptor{ + NodeID: additionTarget.NodeID, + StoreID: additionTarget.StoreID, + ReplicaID: desc.NextReplicaID, + Type: roachpb.ReplicaTypeNonVoter(), + }, + ) } + shouldAdd = true } var transferTarget *roachpb.ReplicationTarget - if len(args.targetsToRemove) > 0 { + if len(args.targetsToRemove()) > 0 { // Pick a replica to remove. Note that existingVoters/existingNonVoters may // already reflect a replica we're adding in the current round. This is the // right thing to do. For example, consider relocating from (s1,s2,s3) to @@ -2939,13 +2965,17 @@ func (s *Store) relocateOne( // (s1,s2,s3,s4) which is a reasonable request; that replica set is // overreplicated. If we asked it instead to remove s3 from (s1,s2,s3) it // may not want to do that due to constraints. - targetStore, _, err := s.allocator.removeTarget(ctx, zone, args.targetsToRemove, existingVoters, - existingNonVoters, args.targetType) + targetStore, _, err := s.allocator.removeTarget( + ctx, zone, args.targetsToRemove(), existingVoters, + existingNonVoters, args.targetType, + ) if err != nil { - return nil, nil, errors.Wrapf(err, "unable to select removal target from %v; current replicas %v", - args.targetsToRemove, existingReplicas) + return nil, nil, errors.Wrapf( + err, "unable to select removal target from %v; current replicas %v", + args.targetsToRemove(), existingReplicas, + ) } - removalTarget := roachpb.ReplicationTarget{ + removalTarget = roachpb.ReplicationTarget{ NodeID: targetStore.NodeID, StoreID: targetStore.StoreID, } @@ -2962,39 +2992,72 @@ func (s *Store) relocateOne( return nil, nil, errors.Wrap(err, "looking up lease") } curLeaseholder := b.RawResponse().Responses[0].GetLeaseInfo().Lease.Replica - ok := curLeaseholder.StoreID != removalTarget.StoreID - if !ok && args.targetType == voterTarget { - // Pick a voting replica that we can give the lease to. We sort the first - // target to the beginning (if it's there) because that's where the lease - // needs to be in the end. We also exclude the last replica if it was - // added by the add branch above (in which case it doesn't exist yet). - sortedTargetReplicas := append([]roachpb.ReplicaDescriptor(nil), existingVoters[:len(existingVoters)-len(ops)]...) - sort.Slice(sortedTargetReplicas, func(i, j int) bool { - sl := sortedTargetReplicas - // relocationTargets[0] goes to the front (if it's present). - return sl[i].StoreID == args.relocationTargets[0].StoreID - }) - for _, rDesc := range sortedTargetReplicas { - if rDesc.StoreID != curLeaseholder.StoreID { - transferTarget = &roachpb.ReplicationTarget{ - NodeID: rDesc.NodeID, - StoreID: rDesc.StoreID, + shouldRemove = curLeaseholder.StoreID != removalTarget.StoreID + if args.targetType == voterTarget { + // If the voter being removed is about to be added as a non-voter, then we + // can just demote it. + for _, target := range args.nonVotersToAdd { + if target.StoreID == removalTarget.StoreID { + canDemoteVoter = true + } + } + if !shouldRemove { + // Pick a voting replica that we can give the lease to. We sort the first + // target to the beginning (if it's there) because that's where the lease + // needs to be in the end. We also exclude the last voter if it was + // added by the add branch above (in which case it doesn't exist yet). + added := 0 + if shouldAdd { + added++ + } + sortedTargetReplicas := append( + []roachpb.ReplicaDescriptor(nil), + existingVoters[:len(existingVoters)-added]..., + ) + sort.Slice( + sortedTargetReplicas, func(i, j int) bool { + sl := sortedTargetReplicas + // finalRelocationTargets[0] goes to the front (if it's present). + return sl[i].StoreID == args.finalRelocationTargets()[0].StoreID + }, + ) + for _, rDesc := range sortedTargetReplicas { + if rDesc.StoreID != curLeaseholder.StoreID { + transferTarget = &roachpb.ReplicationTarget{ + NodeID: rDesc.NodeID, + StoreID: rDesc.StoreID, + } + shouldRemove = true + break } - ok = true - break } } } + } - // Carry out the removal only if there was no lease problem above. If - // there was, we're not going to do a swap in this round but just do the - // addition. (Note that !ok implies that len(ops) is not empty, or we're - // trying to remove the last replica left in the descriptor which is - // illegal). - if ok { - ops = append(ops, roachpb.MakeReplicationChanges( - args.removeOp, - removalTarget)...) + var ops []roachpb.ReplicationChange + if shouldAdd && shouldRemove { + ops, _, err = replicationChangesForRebalance( + ctx, desc, len(existingVoters), additionTarget, removalTarget, args.targetType, + ) + if err != nil { + return nil, nil, err + } + } else if shouldAdd { + if canPromoteNonVoter { + ops = roachpb.ReplicationChangesForPromotion(additionTarget) + } else { + ops = roachpb.MakeReplicationChanges(args.targetType.AddChangeType(), additionTarget) + } + } else if shouldRemove { + // Carry out the removal only if there was no lease problem above. If there + // was, we're not going to do a swap in this round but just do the addition. + // (Note that !shouldRemove implies that we're trying to remove the last + // replica left in the descriptor which is illegal). + if canDemoteVoter { + ops = roachpb.ReplicationChangesForDemotion(removalTarget) + } else { + ops = roachpb.MakeReplicationChanges(args.targetType.RemoveChangeType(), removalTarget) } } @@ -3003,10 +3066,45 @@ func (s *Store) relocateOne( // AdminRelocateRange specifies. transferTarget = &voterTargets[0] } - return ops, transferTarget, nil } +func getRelocationArgs( + desc *roachpb.RangeDescriptor, voterTargets, nonVoterTargets []roachpb.ReplicationTarget, +) relocationArgs { + args := relocationArgs{ + votersToAdd: subtractTargets( + voterTargets, + desc.Replicas().Voters().ReplicationTargets(), + ), + votersToRemove: subtractTargets( + desc.Replicas().Voters().ReplicationTargets(), + voterTargets, + ), + nonVotersToAdd: subtractTargets( + nonVoterTargets, + desc.Replicas().NonVoters().ReplicationTargets(), + ), + nonVotersToRemove: subtractTargets( + desc.Replicas().NonVoters().ReplicationTargets(), + nonVoterTargets, + ), + finalVoterTargets: voterTargets, + finalNonVoterTargets: nonVoterTargets, + targetType: voterTarget, + } + + // If there are no voters to relocate, we relocate the non-voters. + // + // NB: This means that non-voters are handled after all voters have been + // relocated since relocateOne is expected to be called repeatedly until + // there are no more replicas to relocate. + if len(args.votersToAdd) == 0 && len(args.votersToRemove) == 0 { + args.targetType = nonVoterTarget + } + return args +} + // subtractTargets returns the set of replica descriptors in `left` but not in // `right` (i.e. left - right). // diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index a9a2040e25ee..66086d94f187 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -617,7 +617,6 @@ func (rq *replicateQueue) addOrReplaceVoters( } else { ops = roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, newVoter) } - if removeIdx < 0 { log.VEventf(ctx, 1, "adding voter %+v: %s", newVoter, rangeRaftProgress(repl.RaftStatus(), existingVoters)) @@ -932,18 +931,15 @@ func (rq *replicateQueue) removeDecommissioning( ) (requeue bool, _ error) { desc, _ := repl.DescAndZone() var decommissioningReplicas []roachpb.ReplicaDescriptor - var removeOp roachpb.ReplicaChangeType switch targetType { case voterTarget: decommissioningReplicas = rq.allocator.storePool.decommissioningReplicas( desc.Replicas().VoterDescriptors(), ) - removeOp = roachpb.REMOVE_VOTER case nonVoterTarget: decommissioningReplicas = rq.allocator.storePool.decommissioningReplicas( desc.Replicas().NonVoterDescriptors(), ) - removeOp = roachpb.REMOVE_NON_VOTER default: panic(fmt.Sprintf("unknown targetReplicaType: %s", targetType)) } @@ -975,7 +971,7 @@ func (rq *replicateQueue) removeDecommissioning( if err := rq.changeReplicas( ctx, repl, - roachpb.MakeReplicationChanges(removeOp, target), + roachpb.MakeReplicationChanges(targetType.RemoveChangeType(), target), desc, SnapshotRequest_UNKNOWN, // unused kvserverpb.ReasonStoreDecommissioning, "", dryRun, @@ -1011,15 +1007,6 @@ func (rq *replicateQueue) removeDead( NodeID: deadReplica.NodeID, StoreID: deadReplica.StoreID, } - var removeOp roachpb.ReplicaChangeType - switch targetType { - case voterTarget: - removeOp = roachpb.REMOVE_VOTER - case nonVoterTarget: - removeOp = roachpb.REMOVE_NON_VOTER - default: - panic(fmt.Sprintf("unknown targetReplicaType: %s", targetType)) - } // NB: When removing a dead voter, we don't check whether to transfer the // lease away because if the removal target is dead, it's not the voter being @@ -1028,7 +1015,7 @@ func (rq *replicateQueue) removeDead( if err := rq.changeReplicas( ctx, repl, - roachpb.MakeReplicationChanges(removeOp, target), + roachpb.MakeReplicationChanges(targetType.RemoveChangeType(), target), desc, SnapshotRequest_UNKNOWN, // unused kvserverpb.ReasonStoreDead, @@ -1123,12 +1110,16 @@ func (rq *replicateQueue) considerRebalance( } else { // If we have a valid rebalance action (ok == true) and we haven't // transferred our lease away, execute the rebalance. - chgs, err := rq.replicationChangesForRebalance(ctx, desc, len(existingVoters), addTarget, + chgs, performingSwap, err := replicationChangesForRebalance(ctx, desc, len(existingVoters), addTarget, removeTarget, rebalanceTargetType) if err != nil { return false, err } rq.metrics.RebalanceReplicaCount.Inc(1) + if performingSwap { + rq.metrics.VoterDemotionsCount.Inc(1) + rq.metrics.NonVoterPromotionsCount.Inc(1) + } log.VEventf(ctx, 1, "rebalancing %s %+v to %+v: %s", @@ -1178,14 +1169,17 @@ func (rq *replicateQueue) considerRebalance( // replicationChangesForRebalance returns a list of ReplicationChanges to // execute for a rebalancing decision made by the allocator. -func (rq *replicateQueue) replicationChangesForRebalance( +// +// This function assumes that `addTarget` and `removeTarget` are produced by the +// allocator (i.e. they satisfy replica `constraints` and potentially +// `voter_constraints` if we're operating over voter targets). +func replicationChangesForRebalance( ctx context.Context, desc *roachpb.RangeDescriptor, numExistingVoters int, - addTarget roachpb.ReplicationTarget, - removeTarget roachpb.ReplicationTarget, + addTarget, removeTarget roachpb.ReplicationTarget, rebalanceTargetType targetReplicaType, -) (chgs []roachpb.ReplicationChange, err error) { +) (chgs []roachpb.ReplicationChange, performingSwap bool, err error) { if rebalanceTargetType == voterTarget && numExistingVoters == 1 { // If there's only one replica, the removal target is the // leaseholder and this is unsupported and will fail. However, @@ -1212,7 +1206,7 @@ func (rq *replicateQueue) replicationChangesForRebalance( {ChangeType: roachpb.ADD_VOTER, Target: addTarget}, } log.VEventf(ctx, 1, "can't swap replica due to lease; falling back to add") - return chgs, err + return chgs, false, err } rdesc, found := desc.GetReplicaDescriptor(addTarget.StoreID) @@ -1229,21 +1223,23 @@ func (rq *replicateQueue) replicationChangesForRebalance( // the `voter_constraints`, it is copacetic to make this swap since: // // 1. `addTarget` must already be a valid target for a voting replica - // (i.e. it must already satisfy both *constraints fields) since - // `Allocator.RebalanceVoter` just handed it to us. + // (i.e. it must already satisfy both *constraints fields) since an + // allocator method (`allocateTarget..` or `Rebalance{Non}Voter`) just + // handed it to us. // 2. `removeTarget` may or may not be a valid target for a non-voting // replica, but `considerRebalance` takes care to `requeue` the current // replica into the replicateQueue. So we expect the replicateQueue's next // attempt at rebalancing this range to rebalance the non-voter if it ends // up being in violation of the range's constraints. - rq.metrics.NonVoterPromotionsCount.Inc(1) - rq.metrics.VoterDemotionsCount.Inc(1) promo := roachpb.ReplicationChangesForPromotion(addTarget) demo := roachpb.ReplicationChangesForDemotion(removeTarget) chgs = append(promo, demo...) + performingSwap = true } else if found { - return nil, errors.AssertionFailedf("programming error:"+ - " store being rebalanced to(%s) already has a voting replica", addTarget.StoreID) + return nil, false, errors.AssertionFailedf( + "programming error:"+ + " store being rebalanced to(%s) already has a voting replica", addTarget.StoreID, + ) } else { // We have a replica to remove and one we can add, so let's swap them out. chgs = []roachpb.ReplicationChange{ @@ -1256,15 +1252,17 @@ func (rq *replicateQueue) replicationChangesForRebalance( // Non-voters should not consider any of the range's existing stores as // valid candidates. If we get here, we must have raced with another // rebalancing decision. - return nil, errors.AssertionFailedf("invalid rebalancing decision: trying to"+ - " move non-voter to a store that already has a replica %s for the range", rdesc) + return nil, false, errors.AssertionFailedf( + "invalid rebalancing decision: trying to"+ + " move non-voter to a store that already has a replica %s for the range", rdesc, + ) } chgs = []roachpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: addTarget}, {ChangeType: roachpb.REMOVE_NON_VOTER, Target: removeTarget}, } } - return chgs, nil + return chgs, performingSwap, nil } type transferLeaseOptions struct { diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index ed18b0d494d2..33fc98d4acc6 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -221,6 +221,10 @@ type StoreTestingKnobs struct { // BeforeRelocateOne intercepts the return values of s.relocateOne before // they're being put into effect. BeforeRelocateOne func(_ []roachpb.ReplicationChange, leaseTarget *roachpb.ReplicationTarget, _ error) + // DontIgnoreFailureToTransferLease makes `AdminRelocateRange` return an error + // to its client if it failed to transfer the lease to the first voting + // replica in the set of relocation targets. + DontIgnoreFailureToTransferLease bool // MaxApplicationBatchSize enforces a maximum size on application batches. // This can be useful for testing conditions which require commands to be // applied in separate batches. From d7a7a49a6ca48758fb979c98a2aa04e358effab2 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Thu, 25 Mar 2021 20:28:58 -0400 Subject: [PATCH 3/3] kvserver: get rid of bespoke relocation logic used by the mergeQueue This commit removes the relocation logic used by the `mergeQueue` thus far to align replica sets (added in #56197). This logic previously existed in order to allow us to align the replica sets of a pair of ranges (which is required for the range merge to proceed), while avoiding redundant data movement. Before #58627 and the previous commit in this PR, `AdminRelocateRange` couldn't be directly used by the mergeQueue under various configurations of the LHS and RHS ranges. Furthermore, even when it could be used, it would involve redundant data movement. This all required us to compute relocation targets for the RHS of a merge separately, above the call to `AdminRelocateRange`, for the range merge to proceed. All these limitations have been resolved by the previous commit which teaches `AdminRelocateRange` to promote non-voters and demote voters when needed, and the aforementioned bespoke relocation logic is no longer needed. Release note: None --- pkg/kv/kvserver/BUILD.bazel | 1 - pkg/kv/kvserver/client_merge_test.go | 197 +++++++++++---------------- pkg/kv/kvserver/merge_queue.go | 6 +- pkg/kv/kvserver/replica_command.go | 95 ------------- pkg/roachpb/metadata_replicas.go | 12 -- 5 files changed, 81 insertions(+), 230 deletions(-) diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 726abae8625a..8c0034adab20 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -99,7 +99,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/build", "//pkg/clusterversion", "//pkg/config", "//pkg/config/zonepb", diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 8feb60467878..a104540db7ed 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -18,7 +18,6 @@ import ( "math/rand" "reflect" "regexp" - "sort" "strings" "sync" "sync/atomic" @@ -4175,137 +4174,117 @@ func TestMergeQueueSeesNonVoters(t *testing.T) { type test struct { name string leftVoters, rightVoters, leftNonVoters, rightNonVoters []int - expectedRightVoters, expectedRightNonVoters []int } // NB: The test setup code places a single voter replica on (n1,s1) for both // left and right range, which we remove after setting the test up. tests := []test{ { - name: "collocated-per-type", - leftVoters: []int{2, 3, 4}, - rightVoters: []int{2, 3, 4}, - leftNonVoters: []int{1}, - rightNonVoters: []int{1}, - expectedRightVoters: []int{2, 3, 4}, - expectedRightNonVoters: []int{1}, + name: "collocated-per-type", + leftVoters: []int{2, 3, 4}, + rightVoters: []int{2, 3, 4}, + leftNonVoters: []int{1}, + rightNonVoters: []int{1}, }, { - name: "collocated-overall", - leftVoters: []int{3, 4}, - rightVoters: []int{1, 2}, - leftNonVoters: []int{1, 2}, - rightNonVoters: []int{3, 4}, - expectedRightVoters: []int{1, 2}, - expectedRightNonVoters: []int{3, 4}, + name: "collocated-overall", + leftVoters: []int{3, 4}, + rightVoters: []int{1, 2}, + leftNonVoters: []int{1, 2}, + rightNonVoters: []int{3, 4}, }, { - name: "collocated-voters-only", - leftVoters: []int{3, 4}, - rightVoters: []int{3, 4}, - leftNonVoters: []int{2}, - rightNonVoters: []int{1}, - expectedRightVoters: []int{3, 4}, - expectedRightNonVoters: []int{2}, + name: "collocated-voters-only", + leftVoters: []int{3, 4}, + rightVoters: []int{3, 4}, + leftNonVoters: []int{2}, + rightNonVoters: []int{1}, }, { - name: "collocated-non-voters-only", - leftVoters: []int{3}, - rightVoters: []int{4}, - leftNonVoters: []int{1, 2}, - rightNonVoters: []int{1, 2}, - expectedRightVoters: []int{3}, - expectedRightNonVoters: []int{1, 2}, + name: "collocated-non-voters-only", + leftVoters: []int{3}, + rightVoters: []int{4}, + leftNonVoters: []int{1, 2}, + rightNonVoters: []int{1, 2}, }, { - name: "not-collocated", - leftVoters: []int{3}, - rightVoters: []int{4}, - leftNonVoters: []int{2}, - rightNonVoters: []int{1}, - expectedRightVoters: []int{3}, - expectedRightNonVoters: []int{2}, + name: "not-collocated", + leftVoters: []int{3}, + rightVoters: []int{4}, + leftNonVoters: []int{2}, + rightNonVoters: []int{1}, }, { - name: "partially-collocated-voters-only", - leftVoters: []int{2, 3}, - rightVoters: []int{1, 4}, - leftNonVoters: []int{1}, - rightNonVoters: []int{2}, - expectedRightVoters: []int{1, 3}, - expectedRightNonVoters: []int{2}, + name: "partially-collocated-voters-only", + leftVoters: []int{2, 3}, + rightVoters: []int{1, 4}, + leftNonVoters: []int{1}, + rightNonVoters: []int{2}, }, { - name: "partially-collocated-non-voters-only", - leftVoters: []int{4}, - rightVoters: []int{4}, - leftNonVoters: []int{1, 3}, - rightNonVoters: []int{1, 2}, - expectedRightVoters: []int{4}, - expectedRightNonVoters: []int{1, 3}, + name: "partially-collocated-non-voters-only", + leftVoters: []int{4}, + rightVoters: []int{4}, + leftNonVoters: []int{1, 3}, + rightNonVoters: []int{1, 2}, }, { - name: "partially-collocated", - leftVoters: []int{2}, - rightVoters: []int{4}, - leftNonVoters: []int{1, 3}, - rightNonVoters: []int{1, 2}, - expectedRightVoters: []int{3}, - expectedRightNonVoters: []int{1, 2}, + name: "partially-collocated", + leftVoters: []int{2}, + rightVoters: []int{4}, + leftNonVoters: []int{1, 3}, + rightNonVoters: []int{1, 2}, }, { - name: "collocated-rhs-being-reconfigured-1", - leftVoters: []int{1, 2, 3}, - rightVoters: []int{1, 2, 3, 4, 5, 6}, - leftNonVoters: []int{4, 5, 6}, - rightNonVoters: []int{}, - expectedRightVoters: []int{1, 2, 3, 4, 5, 6}, - expectedRightNonVoters: []int{}, + name: "collocated-rhs-being-reconfigured-1", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1, 2, 3, 4, 5, 6}, + leftNonVoters: []int{4, 5, 6}, + rightNonVoters: []int{}, }, { - name: "collocated-rhs-being-reconfigured-2", - leftVoters: []int{1, 2, 3}, - rightVoters: []int{1, 2, 3, 4}, - leftNonVoters: []int{4, 5, 6}, - rightNonVoters: []int{}, - expectedRightVoters: []int{1, 2, 3, 4}, - expectedRightNonVoters: []int{5, 6}, + name: "collocated-rhs-being-reconfigured-2", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1, 2, 3, 4}, + leftNonVoters: []int{4, 5, 6}, + rightNonVoters: []int{}, }, { - name: "collocated-rhs-being-reconfigured-3", - leftVoters: []int{1, 2, 3}, - rightVoters: []int{1}, - leftNonVoters: []int{4, 5, 6}, - rightNonVoters: []int{2, 3, 4, 5, 6}, - expectedRightVoters: []int{1}, - expectedRightNonVoters: []int{2, 3, 4, 5, 6}, + name: "collocated-rhs-being-reconfigured-3", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1}, + leftNonVoters: []int{4, 5, 6}, + rightNonVoters: []int{2, 3, 4, 5, 6}, }, { - name: "non-collocated-rhs-being-reconfigured", - leftVoters: []int{1, 2, 3}, - rightVoters: []int{5}, - leftNonVoters: []int{4, 6}, - rightNonVoters: []int{}, - expectedRightVoters: []int{1, 2, 3}, - expectedRightNonVoters: []int{4, 6}, + name: "non-collocated-rhs-being-reconfigured", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{5}, + leftNonVoters: []int{4, 6}, + rightNonVoters: []int{}, }, { - name: "partially-collocated-rhs-being-downreplicated", - leftVoters: []int{1, 2, 3}, - rightVoters: []int{1, 2, 3, 4, 5, 6}, - leftNonVoters: []int{4, 5}, - rightNonVoters: []int{}, - expectedRightVoters: []int{1, 2, 3, 4, 5}, - expectedRightNonVoters: []int{}, + name: "partially-collocated-rhs-being-downreplicated", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1, 2, 3, 4, 5, 6}, + leftNonVoters: []int{4, 5}, + rightNonVoters: []int{}, }, { - name: "partially-collocated-rhs-being-upreplicated", - leftVoters: []int{1, 2, 3}, - rightVoters: []int{1}, - leftNonVoters: []int{4, 5, 6}, - rightNonVoters: []int{}, - expectedRightVoters: []int{1, 2, 3}, - expectedRightNonVoters: []int{4, 5, 6}, + name: "partially-collocated-rhs-being-upreplicated", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1}, + leftNonVoters: []int{4, 5, 6}, + rightNonVoters: []int{}, + }, + { + // This is a subtest that should trigger at least 3 voter<->non-voter + // swaps. + name: "lhs-voters-collocated-with-rhs-non-voters", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{4}, + leftNonVoters: []int{}, + rightNonVoters: []int{1, 2, 3}, }, } @@ -4374,24 +4353,6 @@ func TestMergeQueueSeesNonVoters(t *testing.T) { tc.RemoveVotersOrFatal(t, rightDesc.StartKey.AsRawKey(), tc.Target(0)) rightDesc = tc.LookupRangeOrFatal(t, rightDesc.StartKey.AsRawKey()) - // Check that we're avoiding superfluous data movement. - voterTargets, nonVoterTargets, err := kvserver.GetTargetsToCollocateRHSForMerge(ctx, leftDesc.Replicas(), rightDesc.Replicas()) - require.NoError(t, err) - require.Equal(t, len(subtest.expectedRightVoters), len(voterTargets)) - require.Equal(t, len(subtest.expectedRightNonVoters), len(nonVoterTargets)) - sort.Slice(voterTargets, func(i, j int) bool { - return voterTargets[i].NodeID < voterTargets[j].NodeID - }) - sort.Slice(nonVoterTargets, func(i, j int) bool { - return nonVoterTargets[i].NodeID < nonVoterTargets[j].NodeID - }) - for i := range subtest.expectedRightVoters { - require.Equal(t, tc.Target(subtest.expectedRightVoters[i]), voterTargets[i]) - } - for i := range subtest.expectedRightNonVoters { - require.Equal(t, tc.Target(subtest.expectedRightNonVoters[i]), nonVoterTargets[i]) - } - store.SetMergeQueueActive(true) store.MustForceMergeScanAndProcess() verifyMerged(t, store, leftDesc.StartKey, rightDesc.StartKey) diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index 924d7e425dee..81c4bdd57078 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -304,10 +304,8 @@ func (mq *mergeQueue) process( // these ranges and only try to collocate them if they're not in violation, // which would help us make better guarantees about not transiently // violating constraints during a merge. - voterTargets, nonVoterTargets, err := GetTargetsToCollocateRHSForMerge(ctx, lhsDesc.Replicas(), rhsDesc.Replicas()) - if err != nil { - return false, err - } + voterTargets := lhsDesc.Replicas().Voters().ReplicationTargets() + nonVoterTargets := lhsDesc.Replicas().NonVoters().ReplicationTargets() // AdminRelocateRange moves the lease to the first target in the list, so // sort the existing leaseholder there to leave it unchanged. diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 5e5afd260fb3..b5347fea3b43 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -20,7 +20,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -2467,100 +2466,6 @@ func replicasCollocated(a, b []roachpb.ReplicaDescriptor) bool { return true } -// GetTargetsToCollocateRHSForMerge decides the configuration of RHS replicas -// need before the rhs can be subsumed and then merged into the LHS range. The -// desired RHS voters and non-voters are returned; together they'll cover the -// same stores as LHS's replicas, but the configuration of replicas doesn't -// necessarily match (it doesn't need to match for the merge). -// -// We compute the new voter / non-voter targets for the RHS by first -// bootstrapping our result set with the replicas that are already collocated. -// We then step through RHS's non-collocated voters and try to move them to -// stores that already have a voter for LHS. If this is not possible for all the -// non-collocated voters of RHS (i.e. because the RHS has non-voter(s) on -// store(s) where the LHS has voter(s)), we may move some RHS voters to targets -// that have non-voters for LHS. Likewise, we do the same for the non-collocated -// non-voters of RHS: try to relocate them to stores where the LHS has -// non-voters, but resort to relocating them to stores where the LHS has voters. -// -// TODO(aayush): Can moving a voter replica from RHS to a store that has a -// non-voter for LHS (or vice versa) can lead to constraint violations? Justify -// why or why not. -func GetTargetsToCollocateRHSForMerge( - ctx context.Context, leftRepls, rightRepls roachpb.ReplicaSet, -) (voterTargets, nonVoterTargets []roachpb.ReplicationTarget, _ error) { - notInRight := func(desc roachpb.ReplicaDescriptor) bool { - return !rightRepls.Contains(desc) - } - - // Sets of replicas that exist on the LHS but not on the RHS - leftMinusRight := leftRepls.Filter(notInRight) - leftMinusRightVoters := leftMinusRight.Voters().Descriptors() - leftMinusRightNonVoters := leftMinusRight.NonVoters().Descriptors() - - // We bootstrap our result set by first including the replicas (voting and - // non-voting) that _are_ collocated, as these will stay unchanged and will - // be no-ops when passed through AdminRelocateRange. - finalRightVoters := rightRepls.Voters().Filter(leftRepls.Contains).DeepCopy() - finalRightNonVoters := rightRepls.NonVoters().Filter(leftRepls.Contains).DeepCopy() - - needMore := func() bool { - return len(finalRightVoters.Descriptors())+len(finalRightNonVoters.Descriptors()) < len(leftRepls.Descriptors()) - } - - numVoters := len(leftRepls.VoterDescriptors()) - // We loop through the set of non-collocated replicas and figure out a - // suitable configuration to relocate RHS's replicas to. At the end of these - // two loops, we will have exhausted `leftMinusRight`. - for len(finalRightVoters.Descriptors()) < numVoters && needMore() { - // Prefer to relocate voters for RHS to stores that have voters for LHS, but - // resort to relocating them to stores with non-voters for LHS if that's not - // possible. - if len(leftMinusRightVoters) != 0 { - finalRightVoters.AddReplica(leftMinusRightVoters[0]) - leftMinusRightVoters = leftMinusRightVoters[1:] - } else if len(leftMinusRightNonVoters) != 0 { - finalRightVoters.AddReplica(leftMinusRightNonVoters[0]) - leftMinusRightNonVoters = leftMinusRightNonVoters[1:] - } else { - log.Fatalf(ctx, "programming error: unexpectedly ran out of valid stores to relocate RHS"+ - " voters to; LHS: %s, RHS: %s", leftRepls.Descriptors(), rightRepls.Descriptors()) - } - } - - for needMore() { - // Like above, we try to relocate non-voters for RHS to stores that have - // non-voters for LHS, but resort to relocating them to stores with voters - // for LHS if that's not possible. - if len(leftMinusRightNonVoters) != 0 { - finalRightNonVoters.AddReplica(leftMinusRightNonVoters[0]) - leftMinusRightNonVoters = leftMinusRightNonVoters[1:] - } else if len(leftMinusRightVoters) != 0 { - finalRightNonVoters.AddReplica(leftMinusRightVoters[0]) - leftMinusRightVoters = leftMinusRightVoters[1:] - } else { - log.Fatalf(ctx, "programming error: unexpectedly ran out of valid stores to relocate RHS"+ - " non-voters to; LHS: %s, RHS: %s", leftRepls.Descriptors(), rightRepls.Descriptors()) - } - } - - if len(finalRightVoters.Descriptors()) == 0 { - // TODO(aayush): We can end up in this case for scenarios like the - // following (the digits represent StoreIDs): - // - // LHS-> voters: {1, 2, 3}, non-voters: {} - // RHS-> voters: {4}, non-voters: {1, 2, 3} - // - // Remove this error path once we support swapping voters and non-voters. - return nil, nil, - errors.UnimplementedErrorf(errors.IssueLink{IssueURL: build.MakeIssueURL(58499)}, - "unsupported configuration of RHS(%s) and LHS(%s) as it requires an atomic swap of a"+ - " voter and non-voter", rightRepls, leftRepls) - } - - return finalRightVoters.ReplicationTargets(), finalRightNonVoters.ReplicationTargets(), nil -} - func checkDescsEqual(desc *roachpb.RangeDescriptor) func(*roachpb.RangeDescriptor) bool { return func(desc2 *roachpb.RangeDescriptor) bool { return desc.Equal(desc2) diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index cae6035c4c2d..77e61e9bad9e 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -334,18 +334,6 @@ func (d ReplicaSet) DeepCopy() ReplicaSet { } } -// Contains returns true if the set contains rDesc. -func (d ReplicaSet) Contains(rDesc ReplicaDescriptor) bool { - descs := d.Descriptors() - for i := range descs { - repl := &descs[i] - if repl.StoreID == rDesc.StoreID && repl.NodeID == rDesc.NodeID { - return true - } - } - return false -} - // AddReplica adds the given replica to this set. func (d *ReplicaSet) AddReplica(r ReplicaDescriptor) { d.wrapped = append(d.wrapped, r)