Skip to content

Commit

Permalink
kvserver: add GlobalMVCCRangeTombstone testing knob
Browse files Browse the repository at this point in the history
This patch adds a set of testing knobs to enable writing an MVCC range
tombstone at the bottom of the keyspace during cluster bootstrapping.
This also subsumes the previous mechanism to trigger this from the
envvar `COCKROACH_GLOBAL_MVCC_RANGE_TOMBSTONE`, for e.g. roachtests and
roachperf benchmarks.

Release justification: non-production code changes

Release note: None
  • Loading branch information
erikgrinaker committed Aug 29, 2022
1 parent 3b16435 commit 31947b8
Show file tree
Hide file tree
Showing 9 changed files with 50 additions and 16 deletions.
1 change: 0 additions & 1 deletion pkg/kv/kvclient/rangefeed/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ go_library(
"//pkg/keys",
"//pkg/kv",
"//pkg/kv/kvclient/kvcoord",
"//pkg/kv/kvserver/kvserverbase",
"//pkg/roachpb",
"//pkg/settings",
"//pkg/settings/cluster",
Expand Down
9 changes: 7 additions & 2 deletions pkg/kv/kvclient/rangefeed/rangefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
Expand Down Expand Up @@ -83,6 +82,12 @@ type TestingKnobs struct {

// OnRangefeedRestart is called when a rangefeed restarts.
OnRangefeedRestart func()

// IgnoreOnDeleteRangeError will ignore any errors where a DeleteRange event
// is emitted without an OnDeleteRange handler. This can be used e.g. with
// StoreTestingKnobs.GlobalMVCCRangeTombstone, to prevent the global tombstone
// causing rangefeed errors for consumers who don't expect it.
IgnoreOnDeleteRangeError bool
}

// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
Expand Down Expand Up @@ -361,7 +366,7 @@ func (f *RangeFeed) processEvents(
f.onSSTable(ctx, ev.SST, ev.RegisteredSpan)
case ev.DeleteRange != nil:
if f.onDeleteRange == nil {
if kvserverbase.GlobalMVCCRangeTombstoneForTesting {
if f.knobs.IgnoreOnDeleteRangeError {
continue
}
return errors.AssertionFailedf(
Expand Down
3 changes: 1 addition & 2 deletions pkg/kv/kvserver/batcheval/cmd_init_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
)
Expand All @@ -33,7 +32,7 @@ func InitPut(
args := cArgs.Args.(*roachpb.InitPutRequest)
h := cArgs.Header

if args.FailOnTombstones && kvserverbase.GlobalMVCCRangeTombstoneForTesting {
if args.FailOnTombstones && cArgs.EvalCtx.EvalKnobs().DisableInitPutFailOnTombstones {
args.FailOnTombstones = false
}

Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvserver/kvserverbase/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ go_library(
"//pkg/kv/kvserver/kvserverpb",
"//pkg/roachpb",
"//pkg/settings",
"//pkg/util/envutil",
"//pkg/util/errorutil",
"//pkg/util/hlc",
"//pkg/util/quotapool",
Expand Down
8 changes: 0 additions & 8 deletions pkg/kv/kvserver/kvserverbase/base.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
Expand Down Expand Up @@ -219,10 +218,3 @@ var SplitByLoadMergeDelay = settings.RegisterDurationSetting(
// MaxCommandSizeDefault is the default for the kv.raft.command.max_size
// cluster setting.
const MaxCommandSizeDefault = 64 << 20

// GlobalMVCCRangeTombstoneForTesting will write an MVCC range tombstone at the
// bottom of the SQL table data keyspace during cluster bootstrapping, for
// performance and correctness testing. This shouldn't affect data written above
// it, but activates range key-specific code paths in the storage layer.
var GlobalMVCCRangeTombstoneForTesting = envutil.EnvOrDefaultBool(
"COCKROACH_GLOBAL_MVCC_RANGE_TOMBSTONE", false)
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/kvserverbase/knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,11 @@ type BatchEvalTestingKnobs struct {
// default, this is not allowed because it is unsafe. See cmd_gc.go for an
// explanation of why.
AllowGCWithNewThresholdAndKeys bool

// DisableInitPutFailOnTombstones disables FailOnTombstones for InitPut. This
// is useful together with e.g. StoreTestingKnobs.GlobalMVCCRangeTombstone,
// where we still want InitPut to succeed on top of the range tombstone.
DisableInitPutFailOnTombstones bool
}

// IntentResolverTestingKnobs contains testing helpers that are used during
Expand Down
3 changes: 1 addition & 2 deletions pkg/kv/kvserver/store_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
Expand Down Expand Up @@ -204,7 +203,7 @@ func WriteInitialClusterData(

// If requested, write an MVCC range tombstone at the bottom of the
// keyspace, for performance and correctness testing.
if kvserverbase.GlobalMVCCRangeTombstoneForTesting {
if knobs.GlobalMVCCRangeTombstone {
if err := writeGlobalMVCCRangeTombstone(ctx, batch, desc, now.Prev()); err != nil {
return err
}
Expand Down
12 changes: 12 additions & 0 deletions pkg/kv/kvserver/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -422,6 +422,18 @@ type StoreTestingKnobs struct {

// EnqueueReplicaInterceptor intercepts calls to `store.Enqueue()`.
EnqueueReplicaInterceptor func(queueName string, replica *Replica)

// GlobalMVCCRangeTombstone will write a global MVCC range tombstone across
// the entire user keyspace during cluster bootstrapping. This will be written
// below all other data, and thus won't affect query results, but it does
// activate MVCC range tombstone code paths in the storage layer for testing.
//
// This must typically be combined with the following knobs to prevent
// various components choking on the range tombstone:
//
// - rangefeed.TestingKnobs.IgnoreOnDeleteRangeError
// - kvserverbase.BatchEvalTestingKnobs.DisableInitPutFailOnTombstones
GlobalMVCCRangeTombstone bool
}

// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
Expand Down
24 changes: 24 additions & 0 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/docs"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/status"
Expand Down Expand Up @@ -224,9 +225,32 @@ func MakeBaseConfig(st *cluster.Settings, tr *tracing.Tracer) BaseConfig {
// in a tag that is prefixed with "nsql".
baseCfg.AmbientCtx.AddLogTag("n", baseCfg.IDContainer)
baseCfg.InitDefaults()
baseCfg.InitTestingKnobs()

return baseCfg
}

// InitTestingKnobs sets up any testing knobs based on e.g. envvars.
func (cfg *BaseConfig) InitTestingKnobs() {
// If requested, write an MVCC range tombstone at the bottom of the SQL table
// data keyspace during cluster bootstrapping, for performance and correctness
// testing. This shouldn't affect data written above it, but activates range
// key-specific code paths in the storage layer. We'll also have to tweak
// rangefeeds and batcheval to not choke on it.
if envutil.EnvOrDefaultBool("COCKROACH_GLOBAL_MVCC_RANGE_TOMBSTONE", false) {
if cfg.TestingKnobs.Store == nil {
cfg.TestingKnobs.Store = &kvserver.StoreTestingKnobs{}
}
if cfg.TestingKnobs.RangeFeed == nil {
cfg.TestingKnobs.RangeFeed = &rangefeed.TestingKnobs{}
}
storeKnobs := cfg.TestingKnobs.Store.(*kvserver.StoreTestingKnobs)
storeKnobs.GlobalMVCCRangeTombstone = true
storeKnobs.EvalKnobs.DisableInitPutFailOnTombstones = true
cfg.TestingKnobs.RangeFeed.(*rangefeed.TestingKnobs).IgnoreOnDeleteRangeError = true
}
}

// Config holds the parameters needed to set up a combined KV and SQL server.
type Config struct {
BaseConfig
Expand Down

0 comments on commit 31947b8

Please sign in to comment.