Skip to content

Commit

Permalink
Merge #103266
Browse files Browse the repository at this point in the history
103266: kvserver: delay range quiescence r=erikgrinaker a=erikgrinaker

This patch only quiesces ranges after 6 ticks (3 seconds) without any proposals, configurable via `COCKROACH_QUIESCE_AFTER_TICKS`. Unquiescence incurs a proposal, which has a non-negligible cost, and on low-latency clusters with steady write load this may otherwise (un)quiesce ranges very frequently, as often as every tick.

Resolves #63295.
Epic: none

Release note (performance improvement): ranges now only quiesce after 3 seconds without proposals, to avoid frequent unquiescence which incurs an additional Raft proposal. This is configurable via `COCKROACH_QUIESCE_AFTER_TICKS` which defaults to 6.

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
  • Loading branch information
craig[bot] and erikgrinaker committed May 18, 2023
2 parents 0621b3a + da5be73 commit e0c35b4
Show file tree
Hide file tree
Showing 6 changed files with 68 additions and 16 deletions.
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/replica.go
Expand Up @@ -41,7 +41,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/grunning"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -85,8 +84,6 @@ const (
defaultReplicaRaftMuWarnThreshold = 500 * time.Millisecond
)

var testingDisableQuiescence = envutil.EnvOrDefaultBool("COCKROACH_DISABLE_QUIESCENCE", false)

