Skip to content

Commit

Permalink
kv: remove dependency on ticks from maybeDropMsgApp
Browse files Browse the repository at this point in the history
Related to #73838.

In d77bee9, we stopped ticking uninitialized replicas, so we can no longer use
ticks as a proxy for the age of a replica in the escape hatch of `maybeDropMsgApp`.
Instead, we now use the age of the replica directly. We hit the escape hatch for
any replica that is older than 20s, which corresponds to the 100 ticks we used
before.
  • Loading branch information
nvanbenschoten committed Dec 21, 2021
1 parent 63dfc14 commit 625ee8b
Show file tree
Hide file tree
Showing 4 changed files with 22 additions and 14 deletions.
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -214,6 +214,9 @@ type Replica struct {
// The writes to this key happen in Replica.setStartKeyLocked.
startKey roachpb.RKey

// creationTime is the time that the Replica struct was initially constructed.
creationTime time.Time

store *Store
abortSpan *abortspan.AbortSpan // Avoids anomalous reads after abort

Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/replica_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ func newUnloadedReplica(
r := &Replica{
AmbientContext: store.cfg.AmbientCtx,
RangeID: desc.RangeID,
creationTime: timeutil.Now(),
store: store,
abortSpan: abortspan.New(desc.RangeID),
concMgr: concurrency.NewManager(concurrency.Config{
Expand Down
21 changes: 12 additions & 9 deletions pkg/kv/kvserver/split_trigger_helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,23 +13,26 @@ package kvserver
import (
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"go.etcd.io/etcd/raft/v3/raftpb"
)

const maxDelaySplitTriggerTicks = 100
const maxDelaySplitTriggerDur = 20 * time.Second

type replicaMsgAppDropper Replica

func (rd *replicaMsgAppDropper) Args() (initialized bool, ticks int) {
func (rd *replicaMsgAppDropper) Args() (initialized bool, age time.Duration) {
r := (*Replica)(rd)
r.mu.RLock()
initialized = r.isInitializedRLocked()
ticks = r.mu.ticks
creationTime := r.creationTime
r.mu.RUnlock()
return initialized, ticks
age = timeutil.Since(creationTime)
return initialized, age
}

func (rd *replicaMsgAppDropper) ShouldDrop(
Expand All @@ -44,7 +47,7 @@ func (rd *replicaMsgAppDropper) ShouldDrop(
}

type msgAppDropper interface {
Args() (initialized bool, ticks int)
Args() (initialized bool, age time.Duration)
ShouldDrop(ctx context.Context, key roachpb.RKey) (fmt.Stringer, bool)
}

Expand All @@ -71,7 +74,7 @@ func maybeDropMsgApp(
// message via msg.Context. Check if this replica might be waiting for a
// split trigger. The first condition for that is not knowing the key
// bounds, i.e. not being initialized.
initialized, ticks := r.Args()
initialized, age := r.Args()

if initialized {
return false
Expand Down Expand Up @@ -135,16 +138,16 @@ func maybeDropMsgApp(
if verbose {
log.Infof(ctx, "start key is contained in replica %v", lhsRepl)
}
if ticks > maxDelaySplitTriggerTicks {
if age > maxDelaySplitTriggerDur {
// This is an escape hatch in case there are other scenarios (missed in
// the above analysis) in which a split trigger just isn't coming. If
// there are, the idea is that we notice this log message and improve
// the heuristics.
log.Warningf(
ctx,
"would have dropped incoming MsgApp to wait for split trigger, "+
"but allowing due to %d (>%d) ticks",
ticks, maxDelaySplitTriggerTicks)
"but allowing because uninitialized replica was created %s (>%s) ago",
age, maxDelaySplitTriggerDur)
return false
}
if verbose {
Expand Down
11 changes: 6 additions & 5 deletions pkg/kv/kvserver/split_trigger_helper_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
"fmt"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand All @@ -26,14 +27,14 @@ import (

type testMsgAppDropper struct {
initialized bool
ticks int
age time.Duration
lhs bool

startKey string // set by ShouldDrop
}

func (td *testMsgAppDropper) Args() (initialized bool, ticks int) {
return td.initialized, td.ticks
func (td *testMsgAppDropper) Args() (initialized bool, age time.Duration) {
return td.initialized, td.age
}

func (td *testMsgAppDropper) ShouldDrop(
Expand All @@ -58,9 +59,9 @@ func TestMaybeDropMsgApp(t *testing.T) {
// Drop message to wait for trigger.
{initialized: false, lhs: true}: true,
// Drop message to wait for trigger.
{initialized: false, lhs: true, ticks: maxDelaySplitTriggerTicks}: true,
{initialized: false, lhs: true, age: maxDelaySplitTriggerDur}: true,
// Escape hatch fires.
{initialized: false, lhs: true, ticks: maxDelaySplitTriggerTicks + 1}: false,
{initialized: false, lhs: true, age: maxDelaySplitTriggerDur + 1}: false,
}

msgHeartbeat := &raftpb.Message{
Expand Down

0 comments on commit 625ee8b

Please sign in to comment.