Skip to content

Commit

Permalink
kvserver: don't refuse to fwd lease proposals in some edge cases
Browse files Browse the repository at this point in the history
This patch backpedals a little bit on the logic introduced in #55148.
That patch said that, if a leader is known, every other replica refuses
to propose a lease acquisition. Instead, the replica in question
redirects whomever was triggering the lease acquisition to the leader,
thinking that the leader should take the lease.
That patch introduced a deadlock: some replicas refuse to take the lease
because they are not VOTER_FULL (see CheckCanReceiveLease()). To fix the
deadlock, this patch incorporates that check in the proposal buffer's
decision about whether or not to reject a proposal: if the leader is
believed to refuse to take the lease, then we again forward our own
lease request.

An edge case to the edge case is when the leader is not even part of the
proposer's range descriptor. This can happen if the proposer is far
behind. In this case, we assume that the leader is eligible. If it
isn't, the deadlock will resolve once the proposer catches up.

A future patch will relax the conditions under which a replica agrees to
take the lease. VOTER_INCOMING replicas should take the lease.
VOTER_DEMOTING are more controversial.

Fixes #57798

Release note: None
  • Loading branch information
andreimatei committed Jan 14, 2021
1 parent 5e2e6d6 commit 230a2cc
Show file tree
Hide file tree
Showing 6 changed files with 193 additions and 31 deletions.
5 changes: 2 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,8 @@ func newFailedLeaseTrigger(isTransfer bool) result.Result {
// co-locate the leaseholder + raft leader, which is going to affect tail
// latencies. Additionally, as of the time of writing, learner replicas are
// only used for a short time in replica addition, so it's not worth working
// out the edge cases. If we decide to start using long-lived learners at some
// point, that math may change.
func checkCanReceiveLease(
// out the edge cases.
func CheckCanReceiveLease(
wouldbeLeaseholer roachpb.ReplicaID, rngDesc *roachpb.RangeDescriptor,
) error {
repDesc, ok := rngDesc.GetReplicaDescriptorByID(wouldbeLeaseholer)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ func RequestLease(

// If this check is removed at some point, the filtering of learners on the
// sending side would have to be removed as well.
if err := checkCanReceiveLease(args.Lease.Replica.ReplicaID, cArgs.EvalCtx.Desc()); err != nil {
if err := CheckCanReceiveLease(args.Lease.Replica.ReplicaID, cArgs.EvalCtx.Desc()); err != nil {
rErr.Message = err.Error()
return newFailedLeaseTrigger(false /* isTransfer */), rErr
}
Expand Down
33 changes: 33 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,3 +156,36 @@ func TestLeaseCommandLearnerReplica(t *testing.T) {
`replica (n2,s2):2LEARNER of type LEARNER cannot hold lease`
require.EqualError(t, err, expForLearner)
}

func TestCheckCanReceiveLease(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

for _, tc := range []struct {
leaseholderType roachpb.ReplicaType
eligible bool
}{
{leaseholderType: roachpb.VOTER_FULL, eligible: true},
{leaseholderType: roachpb.VOTER_INCOMING, eligible: false},
{leaseholderType: roachpb.VOTER_OUTGOING, eligible: false},
{leaseholderType: roachpb.VOTER_DEMOTING, eligible: false},
{leaseholderType: roachpb.LEARNER, eligible: false},
{leaseholderType: roachpb.NON_VOTER, eligible: false},
} {
t.Run(tc.leaseholderType.String(), func(t *testing.T) {
rngDesc := roachpb.RangeDescriptor{
InternalReplicas: []roachpb.ReplicaDescriptor{{
ReplicaID: 1,
Type: &tc.leaseholderType,
}},
}
err := CheckCanReceiveLease(1, &rngDesc)
require.Equal(t, tc.eligible, err == nil, "err: %v", err)
})
}

t.Run("replica not in range desc", func(t *testing.T) {
rngDesc := roachpb.RangeDescriptor{}
require.Regexp(t, "replica.*not found", CheckCanReceiveLease(1, &rngDesc))
})
}
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_transfer.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ func TransferLease(

// If this check is removed at some point, the filtering of learners on the
// sending side would have to be removed as well.
if err := checkCanReceiveLease(args.Lease.Replica.ReplicaID, cArgs.EvalCtx.Desc()); err != nil {
if err := CheckCanReceiveLease(args.Lease.Replica.ReplicaID, cArgs.EvalCtx.Desc()); err != nil {
return newFailedLeaseTrigger(true /* isTransfer */), err
}

Expand Down
94 changes: 74 additions & 20 deletions pkg/kv/kvserver/replica_proposal_buf.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"sync"
"sync/atomic"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
Expand Down Expand Up @@ -154,6 +155,21 @@ type propBuf struct {
}
}

type rangeLeaderInfo struct {
// 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
// leaderIneligibleForLease is set if the leader is known, but its type of
// replica prevents it from acquiring a lease.
leaderIneligibleForLease bool
}

// A proposer is an object that uses a propBuf to coordinate Raft proposals.
type proposer interface {
locker() sync.Locker
Expand All @@ -166,6 +182,7 @@ type proposer interface {
// The following require the proposer to hold an exclusive lock.
withGroupLocked(func(proposerRaft) error) error
registerProposalLocked(*ProposalData)
leaderStatusRLocked(raftGroup proposerRaft) rangeLeaderInfo
// rejectProposalWithRedirectLocked rejects a proposal and redirects the
// proposer to try it on another node. This is used to sometimes reject lease
// acquisitions when another replica is the leader; the intended consequence
Expand Down Expand Up @@ -430,19 +447,14 @@ func (b *propBuf) FlushLockedWithRaftGroup(

// Figure out leadership info. We'll use it to conditionally drop some
// requests.
var leaderKnown, iAmTheLeader bool
var leader roachpb.ReplicaID
var leaderInfo rangeLeaderInfo
if raftGroup != nil {
status := raftGroup.BasicStatus()
iAmTheLeader = status.RaftState == raft.StateLeader
leaderKnown = status.Lead != raft.None
if leaderKnown {
leader = roachpb.ReplicaID(status.Lead)
if !iAmTheLeader && leader == b.p.replicaID() {
log.Fatalf(ctx,
"inconsistent Raft state: state %s while the current replica is also the lead: %d",
status.RaftState, leader)
}
leaderInfo = b.p.leaderStatusRLocked(raftGroup)
// Sanity check.
if leaderInfo.leaderKnown && leaderInfo.leader == b.p.replicaID() && !leaderInfo.iAmTheLeader {
log.Fatalf(ctx,
"inconsistent Raft state: state %s while the current replica is also the lead: %d",
raftGroup.BasicStatus().RaftState, leaderInfo.leader)
}
}

Expand Down Expand Up @@ -484,14 +496,24 @@ func (b *propBuf) FlushLockedWithRaftGroup(
// ErrProposalDropped. We'll eventually re-propose it once a leader is
// known, at which point it will either go through or be rejected based on
// whether or not it is this replica that became the leader.
if !iAmTheLeader && p.Request.IsLeaseRequest() {
if leaderKnown && !b.testing.allowLeaseProposalWhenNotLeader {
//
// A special case is when the leader is known, but is ineligible to get the
// lease. In that case, we have no choice but to continue with the proposal.
if !leaderInfo.iAmTheLeader && p.Request.IsLeaseRequest() {
proposeAnyway := leaderInfo.leaderIneligibleForLease || b.testing.allowLeaseProposalWhenNotLeader
if leaderInfo.leaderKnown && !proposeAnyway {
log.VEventf(ctx, 2, "not proposing lease acquisition because we're not the leader; replica %d is",
leader)
b.p.rejectProposalWithRedirectLocked(ctx, p, leader)
leaderInfo.leader)
b.p.rejectProposalWithRedirectLocked(ctx, p, leaderInfo.leader)
continue
}
// If the leader is not known, continue with the proposal as explained above.
// If the leader is not known, or if it is known but it's ineligible for
// the lease, continue with the proposal as explained above.
if leaderInfo.leaderIneligibleForLease {
log.VEventf(ctx, 2, "proposing lease acquisition even though we're not the leader; the leader is ineligible")
} else {
log.VEventf(ctx, 2, "proposing lease acquisition even though we're not the leader; the leader is unknown")
}
}

// Raft processing bookkeeping.
Expand Down Expand Up @@ -721,18 +743,50 @@ func (rp *replicaProposer) registerProposalLocked(p *ProposalData) {
rp.mu.proposals[p.idKey] = p
}

func (rp *replicaProposer) leaderStatusRLocked(raftGroup proposerRaft) rangeLeaderInfo {
r := (*Replica)(rp)

status := raftGroup.BasicStatus()
iAmTheLeader := status.RaftState == raft.StateLeader
leader := status.Lead
leaderKnown := leader != raft.None
leaderIneligibleForLease := false
rangeDesc := r.descRLocked()
if leaderKnown {
// Figure out if the leader is eligible for getting a lease.
leaderRep, ok := rangeDesc.GetReplicaDescriptorByID(roachpb.ReplicaID(leader))
if !ok {
// There is a leader, but it's not part of our descriptor. The descriptor
// must be stale, so we are behind in applying the log. We don't want the
// lease ourselves (as we're behind), so let's assume that the leader is
// eligible. If it proves that it isn't, we might be asked to get the
// lease again, and by then hopefully we will have caught up.
leaderIneligibleForLease = false
} else {
err := batcheval.CheckCanReceiveLease(leaderRep.ReplicaID, rangeDesc)
leaderIneligibleForLease = err != nil
}
}
return rangeLeaderInfo{
leaderKnown: leaderKnown,
leader: roachpb.ReplicaID(leader),
iAmTheLeader: iAmTheLeader,
leaderIneligibleForLease: leaderIneligibleForLease,
}
}

// rejectProposalWithRedirectLocked is part of the proposer interface.
func (rp *replicaProposer) rejectProposalWithRedirectLocked(
ctx context.Context, prop *ProposalData, redirectTo roachpb.ReplicaID,
) {
r := (*Replica)(rp)
rangeDesc := r.descRLocked()
storeID := r.store.StoreID()
leaderRep, _ /* ok */ := rangeDesc.GetReplicaDescriptorByID(redirectTo)
redirectRep, _ /* ok */ := rangeDesc.GetReplicaDescriptorByID(redirectTo)
speculativeLease := &roachpb.Lease{
Replica: leaderRep,
Replica: redirectRep,
}
log.VEventf(ctx, 2, "redirecting proposal to node %s; request: %s", leaderRep.NodeID, prop.Request)
log.VEventf(ctx, 2, "redirecting proposal to node %s; request: %s", redirectRep.NodeID, prop.Request)
r.cleanupFailedProposalLocked(prop)
prop.finishApplication(ctx, proposalResult{
Err: roachpb.NewError(newNotLeaseHolderError(
Expand Down
88 changes: 82 additions & 6 deletions pkg/kv/kvserver/replica_proposal_buf_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -44,6 +45,15 @@ type testProposer struct {
// If not nil, this is called by RejectProposalWithRedirectLocked(). If nil,
// RejectProposalWithRedirectLocked() panics.
onRejectProposalWithRedirectLocked func(prop *ProposalData, redirectTo roachpb.ReplicaID)

// 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
// current replica). This can be used to simulate the local replica being so
// far behind that it doesn't have an up to date descriptor.
leaderReplicaInDescriptor bool
// If leaderReplicaInDescriptor is set, this specifies what type of replica it
// is. Some types of replicas are not eligible to get a lease.
leaderReplicaType roachpb.ReplicaType
}

type testProposerRaft struct {
Expand Down Expand Up @@ -99,6 +109,39 @@ func (t *testProposer) registerProposalLocked(p *ProposalData) {
t.registered++
}

func (t *testProposer) leaderStatusRLocked(raftGroup proposerRaft) rangeLeaderInfo {
leaderKnown := raftGroup.BasicStatus().Lead != raft.None
var leaderRep roachpb.ReplicaID
var iAmTheLeader, leaderIneligibleForLease bool
if leaderKnown {
leaderRep = roachpb.ReplicaID(raftGroup.BasicStatus().Lead)
iAmTheLeader = leaderRep == t.replicaID()
repDesc := roachpb.ReplicaDescriptor{
ReplicaID: leaderRep,
Type: &t.leaderReplicaType,
}

if t.leaderReplicaInDescriptor {
// Fill in a RangeDescriptor just enough for the CheckCanReceiveLease()
// call.
rngDesc := roachpb.RangeDescriptor{
InternalReplicas: []roachpb.ReplicaDescriptor{repDesc},
}
err := batcheval.CheckCanReceiveLease(repDesc.ReplicaID, &rngDesc)
leaderIneligibleForLease = err != nil
} else {
// This matches replicaProposed.leaderStatusRLocked().
leaderIneligibleForLease = false
}
}
return rangeLeaderInfo{
leaderKnown: leaderKnown,
leader: leaderRep,
iAmTheLeader: iAmTheLeader,
leaderIneligibleForLease: leaderIneligibleForLease,
}
}

func (t *testProposer) rejectProposalWithRedirectLocked(
ctx context.Context, prop *ProposalData, redirectTo roachpb.ReplicaID,
) {
Expand Down Expand Up @@ -391,9 +434,16 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) {
// Each subtest will try to propose a lease acquisition in a different Raft
// scenario. Some proposals should be allowed, some should be rejected.
for _, tc := range []struct {
name string
state raft.StateType
leader uint64
name string
state raft.StateType
// raft.None means there's no leader, or the leader is unknown.
leader uint64
// Empty means VOTER_FULL.
leaderRepType roachpb.ReplicaType
// Set to simulate situations where the local replica is so behind that the
// leader is not even part of the range descriptor.
leaderNotInRngDesc bool

expRejection bool
}{
{
Expand All @@ -404,15 +454,36 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) {
expRejection: false,
},
{
name: "follower known leader",
name: "follower, known eligible leader",
state: raft.StateFollower,
// Someone else is leader.
leader: self + 1,
// Rejection - a follower can't request a lease.
expRejection: true,
},
{
name: "follower unknown leader",
name: "follower, known ineligible leader",
state: raft.StateFollower,
// Someone else is leader.
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,
expRejection: false,
},
{
// Here we simulate the leader being known by Raft, but the local replica
// is so far behind that it doesn't contain the leader replica.
name: "follower, known leader not in range descriptor",
state: raft.StateFollower,
// Someone else is leader.
leader: self + 1,
leaderNotInRngDesc: true,
// We assume that the leader is eligible, and redirect.
expRejection: true,
},
{
name: "follower, unknown leader",
state: raft.StateFollower,
// Unknown leader.
leader: raft.None,
Expand Down Expand Up @@ -448,8 +519,13 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) {
Lead: tc.leader,
},
}
r := testProposerRaft{status: raftStatus}
r := testProposerRaft{
status: raftStatus,
}
p.raftGroup = r
p.leaderReplicaInDescriptor = !tc.leaderNotInRngDesc
p.leaderReplicaType = tc.leaderRepType

var b propBuf
b.Init(&p)

Expand Down

0 comments on commit 230a2cc

Please sign in to comment.