Skip to content

Commit

Permalink
Merge #40363
Browse files Browse the repository at this point in the history
40363: storage: work around can't-swap-leaseholder r=nvanbenschoten a=tbg

As of #40284, the replicate queue was issuing swaps (atomic add+remove)
during rebalancing. TestInitialPartitioning helpfully points out (once you
flip atomic rebalancing on) that when the replication factor is one, there
is no way to perform such an atomic swap because it will necessarily have
to remove the leaseholder.

To work around this restriction (which, by the way, we dislike - see
\#40333), fall back to just adding a replica in this case without also
removing one. In the next scanner cycle (which should happen immediately
since we requeue the range) the range will be over-replicated and hopefully
the lease will be transferred over and then the original leaseholder
removed. I would be very doubtful that this all works, but it is how things
worked until #40284, so this PR really just falls back to the previous
behavior in cases where we can't do better.

Release note: None

Co-authored-by: Tobias Schottdorf <[email protected]>
  • Loading branch information
craig[bot] and tbg committed Sep 3, 2019
2 parents 3d8394e + 3686af8 commit 77f6db6
Show file tree
Hide file tree
Showing 2 changed files with 53 additions and 15 deletions.
9 changes: 5 additions & 4 deletions pkg/storage/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -975,10 +975,11 @@ func (r *Replica) changeReplicasImpl(
}
// Don't leave a learner replica lying around if we didn't succeed in
// promoting it to a voter.
targets := chgs.Additions()
log.Infof(ctx, "could not promote %v to voter, rolling back: %v", targets, err)
for _, target := range targets {
r.tryRollBackLearnerReplica(ctx, r.Desc(), target, reason, details)
if targets := chgs.Additions(); len(targets) > 0 {
log.Infof(ctx, "could not promote %v to voter, rolling back: %v", targets, err)
for _, target := range targets {
r.tryRollBackLearnerReplica(ctx, r.Desc(), target, reason, details)
}
}
return nil, err
}
Expand Down
59 changes: 48 additions & 11 deletions pkg/storage/replicate_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -501,16 +501,24 @@ func (rq *replicateQueue) findRemoveTarget(
}
if len(candidates) == 0 {
// If we timed out and still don't have any valid candidates, give up.
return roachpb.ReplicaDescriptor{}, "", errors.Errorf("no removable replicas from range that needs a removal: %s",
rangeRaftProgress(repl.RaftStatus(), existingReplicas))
return roachpb.ReplicaDescriptor{}, "", &benignError{errors.Errorf("no removable replicas from range that needs a removal: %s",
rangeRaftProgress(repl.RaftStatus(), existingReplicas))}
}

return rq.allocator.RemoveTarget(ctx, zone, candidates, existingReplicas)
}

// maybeTransferLeaseAway is called whenever a replica on a given store is
// slated for removal. If the store corresponds to the store of the caller
// (which is very likely to be the leaseholder), then this removal would fail.
// Instead, this method will attempt to transfer the lease away, and returns
// true to indicate to the caller that it should not pursue the current
// replication change further because it is no longer the leaseholder. When the
// returned bool is false, it should continue. On error, the caller should also
// stop.
func (rq *replicateQueue) maybeTransferLeaseAway(
ctx context.Context, repl *Replica, removeStoreID roachpb.StoreID, dryRun bool,
) (transferred bool, _ error) {
) (done bool, _ error) {
if removeStoreID != repl.store.StoreID() {
return false, nil
}
Expand Down Expand Up @@ -710,20 +718,49 @@ func (rq *replicateQueue) considerRebalance(
} else {
// We have a replica to remove and one we can add, so let's swap them
// out.
chgs := []roachpb.ReplicationChange{
// NB: we place the addition first because in the case of
// atomic replication changes being turned off, the changes
// will be executed individually in the order in which they
// appear.
{Target: addTarget, ChangeType: roachpb.ADD_REPLICA},
{Target: removeTarget, ChangeType: roachpb.REMOVE_REPLICA},
}

if len(existingReplicas) == 1 {
// If there's only one replica, the removal target is the
// leaseholder and this is unsupported and will fail. However,
// this is also the only way to rebalance in a single-replica
// range. If we try the atomic swap here, we'll fail doing
// nothing, and so we stay locked into the current distribution
// of replicas. (Note that maybeTransferLeaseAway above will not
// have found a target, and so will have returned (false, nil).
//
// Do the best thing we can, which is carry out the addition
// only, which should succeed, and the next time we touch this
// range, we will have one more replica and hopefully it will
// take the lease and remove the current leaseholder.
//
// It's possible that "rebalancing deadlock" can occur in other
// scenarios, it's really impossible to tell from the code given
// the constraints we support. However, the lease transfer often
// does not happen spuriously, and we can't enter dangerous
// configurations sporadically, so this code path is only hit
// when we know it's necessary, picking the smaller of two evils.
//
// See https://github.com/cockroachdb/cockroach/issues/40333.
chgs = chgs[:1]
log.VEventf(ctx, 1, "can't swap replica due to lease; falling back to add")
}

rq.metrics.RebalanceReplicaCount.Inc(1)
log.VEventf(ctx, 1, "rebalancing %+v to %+v: %s",
removeTarget, addTarget, rangeRaftProgress(repl.RaftStatus(), existingReplicas))

if err := rq.changeReplicas(
ctx,
repl,
[]roachpb.ReplicationChange{
// NB: we place the addition first because in the case of
// atomic replication changes being turned off, the changes
// will be executed individually in the order in which they
// appear.
{Target: addTarget, ChangeType: roachpb.ADD_REPLICA},
{Target: removeTarget, ChangeType: roachpb.REMOVE_REPLICA},
},
chgs,
desc,
SnapshotRequest_REBALANCE,
storagepb.ReasonRebalance,
Expand Down

0 comments on commit 77f6db6

Please sign in to comment.