Skip to content

Commit 4d115e0

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. Fixes: #156537 Release note: None
1 parent e118836 commit 4d115e0

File tree

7 files changed

+357
-7
lines changed

7 files changed

+357
-7
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) (rangeID roachpb.RangeID, err 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
@@ -1328,8 +1328,10 @@ func splitTriggerHelper(
13281328
if err != nil {
13291329
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to fetch last replica GC timestamp")
13301330
}
1331+
13311332
if err := storage.MVCCPutProto(
1332-
ctx, batch, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{},
1333+
ctx, spanset.DisableForbiddenSpanAssertionsOnBatch(batch),
1334+
keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{},
13331335
&replicaGCTS, storage.MVCCWriteOptions{Category: fs.BatchEvalReadCategory}); err != nil {
13341336
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to copy last replica GC timestamp")
13351337
}
@@ -1529,7 +1531,8 @@ func splitTriggerHelper(
15291531
// as all replicas will be responsible for writing it locally before
15301532
// applying the split.
15311533
if !rec.ClusterSettings().Version.IsActive(ctx, clusterversion.V25_4_WriteInitialTruncStateBeforeSplitApplication) {
1532-
if err := kvstorage.WriteInitialTruncState(ctx, batch, split.RightDesc.RangeID); err != nil {
1534+
if err := kvstorage.WriteInitialTruncState(ctx,
1535+
spanset.DisableForbiddenSpanAssertionsOnBatch(batch), split.RightDesc.RangeID); err != nil {
15331536
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to write initial Replica state")
15341537
}
15351538
}

pkg/kv/kvserver/batcheval/cmd_truncate_log.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -118,7 +118,8 @@ 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+
ms, err := storage.ComputeStats(ctx,
122+
spanset.DisableForbiddenSpanAssertionsOnBatch(readWriter), start, end, 0 /* nowNanos */)
122123
if err != nil {
123124
return result.Result{}, errors.Wrap(err, "while computing stats of Raft log freed by truncation")
124125
}

pkg/kv/kvserver/replica_test.go

Lines changed: 209 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15197,3 +15197,212 @@ 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+
{
15213+
name: "point start key before unreplicated rangeID local span",
15214+
span: roachpb.Span{Key: keys.MakeRangeIDUnreplicatedPrefix(1).
15215+
Prevish(roachpb.PrevishKeyLength), EndKey: nil},
15216+
expectErr: false,
15217+
},
15218+
{
15219+
name: "point start key within unreplicated rangeID local span",
15220+
span: roachpb.Span{Key: keys.RaftTruncatedStateKey(1), EndKey: nil},
15221+
expectErr: true,
15222+
},
15223+
{
15224+
name: "point start key at unreplicated rangeID local span end",
15225+
span: roachpb.Span{Key: keys.MakeRangeIDUnreplicatedPrefix(1).
15226+
PrefixEnd(), EndKey: nil},
15227+
expectErr: false,
15228+
},
15229+
{
15230+
name: "point end key before unreplicated rangeID local span",
15231+
span: roachpb.Span{Key: nil, EndKey: keys.MakeRangeIDUnreplicatedPrefix(1)},
15232+
expectErr: false,
15233+
},
15234+
{
15235+
name: "point end key within unreplicated rangeID local span",
15236+
span: roachpb.Span{Key: nil, EndKey: keys.RaftTruncatedStateKey(1)},
15237+
expectErr: true,
15238+
},
15239+
{
15240+
name: "point end key at unreplicated rangeID local span end",
15241+
span: roachpb.Span{Key: nil, EndKey: keys.MakeRangeIDUnreplicatedPrefix(1).
15242+
PrefixEnd()},
15243+
expectErr: true,
15244+
},
15245+
{
15246+
name: "point end key after unreplicated rangeID local span",
15247+
span: roachpb.Span{Key: nil, EndKey: keys.MakeRangeIDUnreplicatedPrefix(1).
15248+
PrefixEnd().Next()},
15249+
expectErr: false,
15250+
},
15251+
{
15252+
name: "span fully before unreplicated rangeID local span",
15253+
span: roachpb.Span{Key: roachpb.KeyMin, EndKey: keys.MakeRangeIDUnreplicatedPrefix(0)},
15254+
expectErr: false,
15255+
},
15256+
{
15257+
name: "span fully after unreplicated rangeID local span",
15258+
span: roachpb.Span{Key: keys.LocalRangePrefix, EndKey: roachpb.KeyMax},
15259+
expectErr: false,
15260+
},
15261+
{
15262+
name: "span fully overlaps unreplicated rangeID local span",
15263+
span: roachpb.Span{Key: roachpb.KeyMin, EndKey: roachpb.KeyMax},
15264+
expectErr: true,
15265+
},
15266+
{
15267+
name: "span partially overlaps with start of unreplicated rangeID local span",
15268+
span: roachpb.Span{Key: roachpb.KeyMin, EndKey: keys.RaftTruncatedStateKey(1)},
15269+
expectErr: true,
15270+
},
15271+
{
15272+
name: "span partially overlaps with end of unreplicated rangeID local span",
15273+
span: roachpb.Span{Key: keys.RaftTruncatedStateKey(1), EndKey: roachpb.KeyMax},
15274+
expectErr: true,
15275+
},
15276+
{
15277+
name: "span within with unreplicated rangeID local span",
15278+
span: roachpb.Span{Key: keys.RaftTruncatedStateKey(1),
15279+
EndKey: keys.RangeLastReplicaGCTimestampKey(1)},
15280+
expectErr: true,
15281+
},
15282+
{
15283+
name: "span overlaps with multiple unreplicated range local span",
15284+
span: roachpb.Span{Key: keys.RaftTruncatedStateKey(1),
15285+
EndKey: keys.RaftTruncatedStateKey(3)},
15286+
expectErr: true,
15287+
},
15288+
{
15289+
name: "span from end of unreplicated RangeID, to next replicated RangeID",
15290+
span: roachpb.Span{Key: keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd(),
15291+
EndKey: keys.RangeVersionKey(2)},
15292+
expectErr: false,
15293+
},
15294+
{
15295+
name: "span from end of unreplicated RangeID, to next unreplicated RangeID",
15296+
span: roachpb.Span{Key: keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd(),
15297+
EndKey: keys.RangeLastReplicaGCTimestampKey(2)},
15298+
expectErr: true,
15299+
},
15300+
}
15301+
15302+
for _, tc := range testCases {
15303+
t.Run(tc.name, func(t *testing.T) {
15304+
err := overlapsUnreplicatedRangeIDLocalKeys(tc.span)
15305+
if tc.expectErr {
15306+
require.Errorf(t, err, "expected error for span %s", tc.span)
15307+
} else {
15308+
require.NoErrorf(t, err, "expected no error for span %s", tc.span)
15309+
}
15310+
})
15311+
}
15312+
}
15313+
15314+
// TestOverlapsStoreLocalKeys verifies that the function
15315+
// overlapsStoreLocalKeys() successfully catches any overlap with
15316+
// store local keys.
15317+
func TestOverlapsStoreLocalKeys(t *testing.T) {
15318+
defer leaktest.AfterTest(t)()
15319+
defer log.Scope(t).Close(t)
15320+
testCases := []struct {
15321+
name string
15322+
span roachpb.Span
15323+
expectErr bool
15324+
}{
15325+
{
15326+
name: "point start key before store local span",
15327+
span: roachpb.Span{Key: roachpb.Key(keys.LocalStorePrefix).
15328+
Prevish(roachpb.PrevishKeyLength), EndKey: nil},
15329+
expectErr: false,
15330+
},
15331+
{
15332+
name: "point start key at store local span start",
15333+
span: roachpb.Span{Key: keys.LocalStorePrefix, EndKey: nil},
15334+
expectErr: true,
15335+
},
15336+
{
15337+
name: "point start key within store local span",
15338+
span: roachpb.Span{Key: keys.StoreIdentKey(), EndKey: nil},
15339+
expectErr: true,
15340+
},
15341+
{
15342+
name: "point start key at store local span end",
15343+
span: roachpb.Span{Key: keys.LocalStoreMax, EndKey: nil},
15344+
expectErr: false,
15345+
},
15346+
{
15347+
name: "point end key before store local span",
15348+
span: roachpb.Span{Key: nil, EndKey: keys.LocalStorePrefix},
15349+
expectErr: false,
15350+
},
15351+
{
15352+
name: "point end key at store local span start",
15353+
span: roachpb.Span{Key: nil, EndKey: roachpb.Key(keys.LocalStorePrefix).Next()},
15354+
expectErr: true,
15355+
},
15356+
{
15357+
name: "point end key within store local span",
15358+
span: roachpb.Span{Key: nil, EndKey: keys.StoreIdentKey()},
15359+
expectErr: true,
15360+
},
15361+
{
15362+
name: "point end key at store local span end",
15363+
span: roachpb.Span{Key: nil, EndKey: keys.LocalStoreMax},
15364+
expectErr: true,
15365+
},
15366+
{
15367+
name: "span fully before store local span",
15368+
span: roachpb.Span{Key: roachpb.KeyMin, EndKey: keys.LocalStorePrefix},
15369+
expectErr: false,
15370+
},
15371+
{
15372+
name: "span fully after store local span",
15373+
span: roachpb.Span{Key: keys.LocalStoreMax, EndKey: roachpb.KeyMax},
15374+
expectErr: false,
15375+
},
15376+
{
15377+
name: "span fully overlaps store local span",
15378+
span: roachpb.Span{Key: roachpb.KeyMin, EndKey: roachpb.KeyMax},
15379+
expectErr: true,
15380+
},
15381+
{
15382+
name: "span partially overlaps with start of store local span",
15383+
span: roachpb.Span{Key: roachpb.KeyMin, EndKey: keys.StoreIdentKey()},
15384+
expectErr: true,
15385+
},
15386+
{
15387+
name: "span partially overlaps with end of store local span",
15388+
span: roachpb.Span{Key: keys.StoreIdentKey(), EndKey: roachpb.KeyMax},
15389+
expectErr: true,
15390+
},
15391+
{
15392+
name: "span within store local span",
15393+
span: roachpb.Span{Key: keys.StoreGossipKey(), EndKey: keys.StoreIdentKey()},
15394+
expectErr: true,
15395+
},
15396+
}
15397+
15398+
for _, tc := range testCases {
15399+
t.Run(tc.name, func(t *testing.T) {
15400+
err := overlapsStoreLocalKeys(tc.span)
15401+
if tc.expectErr {
15402+
require.Errorf(t, err, "expected error for span %s", tc.span)
15403+
} else {
15404+
require.NoErrorf(t, err, "expected no error for span %s", tc.span)
15405+
}
15406+
})
15407+
}
15408+
}

pkg/kv/kvserver/replica_write.go

Lines changed: 105 additions & 1 deletion
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"
@@ -814,8 +815,12 @@ func (r *Replica) newBatchedEngine(g *concurrency.Guard) (storage.Batch, *storag
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,102 @@ 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+
func overlapsUnreplicatedRangeIDLocalKeys(span roachpb.Span) error {
896+
// getRangeIDLocalSpan is a helper function that returns the full
897+
// unreplicated RangeID span for a given RangeID.
898+
getRangeIDLocalSpan := func(rangeID roachpb.RangeID) roachpb.Span {
899+
return roachpb.Span{Key: keys.MakeRangeIDUnreplicatedPrefix(rangeID),
900+
EndKey: keys.MakeRangeIDUnreplicatedPrefix(rangeID).PrefixEnd()}
901+
}
902+
903+
fullRangeIDLocalSpans := roachpb.Span{
904+
Key: keys.MakeRangeIDUnreplicatedPrefix(0),
905+
EndKey: keys.LocalRangePrefix,
906+
}
907+
908+
// If the provided span is completely outside the rangeID local spans for any
909+
// rangeID, then there is no overlap with any rangeID local keys.
910+
if !spanset.Overlaps(fullRangeIDLocalSpans, span) {
911+
return nil
912+
}
913+
914+
// Overlapping the full rangeID local span. We must be overlapping rangeID
915+
// local keys.
916+
if spanset.Contains(span, fullRangeIDLocalSpans) {
917+
return errors.Errorf("overlapping an unreplicated rangeID span")
918+
}
919+
920+
// At this point, we know that there is an overlap. If our span is not fully
921+
// inside fullRangeIDLocalSpans, then we must be partially overlapping with
922+
// RangeID local spans.
923+
if !spanset.Contains(fullRangeIDLocalSpans, span) {
924+
return errors.Errorf("partially overlapping an unreplicated rangeID span")
925+
}
926+
927+
// Our span is fully inside fullRangeIDLocalSpans.
928+
if span.Key == nil {
929+
if span.EndKey.Compare(fullRangeIDLocalSpans.EndKey) == 0 {
930+
return errors.Errorf("overlapping an unreplicated rangeID span")
931+
}
932+
933+
rangeID, err := keys.DecodeRangeIDPrefix(span.EndKey)
934+
if err != nil {
935+
panic(err) // cannot happen
936+
}
937+
938+
if spanset.Overlaps(getRangeIDLocalSpan(rangeID), span) {
939+
return errors.Errorf("overlapping an unreplicated rangeID span")
940+
}
941+
942+
return nil
943+
}
944+
945+
rangeID, err := keys.DecodeRangeIDPrefix(span.Key)
946+
if err != nil {
947+
panic(err) // cannot happen
948+
}
949+
950+
if spanset.Overlaps(getRangeIDLocalSpan(rangeID), span) {
951+
return errors.Errorf("overlapping an unreplicated rangeID span")
952+
}
953+
954+
// One last case to check. If the start key is just at the end of some RangeID
955+
// local key, we need to check if the span is overlapping some RangeID local
956+
// keys that are ahead of startKey (on the next RangeID).
957+
if span.EndKey != nil {
958+
endRangeID, err := keys.DecodeRangeIDPrefix(span.EndKey)
959+
if err != nil {
960+
panic(err) // cannot happen
961+
}
962+
963+
if endRangeID-rangeID > 1 {
964+
// If we are crossing more than one rangeID, we must have overlapped with
965+
// some unreplicated local RangeID keys.
966+
return errors.Errorf("overlapping an unreplicated rangeID span")
967+
}
968+
969+
if spanset.Overlaps(getRangeIDLocalSpan(endRangeID), span) {
970+
return errors.Errorf("overlapping an unreplicated rangeID span")
971+
}
972+
}
973+
974+
return nil
975+
}
976+
977+
// overlapsStoreLocalKeys returns an error if the provided span overlaps
978+
// with any store local keys.
979+
func overlapsStoreLocalKeys(span roachpb.Span) error {
980+
localStoreSpan := roachpb.Span{
981+
Key: keys.LocalStorePrefix,
982+
EndKey: keys.LocalStoreMax,
983+
}
984+
985+
if spanset.Overlaps(localStoreSpan, span) {
986+
return errors.Errorf("overlaps with store local keys")
987+
}
988+
989+
return nil
990+
}

0 commit comments

Comments
 (0)