From 8bacca2d9b92896d5f63c1f6dd99f0045a11a093 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 31 Aug 2022 23:27:45 -0400 Subject: [PATCH] kv: campaign on rejected lease request when leader not live in node liveness MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fixes #84655. Related to #49220. This commit extends the logic introduced in 8aa1c14 to simultaneously campaign for Raft leadership when rejecting a lease request on a Raft follower that observes that the current Raft leader is not live according to node liveness. These kinds of cases are often associated with asymmetric network partitions. In such cases, the Raft leader will be unable to acquire an epoch-based lease until it heartbeats its liveness record. As a result, the range can only regain availability if a different replica acquires the lease. However, the protection added in 8aa1c14 prevents followers from acquiring leases to protect against a different form of unavailability. After this commit, the followers will attempt to acquire Raft leadership when detecting such cases by campaigning. This allows these ranges to recover availability once Raft leadership moves off the partitioned Raft leader to one of the followers that can reach node liveness and can subsequently acquire the lease. Campaigning for Raft leadership is safer than blindly allowing the lease request to be proposed (through a redirected proposal). This is because the follower may be arbitrarily far behind on its Raft log and acquiring the lease in such cases could cause unavailability (the kind we saw in #37906). By instead calling a Raft pre-vote election, the follower can determine whether it is behind on its log without risking disruption. If so, we don't want it to acquire the lease — one of the other followers that is caught up on its log can. If not, it will eventually become leader and can proceed with a future attempt to acquire the lease. The commit adds a test that reproduces the failure mode described in #84655. It creates an asymmetric network partition scenario that looks like: ``` [0] raft leader / initial leaseholder ^ / \ / \ v v [1]<--->[2] raft followers ^ ^ \ / \ / v [3] liveness range ``` It then waits for the raft leader's lease to expire and demonstrates that one of the raft followers will now call a Raft election, which allows it to safely grab Raft leadership, acquire the lease, and recover availability. Without the change, the test failed. Release justification: None. Too risky for the stability period. Potential backport candidate after sufficient baking on master. Release note (bug fix): A bug causing ranges to remain without a leaseholder in cases of asymmetric network partitions has been resolved. --- pkg/kv/kvserver/client_raft_test.go | 134 +++++++++++++++++++ pkg/kv/kvserver/replica_proposal_buf.go | 33 ++++- pkg/kv/kvserver/replica_proposal_buf_test.go | 39 +++++- pkg/kv/kvserver/replica_raft.go | 71 +++++++++- pkg/kv/kvserver/replica_test.go | 111 +++++++++++++++ 5 files changed, 378 insertions(+), 10 deletions(-) diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 4bd72e4511a3..ba00f8f5203c 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -1287,6 +1287,140 @@ func TestRequestsOnLaggingReplica(t *testing.T) { require.Equal(t, leaderReplicaID, nlhe.Lease.Replica.ReplicaID) } +// TestRequestsOnFollowerWithNonLiveLeaseholder tests the availability of a +// range that has an expired epoch-based lease and a live Raft leader that is +// unable to heartbeat its liveness record. Such a range should recover once +// Raft leadership moves off the partitioned Raft leader to one of the followers +// that can reach node liveness. +// +// This test relies on follower replicas campaigning for Raft leadership in +// certain cases when refusing to forward lease acquisition requests to the +// leader. In these cases where they determine that the leader is non-live +// according to node liveness, they will attempt to steal Raft leadership and, +// if successful, will be able to perform future lease acquisition attempts. +func TestRequestsOnFollowerWithNonLiveLeaseholder(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + var installPartition int32 + partitionFilter := func(_ context.Context, ba roachpb.BatchRequest) *roachpb.Error { + if atomic.LoadInt32(&installPartition) == 0 { + return nil + } + if ba.GatewayNodeID == 1 && ba.Replica.NodeID == 4 { + return roachpb.NewError(context.Canceled) + } + return nil + } + + clusterArgs := base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + // Reduce the election timeout some to speed up the test. + RaftConfig: base.RaftConfig{RaftElectionTimeoutTicks: 10}, + Knobs: base.TestingKnobs{ + NodeLiveness: kvserver.NodeLivenessTestingKnobs{ + // This test waits for an epoch-based lease to expire, so we're + // setting the liveness duration as low as possible while still + // keeping the test stable. + LivenessDuration: 3000 * time.Millisecond, + RenewalDuration: 1500 * time.Millisecond, + }, + Store: &kvserver.StoreTestingKnobs{ + // We eliminate clock offsets in order to eliminate the stasis period + // of leases, in order to speed up the test. + MaxOffset: time.Nanosecond, + TestingRequestFilter: partitionFilter, + }, + }, + }, + } + + tc := testcluster.StartTestCluster(t, 4, clusterArgs) + defer tc.Stopper().Stop(ctx) + + { + // Move the liveness range to node 4. + desc := tc.LookupRangeOrFatal(t, keys.NodeLivenessPrefix) + tc.RebalanceVoterOrFatal(ctx, t, desc.StartKey.AsRawKey(), tc.Target(0), tc.Target(3)) + } + + // Create a new range. + _, rngDesc, err := tc.Servers[0].ScratchRangeEx() + require.NoError(t, err) + key := rngDesc.StartKey.AsRawKey() + // Add replicas on all the stores. + tc.AddVotersOrFatal(t, rngDesc.StartKey.AsRawKey(), tc.Target(1), tc.Target(2)) + + // Store 0 holds the lease. + store0 := tc.GetFirstStoreFromServer(t, 0) + store0Repl, err := store0.GetReplica(rngDesc.RangeID) + require.NoError(t, err) + leaseStatus := store0Repl.CurrentLeaseStatus(ctx) + require.True(t, leaseStatus.OwnedBy(store0.StoreID())) + + { + // Write a value so that the respective key is present in all stores and we + // can increment it again later. + _, err := tc.Server(0).DB().Inc(ctx, key, 1) + require.NoError(t, err) + log.Infof(ctx, "test: waiting for initial values...") + tc.WaitForValues(t, key, []int64{1, 1, 1, 0}) + log.Infof(ctx, "test: waiting for initial values... done") + } + + // Begin dropping all node liveness heartbeats from the original raft leader + // while allowing the leader to maintain Raft leadership and otherwise behave + // normally. This mimics cases where the raft leader is partitioned away from + // the liveness range but can otherwise reach its followers. In these cases, + // it is still possible that the followers can reach the liveness range and + // see that the leader becomes non-live. For example, the configuration could + // look like: + // + // [0] raft leader + // ^ + // / \ + // / \ + // v v + // [1]<--->[2] raft followers + // ^ ^ + // \ / + // \ / + // v + // [3] liveness range + // + log.Infof(ctx, "test: partitioning node") + atomic.StoreInt32(&installPartition, 1) + + // Wait until the lease expires. + log.Infof(ctx, "test: waiting for lease expiration") + testutils.SucceedsSoon(t, func() error { + leaseStatus = store0Repl.CurrentLeaseStatus(ctx) + if leaseStatus.IsValid() { + return errors.New("lease still valid") + } + return nil + }) + log.Infof(ctx, "test: lease expired") + + { + // Increment the initial value again, which requires range availability. To + // get there, the request will need to trigger a lease request on a follower + // replica, which will call a Raft election, acquire Raft leadership, then + // acquire the range lease. + _, err := tc.Server(0).DB().Inc(ctx, key, 1) + require.NoError(t, err) + log.Infof(ctx, "test: waiting for new lease...") + tc.WaitForValues(t, key, []int64{2, 2, 2, 0}) + log.Infof(ctx, "test: waiting for new lease... done") + } + + // Store 0 no longer holds the lease. + leaseStatus = store0Repl.CurrentLeaseStatus(ctx) + require.False(t, leaseStatus.OwnedBy(store0.StoreID())) +} + type fakeSnapshotStream struct { nextReq *kvserverpb.SnapshotRequest nextErr error diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 2509b63c88c8..12007e9fd644 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -117,15 +118,14 @@ type propBuf struct { } type rangeLeaderInfo struct { + // iAmTheLeader is set if the local replica is the leader. + iAmTheLeader bool // leaderKnown is set if the local Raft machinery knows who the leader is. If // not set, all other fields are empty. leaderKnown bool - // leader represents the Raft group's leader. Not set if leaderKnown is not // set. leader roachpb.ReplicaID - // iAmTheLeader is set if the local replica is the leader. - iAmTheLeader bool // leaderEligibleForLease is set if the leader is known and its type of // replica allows it to acquire a lease. leaderEligibleForLease bool @@ -144,6 +144,7 @@ type proposer interface { closedTimestampTarget() hlc.Timestamp leaderStatus(ctx context.Context, raftGroup proposerRaft) rangeLeaderInfo ownsValidLease(ctx context.Context, now hlc.ClockTimestamp) bool + shouldCampaignOnRedirect(raftGroup proposerRaft) bool // The following require the proposer to hold an exclusive lock. withGroupLocked(func(proposerRaft) error) error @@ -182,7 +183,9 @@ type proposer interface { type proposerRaft interface { Step(raftpb.Message) error Status() raft.Status + BasicStatus() raft.BasicStatus ProposeConfChange(raftpb.ConfChangeI) error + Campaign() error } // Init initializes the proposal buffer and binds it to the provided proposer. @@ -614,6 +617,12 @@ func (b *propBuf) maybeRejectUnsafeProposalLocked( log.VEventf(ctx, 2, "not proposing lease acquisition because we're not the leader; replica %d is", li.leader) b.p.rejectProposalWithRedirectLocked(ctx, p, li.leader) + if b.p.shouldCampaignOnRedirect(raftGroup) { + log.VEventf(ctx, 2, "campaigning because Raft leader not live in node liveness map") + if err := raftGroup.Campaign(); err != nil { + log.VEventf(ctx, 1, "failed to campaign: %s", err) + } + } return true } // If the leader is not known, or if it is known but it's ineligible @@ -715,7 +724,7 @@ func (b *propBuf) leaderStatusRLocked(ctx context.Context, raftGroup proposerRaf !leaderInfo.iAmTheLeader { log.Fatalf(ctx, "inconsistent Raft state: state %s while the current replica is also the lead: %d", - raftGroup.Status().RaftState, leaderInfo.leader) + raftGroup.BasicStatus().RaftState, leaderInfo.leader) } return leaderInfo } @@ -1186,7 +1195,7 @@ func (rp *replicaProposer) leaderStatus( ) rangeLeaderInfo { r := (*Replica)(rp) - status := raftGroup.Status() + status := raftGroup.BasicStatus() iAmTheLeader := status.RaftState == raft.StateLeader leader := status.Lead leaderKnown := leader != raft.None @@ -1217,9 +1226,9 @@ func (rp *replicaProposer) leaderStatus( } } return rangeLeaderInfo{ + iAmTheLeader: iAmTheLeader, leaderKnown: leaderKnown, leader: roachpb.ReplicaID(leader), - iAmTheLeader: iAmTheLeader, leaderEligibleForLease: leaderEligibleForLease, } } @@ -1228,6 +1237,18 @@ func (rp *replicaProposer) ownsValidLease(ctx context.Context, now hlc.ClockTime return (*Replica)(rp).ownsValidLeaseRLocked(ctx, now) } +func (rp *replicaProposer) shouldCampaignOnRedirect(raftGroup proposerRaft) bool { + r := (*Replica)(rp) + livenessMap, _ := r.store.livenessMap.Load().(liveness.IsLiveMap) + return shouldCampaignOnLeaseRequestRedirect( + raftGroup.BasicStatus(), + livenessMap, + r.descRLocked(), + r.requiresExpiringLeaseRLocked(), + r.store.Clock().Now(), + ) +} + // rejectProposalWithRedirectLocked is part of the proposer interface. func (rp *replicaProposer) rejectProposalWithRedirectLocked( ctx context.Context, prop *ProposalData, redirectTo roachpb.ReplicaID, diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index a1174f7ad69c..6e01fc35af63 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -61,6 +61,8 @@ type testProposer struct { lease *roachpb.Lease, reason raftutil.ReplicaNeedsSnapshotStatus) // validLease is returned by ownsValidLease() validLease bool + // leaderNotLive is returned from shouldCampaignOnRedirect(). + leaderNotLive bool // leaderReplicaInDescriptor is set if the leader (as indicated by raftGroup) // is known, and that leader is part of the range's descriptor (as seen by the @@ -82,7 +84,8 @@ type testProposerRaft struct { status raft.Status // proposals are the commands that the propBuf flushed (i.e. passed to the // Raft group) and have not yet been consumed with consumeProposals(). - proposals []kvserverpb.RaftCommand + proposals []kvserverpb.RaftCommand + campaigned bool } var _ proposerRaft = &testProposerRaft{} @@ -113,11 +116,20 @@ func (t testProposerRaft) Status() raft.Status { return t.status } +func (t testProposerRaft) BasicStatus() raft.BasicStatus { + return t.status.BasicStatus +} + func (t testProposerRaft) ProposeConfChange(i raftpb.ConfChangeI) error { // TODO(andrei, nvanbenschoten): Capture the message and test against it. return nil } +func (t *testProposerRaft) Campaign() error { + t.campaigned = true + return nil +} + func (t *testProposer) locker() sync.Locker { return &t.RWMutex } @@ -174,7 +186,7 @@ func (t *testProposer) registerProposalLocked(p *ProposalData) { } func (t *testProposer) leaderStatus(ctx context.Context, raftGroup proposerRaft) rangeLeaderInfo { - lead := raftGroup.Status().Lead + lead := raftGroup.BasicStatus().Lead leaderKnown := lead != raft.None var leaderRep roachpb.ReplicaID var iAmTheLeader, leaderEligibleForLease bool @@ -200,9 +212,9 @@ func (t *testProposer) leaderStatus(ctx context.Context, raftGroup proposerRaft) } } return rangeLeaderInfo{ + iAmTheLeader: iAmTheLeader, leaderKnown: leaderKnown, leader: leaderRep, - iAmTheLeader: iAmTheLeader, leaderEligibleForLease: leaderEligibleForLease, } } @@ -211,6 +223,10 @@ func (t *testProposer) ownsValidLease(ctx context.Context, now hlc.ClockTimestam return t.validLease } +func (t *testProposer) shouldCampaignOnRedirect(raftGroup proposerRaft) bool { + return t.leaderNotLive +} + func (t *testProposer) rejectProposalWithRedirectLocked( _ context.Context, _ *ProposalData, redirectTo roachpb.ReplicaID, ) { @@ -493,10 +509,14 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) { // Set to simulate situations where the local replica is so behind that the // leader is not even part of the range descriptor. leaderNotInRngDesc bool + // Set to simulate situations where the Raft leader is not live in the node + // liveness map. + leaderNotLive bool // If true, the follower has a valid lease. ownsValidLease bool expRejection bool + expCampaign bool }{ { name: "leader", @@ -552,6 +572,17 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) { // FlushLockedWithRaftGroup(). expRejection: false, }, + { + name: "follower, known eligible non-live leader", + state: raft.StateFollower, + // Someone else is leader. + leader: self + 1, + leaderNotLive: true, + // Rejection - a follower can't request a lease. + expRejection: true, + // The leader is non-live, so we should campaign. + expCampaign: true, + }, } { t.Run(tc.name, func(t *testing.T) { var p testProposer @@ -585,6 +616,7 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) { p.leaderReplicaInDescriptor = !tc.leaderNotInRngDesc p.leaderReplicaType = tc.leaderRepType p.validLease = tc.ownsValidLease + p.leaderNotLive = tc.leaderNotLive var b propBuf clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */) @@ -601,6 +633,7 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) { } else { require.Equal(t, roachpb.ReplicaID(0), rejected) } + require.Equal(t, tc.expCampaign, r.campaigned) require.Zero(t, tracker.Count()) }) } diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index e082c0db604f..b3e00e2eee55 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -1878,7 +1878,7 @@ func shouldCampaignOnWake( if raftStatus.RaftState != raft.StateFollower { return false } - // If we dont know who the leader is, then campaign. + // If we don't know who the leader is, then campaign. if raftStatus.Lead == raft.None { return true } @@ -1926,6 +1926,75 @@ func (r *Replica) maybeCampaignOnWakeLocked(ctx context.Context) { } } +// shouldCampaignOnLeaseRequestRedirect returns whether a replica that is +// redirecting a lease request to its range's Raft leader should simultaneously +// campaign to acquire Raft leadership itself. A follower replica may want to +// campaign in such a case if it determines that the Raft leader is non-live +// according to node liveness despite being able to retain Raft leadership +// within the range. In such cases, the Raft leader will be unable to acquire an +// epoch-based lease until it heartbeats its liveness record, so it would be +// beneficial for one of the followers to step forward as leader/leaseholder. +// +// In these cases, campaigning for Raft leadership is safer than blindly +// allowing the lease request to be proposed (through a redirected proposal). +// This is because the follower may be arbitrarily far behind on its Raft log +// and acquiring the lease in such cases could cause unavailability. By instead +// calling a Raft pre-vote election, the follower can determine whether it is +// behind on its log without risking disruption. If not, it will eventually +// become leader and can proceed with a future attempt to acquire the lease. +func shouldCampaignOnLeaseRequestRedirect( + raftStatus raft.BasicStatus, + livenessMap liveness.IsLiveMap, + desc *roachpb.RangeDescriptor, + requiresExpiringLease bool, + now hlc.Timestamp, +) bool { + // If we're already campaigning don't start a new term. + if raftStatus.RaftState != raft.StateFollower { + return false + } + // If we don't know who the leader is, then campaign. + // NOTE: at the time of writing, we only reject lease requests and call this + // function when the leader is known, so this check is not needed. However, if + // that ever changes, and we do decide to reject a lease request when the + // leader is not known, we should immediately campaign. + if raftStatus.Lead == raft.None { + return true + } + // Avoid a circular dependency on liveness and skip the is leader alive check + // for ranges that always use expiration based leases. These ranges don't need + // to campaign based on liveness state because there can never be a case where + // a node can retain Raft leadership but still be unable to acquire the lease. + // This is possible on ranges that use epoch-based leases because the Raft + // leader may be partitioned from the liveness range. + // See TestRequestsOnFollowerWithNonLiveLeaseholder for an example of a test + // that demonstrates this case. + if requiresExpiringLease { + return false + } + // Determine if we think the leader is alive, if we don't have the leader in + // the descriptor we assume it is, since it could be an indication that this + // replica is behind. + replDesc, ok := desc.GetReplicaDescriptorByID(roachpb.ReplicaID(raftStatus.Lead)) + if !ok { + return false + } + // If we don't know about the leader in our liveness map, then we err on the + // side of caution and don't campaign. + livenessEntry, ok := livenessMap[replDesc.NodeID] + if !ok { + return false + } + // Otherwise, we check if the leader is live according to node liveness and + // campaign if it is not. + // NOTE: we intentionally do not look at the IsLiveMapEntry.IsLive field, + // which accounts for whether the leader is reachable from this node (see + // Store.updateLivenessMap). We only care whether the leader is currently live + // according to node liveness because this determines whether it will be able + // to acquire an epoch-based lease. + return !livenessEntry.Liveness.IsLive(now.GoTime()) +} + func (r *Replica) campaignLocked(ctx context.Context) { log.VEventf(ctx, 3, "campaigning") if err := r.mu.internalRaftGroup.Campaign(); err != nil { diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 531c2be0b34b..8dfaf5c43682 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -11668,6 +11668,117 @@ func TestReplicaShouldCampaignOnWake(t *testing.T) { } } +func TestReplicaShouldCampaignOnLeaseRequestRedirect(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + desc := roachpb.RangeDescriptor{ + RangeID: 1, + StartKey: roachpb.RKeyMin, + EndKey: roachpb.RKeyMax, + InternalReplicas: []roachpb.ReplicaDescriptor{ + { + ReplicaID: 1, + NodeID: 1, + StoreID: 1, + }, + { + ReplicaID: 2, + NodeID: 2, + StoreID: 2, + }, + { + ReplicaID: 3, + NodeID: 3, + StoreID: 3, + }, + }, + NextReplicaID: 4, + } + + now := hlc.Timestamp{WallTime: 100} + livenessMap := liveness.IsLiveMap{ + 1: liveness.IsLiveMapEntry{ + IsLive: true, + Liveness: livenesspb.Liveness{Expiration: now.Add(1, 0).ToLegacyTimestamp()}, + }, + 2: liveness.IsLiveMapEntry{ + // NOTE: we purposefully set IsLive to true in disagreement with the + // Liveness expiration to ensure that we're only looking at node liveness + // in shouldCampaignOnLeaseRequestRedirect and not at whether this node is + // reachable from the local node. + IsLive: true, + Liveness: livenesspb.Liveness{Expiration: now.Add(-1, 0).ToLegacyTimestamp()}, + }, + } + + followerWithoutLeader := raft.BasicStatus{ + SoftState: raft.SoftState{ + RaftState: raft.StateFollower, + Lead: 0, + }, + } + followerWithLeader := raft.BasicStatus{ + SoftState: raft.SoftState{ + RaftState: raft.StateFollower, + Lead: 1, + }, + } + candidate := raft.BasicStatus{ + SoftState: raft.SoftState{ + RaftState: raft.StateCandidate, + Lead: 0, + }, + } + leader := raft.BasicStatus{ + SoftState: raft.SoftState{ + RaftState: raft.StateLeader, + Lead: 1, + }, + } + followerDeadLeader := raft.BasicStatus{ + SoftState: raft.SoftState{ + RaftState: raft.StateFollower, + Lead: 2, + }, + } + followerMissingLiveness := raft.BasicStatus{ + SoftState: raft.SoftState{ + RaftState: raft.StateFollower, + Lead: 3, + }, + } + followerMissingDesc := raft.BasicStatus{ + SoftState: raft.SoftState{ + RaftState: raft.StateFollower, + Lead: 4, + }, + } + + tests := []struct { + name string + raftStatus raft.BasicStatus + requiresExpiringLease bool + exp bool + }{ + {"candidate", candidate, false, false}, + {"leader", leader, false, false}, + {"follower without leader", followerWithoutLeader, false, true}, + {"follower unknown leader", followerMissingDesc, false, false}, + {"follower expiration-based lease", followerDeadLeader, true, false}, + {"follower unknown liveness leader", followerMissingLiveness, false, false}, + {"follower live leader", followerWithLeader, false, false}, + {"follower dead leader", followerDeadLeader, false, true}, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + v := shouldCampaignOnLeaseRequestRedirect(tc.raftStatus, livenessMap, &desc, tc.requiresExpiringLease, now) + require.Equal(t, tc.exp, v) + }) + } +} + func TestRangeStatsRequest(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t)