// StrictGCEnforcement controls whether requests are rejected based on the GC
// threshold and the current GC TTL (true) or just based on the GC threshold
// (false).
Expand Down Expand Up @@ -744,6 +741,9 @@ type Replica struct {
// Counts calls to Replica.tick()
ticks int

// lastProposalAtTicks tracks the time of the last proposal, in ticks.
lastProposalAtTicks int

// Counts Raft messages refused due to queue congestion.
droppedMessages int

Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/replica_proposal_buf.go
Expand Up @@ -1188,6 +1188,7 @@ func (rp *replicaProposer) registerProposalLocked(p *ProposalData) {
if p.createdAtTicks == 0 {
p.createdAtTicks = rp.mu.ticks
}
rp.mu.lastProposalAtTicks = rp.mu.ticks // monotonically increasing
if prev := rp.mu.proposals[p.idKey]; prev != nil && prev != p {
log.Fatalf(rp.store.AnnotateCtx(context.Background()), "two proposals under same ID:\n%+v,\n%+v", prev, p)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/replica_proposal_quota.go
Expand Up @@ -114,6 +114,7 @@ func (r *Replica) updateProposalQuotaRaftMuLocked(
)
r.mu.lastUpdateTimes = make(map[roachpb.ReplicaID]time.Time)
r.mu.lastUpdateTimes.updateOnBecomeLeader(r.mu.state.Desc.Replicas().Descriptors(), timeutil.Now())
r.mu.lastProposalAtTicks = r.mu.ticks // delay imminent quiescence
} else if r.mu.proposalQuota != nil {
// We're becoming a follower.
// We unblock all ongoing and subsequent quota acquisition goroutines
Expand Down
6 changes: 6 additions & 0 deletions pkg/kv/kvserver/replica_raft.go
Expand Up @@ -553,6 +553,12 @@ func (r *Replica) hasPendingProposalQuotaRLocked() bool {
return !r.mu.proposalQuota.Full()
}

// ticksSinceLastProposalRLocked returns the number of ticks since the last
// proposal.
func (r *Replica) ticksSinceLastProposalRLocked() int {
return r.mu.ticks - r.mu.lastProposalAtTicks
}

// isRaftLeader returns true if this replica believes it is the current
// Raft leader.
//
Expand Down
18 changes: 18 additions & 0 deletions pkg/kv/kvserver/replica_raft_quiesce.go
Expand Up @@ -20,12 +20,22 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"go.etcd.io/raft/v3"
"go.etcd.io/raft/v3/raftpb"
)

// quiesceAfterTicks is the number of ticks without proposals after which ranges
// should quiesce. Unquiescing incurs a raft proposal which has a non-neglible
// cost, and low-latency clusters may otherwise (un)quiesce very frequently,
// e.g. on every tick.
var quiesceAfterTicks = envutil.EnvOrDefaultInt("COCKROACH_QUIESCE_AFTER_TICKS", 6)

// testingDisableQuiescence disables replica quiescence.
var testingDisableQuiescence = envutil.EnvOrDefaultBool("COCKROACH_DISABLE_QUIESCENCE", false)

func (r *Replica) quiesceLocked(ctx context.Context, lagging laggingReplicaSet) {
if !r.mu.quiescent {
if log.V(3) {
Expand Down Expand Up @@ -91,6 +101,7 @@ func (r *Replica) maybeUnquiesceAndWakeLeaderLocked() bool {
// Propose an empty command which will wake the leader.
data := raftlog.EncodeRaftCommand(raftlog.EntryEncodingStandardWithoutAC, makeIDKey(), nil)
_ = r.mu.internalRaftGroup.Propose(data)
r.mu.lastProposalAtTicks = r.mu.ticks // delay imminent quiescence
return true
}

Expand Down Expand Up @@ -201,6 +212,7 @@ type quiescer interface {
hasRaftReadyRLocked() bool
hasPendingProposalsRLocked() bool
hasPendingProposalQuotaRLocked() bool
ticksSinceLastProposalRLocked() int
mergeInProgressRLocked() bool
isDestroyedRLocked() (DestroyReason, error)
}
Expand Down Expand Up @@ -287,6 +299,12 @@ func shouldReplicaQuiesce(
}
return nil, nil, false
}
if ticks := q.ticksSinceLastProposalRLocked(); ticks < quiesceAfterTicks {
if log.V(4) {
log.Infof(ctx, "not quiescing: proposed %d ticks ago", ticks)
}
return nil, nil, false
}
// Fast path: don't quiesce expiration-based leases, since they'll likely be
// renewed soon. The lease may not be ours, but in that case we wouldn't be
// able to quiesce anyway (see leaseholder condition below).
Expand Down
52 changes: 39 additions & 13 deletions pkg/kv/kvserver/replica_test.go
Expand Up @@ -9958,17 +9958,18 @@ func TestApplyPaginatedCommittedEntries(t *testing.T) {
}

type testQuiescer struct {
st *cluster.Settings
storeID roachpb.StoreID
desc roachpb.RangeDescriptor
numProposals int
pendingQuota bool
status *raftSparseStatus
lastIndex kvpb.RaftIndex
raftReady bool
leaseStatus kvserverpb.LeaseStatus
mergeInProgress bool
isDestroyed bool
st *cluster.Settings
storeID roachpb.StoreID
desc roachpb.RangeDescriptor
numProposals int
pendingQuota bool
ticksSinceLastProposal int
status *raftSparseStatus
lastIndex kvpb.RaftIndex
raftReady bool
leaseStatus kvserverpb.LeaseStatus
mergeInProgress bool
isDestroyed bool

// Not used to implement quiescer, but used by tests.
livenessMap livenesspb.IsLiveMap
Expand Down Expand Up @@ -10015,6 +10016,10 @@ func (q *testQuiescer) hasPendingProposalQuotaRLocked() bool {
return q.pendingQuota
}

func (q *testQuiescer) ticksSinceLastProposalRLocked() int {
return q.ticksSinceLastProposal
}

func (q *testQuiescer) mergeInProgressRLocked() bool {
return q.mergeInProgress
}
Expand Down Expand Up @@ -10065,8 +10070,9 @@ func TestShouldReplicaQuiesce(t *testing.T) {
3: {Match: logIndex},
},
},
lastIndex: logIndex,
raftReady: false,
lastIndex: logIndex,
raftReady: false,
ticksSinceLastProposal: quiesceAfterTicks,
leaseStatus: kvserverpb.LeaseStatus{
State: kvserverpb.LeaseState_VALID,
Lease: roachpb.Lease{
Expand Down Expand Up @@ -10113,6 +10119,26 @@ func TestShouldReplicaQuiesce(t *testing.T) {
q.pendingQuota = true
return q
})
test(true, func(q *testQuiescer) *testQuiescer {
q.ticksSinceLastProposal = quiesceAfterTicks // quiesce on quiesceAfterTicks
return q
})
test(true, func(q *testQuiescer) *testQuiescer {
q.ticksSinceLastProposal = quiesceAfterTicks + 1 // quiesce above quiesceAfterTicks
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.ticksSinceLastProposal = quiesceAfterTicks - 1 // don't quiesce below quiesceAfterTicks
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.ticksSinceLastProposal = 0 // don't quiesce on 0
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.ticksSinceLastProposal = -1 // don't quiesce on negative (shouldn't happen)
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.mergeInProgress = true
return q
Expand Down

0 comments on commit e0c35b4

Please sign in to comment.