From f10689362a2ec6e23808defa3c25c69f32169598 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 30 Aug 2019 09:51:11 +0200 Subject: [PATCH] storage: work around can't-swap-leaseholder 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 --- pkg/storage/replicate_queue.go | 52 ++++++++++++++++++++++++++++------ 1 file changed, 43 insertions(+), 9 deletions(-) diff --git a/pkg/storage/replicate_queue.go b/pkg/storage/replicate_queue.go index a672d14ad6a8..b1a08dc53d41 100644 --- a/pkg/storage/replicate_queue.go +++ b/pkg/storage/replicate_queue.go @@ -508,9 +508,16 @@ func (rq *replicateQueue) findRemoveTarget( 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. +// 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 } @@ -710,20 +717,47 @@ 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. + // + // 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,