Skip to content

Commit 6f5f0a9

Browse files
committed
spanset: assert that batches don't access store local and unreplicated RangeID local keys
This commit adds the following test-only assertions: 1) Generated batches don't touch store-local keys. 2) Generated batches don't touch unreplicated RangeID local keys. We disable the check in exactly 3 locations we know that we currently touch those keys.
1 parent 16d7a28 commit 6f5f0a9

File tree

6 files changed

+212
-6
lines changed

6 files changed

+212
-6
lines changed

pkg/keys/constants.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -163,6 +163,7 @@ var (
163163
//
164164
// LocalStorePrefix is the prefix identifying per-store data.
165165
LocalStorePrefix = makeKey(LocalPrefix, roachpb.Key("s"))
166+
LocalStoreMax = roachpb.Key(LocalStorePrefix).PrefixEnd()
166167
// localStoreClusterVersionSuffix stores the cluster-wide version
167168
// information for this store, updated any time the operator
168169
// updates the minimum cluster version.

pkg/keys/keys.go

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -308,6 +308,21 @@ func DecodeRangeIDKey(
308308
return roachpb.RangeID(rangeInt), infix, suffix, b, nil
309309
}
310310

311+
// DecodeRangeIDPrefix parses a local range ID prefix into range ID.
312+
func DecodeRangeIDPrefix(key roachpb.Key) (roachpb.RangeID, error) {
313+
if !bytes.HasPrefix(key, LocalRangeIDPrefix) {
314+
return 0, errors.Errorf("key %s does not have %s prefix", key, LocalRangeIDPrefix)
315+
}
316+
// Cut the prefix, the Range ID, and the infix specifier.
317+
b := key[len(LocalRangeIDPrefix):]
318+
_, rangeInt, err := encoding.DecodeUvarintAscending(b)
319+
if err != nil {
320+
return 0, err
321+
}
322+
323+
return roachpb.RangeID(rangeInt), nil
324+
}
325+
311326
// AbortSpanKey returns a range-local key by Range ID for an
312327
// AbortSpan entry, with detail specified by encoding the
313328
// supplied transaction ID.

pkg/kv/kvserver/batcheval/cmd_end_transaction.go

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1340,8 +1340,10 @@ func splitTriggerHelper(
13401340
if err != nil {
13411341
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to fetch last replica GC timestamp")
13421342
}
1343+
13431344
if err := storage.MVCCPutProto(
1344-
ctx, batch, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{},
1345+
ctx, spanset.DisableForbiddenSpanAssertionsOnBatch(batch),
1346+
keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{},
13451347
&replicaGCTS, storage.MVCCWriteOptions{Category: fs.BatchEvalReadCategory}); err != nil {
13461348
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to copy last replica GC timestamp")
13471349
}
@@ -1541,7 +1543,8 @@ func splitTriggerHelper(
15411543
// as all replicas will be responsible for writing it locally before
15421544
// applying the split.
15431545
if !rec.ClusterSettings().Version.IsActive(ctx, clusterversion.V25_4_WriteInitialTruncStateBeforeSplitApplication) {
1544-
if err := kvstorage.WriteInitialTruncState(ctx, batch, split.RightDesc.RangeID); err != nil {
1546+
if err := kvstorage.WriteInitialTruncState(ctx,
1547+
spanset.DisableForbiddenSpanAssertionsOnBatch(batch), split.RightDesc.RangeID); err != nil {
15451548
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to write initial Replica state")
15461549
}
15471550
}

pkg/kv/kvserver/batcheval/cmd_truncate_log.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -118,7 +118,9 @@ func TruncateLog(
118118
// are not tracked in the raft log delta. The delta will be adjusted below
119119
// raft.
120120
// We can pass zero as nowNanos because we're only interested in SysBytes.
121-
ms, err := storage.ComputeStats(ctx, readWriter, start, end, 0 /* nowNanos */)
121+
// TODO(#157895): Use the log engine here instead of the state machine engine.
122+
ms, err := storage.ComputeStats(ctx,
123+
spanset.DisableForbiddenSpanAssertionsOnBatch(readWriter), start, end, 0 /* nowNanos */)
122124
if err != nil {
123125
return result.Result{}, errors.Wrap(err, "while computing stats of Raft log freed by truncation")
124126
}

pkg/kv/kvserver/replica_test.go

Lines changed: 107 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15197,3 +15197,110 @@ func TestLeaderlessWatcherInit(t *testing.T) {
1519715197
t.Fatalf("expected LeaderlessWatcher channel to be closed")
1519815198
}
1519915199
}
15200+
15201+
// TestOverlapsUnreplicatedRangeIDLocalKeys verifies that the function
15202+
// overlapsUnreplicatedRangeIDLocalKeys() successfully catches any overlap with
15203+
// unreplicated rangeID local keys.
15204+
func TestOverlapsUnreplicatedRangeIDLocalKeys(t *testing.T) {
15205+
defer leaktest.AfterTest(t)()
15206+
defer log.Scope(t).Close(t)
15207+
testCases := []struct {
15208+
name string
15209+
span roachpb.Span
15210+
expectErr bool
15211+
}{
15212+
// Span before local RangeID.
15213+
{span: roachpb.Span{Key: roachpb.KeyMin, EndKey: keys.LocalRangeIDPrefix.AsRawKey()},
15214+
expectErr: false},
15215+
// Span after local RangeID.
15216+
{span: roachpb.Span{Key: keys.LocalRangePrefix, EndKey: roachpb.KeyMax},
15217+
expectErr: false},
15218+
// Span equals the full local RangeID span.
15219+
{span: roachpb.Span{Key: keys.LocalRangeIDPrefix.AsRawKey(),
15220+
EndKey: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd()},
15221+
expectErr: true},
15222+
// Span contains the full local RangeID span.
15223+
{span: roachpb.Span{Key: roachpb.KeyMin, EndKey: roachpb.KeyMax},
15224+
expectErr: true},
15225+
// Span partially overlaps the full RangeID span.
15226+
{span: roachpb.Span{Key: roachpb.KeyMin, EndKey: keys.RaftTruncatedStateKey(1)},
15227+
expectErr: true},
15228+
{span: roachpb.Span{Key: keys.RaftTruncatedStateKey(1), EndKey: roachpb.KeyMax},
15229+
expectErr: true},
15230+
// Span inside unreplicated local RangeID span.
15231+
{span: roachpb.Span{Key: keys.RangeTombstoneKey(1), EndKey: keys.RaftTruncatedStateKey(1)},
15232+
expectErr: true},
15233+
// Span inside replicated local RangeID span.
15234+
{span: roachpb.Span{Key: keys.RangeForceFlushKey(1), EndKey: keys.RangeLeaseKey(1)},
15235+
expectErr: false},
15236+
// Point span before local RangeID span.
15237+
{span: roachpb.Span{Key: keys.LocalRangeIDPrefix.AsRawKey().Prevish(1)}, expectErr: false},
15238+
// Point span after local RangeID span.
15239+
{span: roachpb.Span{Key: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd()}, expectErr: false},
15240+
// Point span inside local RangeID span.
15241+
{span: roachpb.Span{Key: keys.RaftTruncatedStateKey(1)}, expectErr: true},
15242+
// Point span with nil startKey before local RangeID span.
15243+
{span: roachpb.Span{EndKey: keys.LocalRangeIDPrefix.AsRawKey().Prevish(1)}, expectErr: false},
15244+
// Point span with nil startKey after local RangeID span.
15245+
{span: roachpb.Span{EndKey: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd().Next()}, expectErr: false},
15246+
}
15247+
15248+
for _, tc := range testCases {
15249+
t.Run(tc.name, func(t *testing.T) {
15250+
err := overlapsUnreplicatedRangeIDLocalKeys(spanset.TrickySpan(tc.span))
15251+
if tc.expectErr {
15252+
require.Errorf(t, err, "expected error for span %s", tc.span)
15253+
} else {
15254+
require.NoErrorf(t, err, "expected no error for span %s", tc.span)
15255+
}
15256+
})
15257+
}
15258+
}
15259+
15260+
// TestOverlapsStoreLocalKeys verifies that the function
15261+
// overlapsStoreLocalKeys() successfully catches any overlap with
15262+
// store local keys.
15263+
func TestOverlapsStoreLocalKeys(t *testing.T) {
15264+
defer leaktest.AfterTest(t)()
15265+
defer log.Scope(t).Close(t)
15266+
testCases := []struct {
15267+
name string
15268+
span roachpb.Span
15269+
expectErr bool
15270+
}{
15271+
// Span before store local span.
15272+
{span: roachpb.Span{Key: roachpb.KeyMin, EndKey: keys.LocalStorePrefix}, expectErr: false},
15273+
// Span after store local span.
15274+
{span: roachpb.Span{Key: keys.LocalStoreMax, EndKey: roachpb.KeyMax}, expectErr: false},
15275+
// Span Contains store local span.
15276+
{span: roachpb.Span{Key: roachpb.KeyMin, EndKey: roachpb.KeyMax}, expectErr: true},
15277+
// Span partially overlaps with store local span.
15278+
{span: roachpb.Span{Key: roachpb.KeyMin, EndKey: keys.StoreIdentKey()}, expectErr: true},
15279+
{span: roachpb.Span{Key: keys.StoreIdentKey(), EndKey: roachpb.KeyMax}, expectErr: true},
15280+
// Span overlaps with store local keys.
15281+
{span: roachpb.Span{Key: keys.StoreGossipKey(), EndKey: keys.StoreIdentKey()}, expectErr: true},
15282+
// Point span before store local span.
15283+
{span: roachpb.Span{Key: roachpb.Key(keys.LocalStorePrefix).Prevish(1)}, expectErr: false},
15284+
// Point span after store local span.
15285+
{span: roachpb.Span{Key: keys.LocalStoreMax}, expectErr: false},
15286+
// Point span inside store local span.
15287+
{span: roachpb.Span{Key: keys.StoreIdentKey()}, expectErr: true},
15288+
// Point span with nil startKey before store local span.
15289+
{span: roachpb.Span{EndKey: roachpb.Key(keys.LocalStorePrefix).Prevish(1)}, expectErr: false},
15290+
// Point span with nil startKey after store local span.
15291+
{span: roachpb.Span{EndKey: keys.LocalStoreMax.Next()}, expectErr: false},
15292+
// Point span with nil startKey inside store local span.
15293+
{span: roachpb.Span{EndKey: keys.LocalStoreMax}, expectErr: true},
15294+
}
15295+
15296+
for _, tc := range testCases {
15297+
t.Run(tc.name, func(t *testing.T) {
15298+
err := overlapsStoreLocalKeys(spanset.TrickySpan(tc.span))
15299+
if tc.expectErr {
15300+
require.Errorf(t, err, "expected error for span %s", tc.span)
15301+
} else {
15302+
require.NoErrorf(t, err, "expected no error for span %s", tc.span)
15303+
}
15304+
})
15305+
}
15306+
}

pkg/kv/kvserver/replica_write.go

Lines changed: 81 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@ import (
1010
"sync"
1111
"time"
1212

13+
"github.com/cockroachdb/cockroach/pkg/keys"
1314
"github.com/cockroachdb/cockroach/pkg/kv"
1415
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1516
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
@@ -24,7 +25,7 @@ import (
2425
"github.com/cockroachdb/cockroach/pkg/settings"
2526
"github.com/cockroachdb/cockroach/pkg/storage"
2627
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
27-
"github.com/cockroachdb/cockroach/pkg/util"
28+
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
2829
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2930
"github.com/cockroachdb/cockroach/pkg/util/log"
3031
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
@@ -808,14 +809,18 @@ func (r *Replica) newBatchedEngine(g *concurrency.Guard) (storage.Batch, *storag
808809
opLogger = storage.NewOpLoggerBatch(batch)
809810
batch = opLogger
810811
}
811-
if util.RaceEnabled {
812+
if buildutil.CrdbTestBuild {
812813
// During writes we may encounter a versioned value newer than the request
813814
// timestamp, and may have to retry at a higher timestamp. This is still
814815
// safe as we're only ever writing at timestamps higher than the timestamp
815816
// any write latch would be declared at. But because of this, we don't
816817
// assert on access timestamps using spanset.NewBatchAt.
817-
batch = spanset.NewBatch(batch, g.LatchSpans())
818+
spans := g.LatchSpans()
819+
spans.AddForbiddenMatcher(overlapsUnreplicatedRangeIDLocalKeys)
820+
spans.AddForbiddenMatcher(overlapsStoreLocalKeys)
821+
batch = spanset.NewBatch(batch, spans)
818822
}
823+
819824
return batch, opLogger
820825
}
821826

@@ -884,3 +889,76 @@ func releaseMVCCStats(ms *enginepb.MVCCStats) {
884889
ms.Reset()
885890
mvccStatsPool.Put(ms)
886891
}
892+
893+
// overlapsUnreplicatedRangeIDLocalKeys checks if the provided span overlaps
894+
// with any unreplicated rangeID local keys.
895+
// Note that we could receive the span with a nil startKey, which has a special
896+
// meaning that the span represents: [endKey.Prev(), endKey).
897+
func overlapsUnreplicatedRangeIDLocalKeys(span spanset.TrickySpan) error {
898+
// getRangeIDLocalSpan is a helper function that returns the full
899+
// unreplicated RangeID span for a given RangeID.
900+
getRangeIDLocalSpan := func(rangeID roachpb.RangeID) roachpb.Span {
901+
return roachpb.Span{Key: keys.MakeRangeIDUnreplicatedPrefix(rangeID),
902+
EndKey: keys.MakeRangeIDUnreplicatedPrefix(rangeID).PrefixEnd()}
903+
}
904+
905+
fullRangeIDLocalSpans := roachpb.Span{
906+
Key: keys.LocalRangeIDPrefix.AsRawKey(),
907+
EndKey: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(),
908+
}
909+
910+
// If the provided span is completely outside the rangeID local spans for any
911+
// rangeID, then there is no overlap with any rangeID local keys.
912+
if !spanset.Overlaps(fullRangeIDLocalSpans, span) {
913+
return nil
914+
}
915+
916+
// If we are partially overlapping the full rangeID local span. We must be
917+
// overlapping rangeID local keys.
918+
if spanset.PartiallyOverlaps(fullRangeIDLocalSpans, span) {
919+
return errors.Errorf("partially overlapping the unreplicated rangeID span")
920+
}
921+
922+
// At this point, we know that either we are containing the
923+
// fullRangeIDLocalSpans, or we are completely within fullRangeIDLocalSpans.
924+
// Check to see if we fully contain fullRangeIDLocalSpans.
925+
if !spanset.Contains(fullRangeIDLocalSpans, span) {
926+
return errors.Errorf("completely containing the unreplicated rangeID span")
927+
}
928+
929+
// We make an assumption here that whoever if the span in inside
930+
// fullRangeIDLocalSpans, we expect that (a) The start key is not nil, and
931+
// (b) both start and end keys should be in the same rangeID.
932+
if span.Key == nil {
933+
return errors.Errorf("unexpected nil start key")
934+
}
935+
936+
rangeID, err := keys.DecodeRangeIDPrefix(span.Key)
937+
if err != nil {
938+
return errors.NewAssertionErrorWithWrappedErrf(err,
939+
"could not decode range ID for key: %s", span.Key)
940+
}
941+
942+
if spanset.Overlaps(getRangeIDLocalSpan(rangeID), span) {
943+
return errors.Errorf("overlapping an unreplicated rangeID span")
944+
}
945+
946+
return nil
947+
}
948+
949+
// overlapsStoreLocalKeys returns an error if the provided span overlaps
950+
// with any store local keys.
951+
// Note that we could receive the span with a nil startKey, which has a special
952+
// meaning that the span represents: [endKey.Prev(), endKey).
953+
func overlapsStoreLocalKeys(span spanset.TrickySpan) error {
954+
localStoreSpan := roachpb.Span{
955+
Key: keys.LocalStorePrefix,
956+
EndKey: keys.LocalStoreMax,
957+
}
958+
959+
if spanset.Overlaps(localStoreSpan, span) {
960+
return errors.Errorf("overlaps with store local keys")
961+
}
962+
963+
return nil
964+
}

0 commit comments

Comments
 (0)