diff --git a/pkg/ccl/crosscluster/replicationutils/utils.go b/pkg/ccl/crosscluster/replicationutils/utils.go index a408b6ce78c6..d5089fe68549 100644 --- a/pkg/ccl/crosscluster/replicationutils/utils.go +++ b/pkg/ccl/crosscluster/replicationutils/utils.go @@ -122,9 +122,11 @@ func ScanSST( intersectedSpan := scanWithin.Intersect(rangeIter.RangeBounds()) mergeRangeKV(storage.MVCCRangeKeyValue{ RangeKey: storage.MVCCRangeKey{ - StartKey: intersectedSpan.Key.Clone(), - EndKey: intersectedSpan.EndKey.Clone(), - Timestamp: rangeKeyVersion.Timestamp}, + StartKey: intersectedSpan.Key.Clone(), + EndKey: intersectedSpan.EndKey.Clone(), + Timestamp: rangeKeyVersion.Timestamp, + EncodedTimestampSuffix: storage.EncodeMVCCTimestampSuffix(rangeKeyVersion.Timestamp), + }, Value: rangeKeyVersion.Value, }) } diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 5525b227c4b3..fb9261e10f51 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -1266,6 +1266,7 @@ func TestEvalAddSSTable(t *testing.T) { require.NoError(t, err) v.LocalTimestamp.WallTime *= 1e9 kv.RangeKey.Timestamp.WallTime *= 1e9 + kv.RangeKey.EncodedTimestampSuffix = storage.EncodeMVCCTimestampSuffix(kv.RangeKey.Timestamp) vBytes, err := storage.EncodeMVCCValue(v) require.NoError(t, err) expect = append(expect, storage.MVCCRangeKeyValue{RangeKey: kv.RangeKey, Value: vBytes}) diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go index 8dd1daf42f64..d4c15c84ff09 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go @@ -202,10 +202,12 @@ func TestDeleteRangeTombstone(t *testing.T) { writeInitialData(t, ctx, engine) + ts := hlc.Timestamp{WallTime: tc.ts} rangeKey := storage.MVCCRangeKey{ - StartKey: roachpb.Key(tc.start), - EndKey: roachpb.Key(tc.end), - Timestamp: hlc.Timestamp{WallTime: tc.ts}, + StartKey: roachpb.Key(tc.start), + EndKey: roachpb.Key(tc.end), + Timestamp: ts, + EncodedTimestampSuffix: storage.EncodeMVCCTimestampSuffix(ts), } // Prepare the request and environment. diff --git a/pkg/kv/kvserver/batcheval/knobs_use_range_tombstones_test.go b/pkg/kv/kvserver/batcheval/knobs_use_range_tombstones_test.go index 4b8d599b32ce..a0f2c3949ecc 100644 --- a/pkg/kv/kvserver/batcheval/knobs_use_range_tombstones_test.go +++ b/pkg/kv/kvserver/batcheval/knobs_use_range_tombstones_test.go @@ -85,6 +85,7 @@ func TestKnobsUseRangeTombstonesForPointDeletes(t *testing.T) { switch kv := kvI.(type) { case storage.MVCCRangeKeyValue: kv.RangeKey.Timestamp = hlc.Timestamp{} + kv.RangeKey.EncodedTimestampSuffix = nil rangeTombstones = append(rangeTombstones, kv.RangeKey) case storage.MVCCKeyValue: diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index 280981aecfc6..1a96a8d02161 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -782,9 +782,10 @@ func mergeRanges(fragments [][]storage.MVCCRangeKeyValue) []storage.MVCCRangeKey // tombstone types. newPartial = append(newPartial, storage.MVCCRangeKeyValue{ RangeKey: storage.MVCCRangeKey{ - StartKey: partialRangeKeys[j].RangeKey.StartKey, - EndKey: stack[i].RangeKey.EndKey, - Timestamp: partialRangeKeys[j].RangeKey.Timestamp, + StartKey: partialRangeKeys[j].RangeKey.StartKey, + EndKey: stack[i].RangeKey.EndKey, + Timestamp: partialRangeKeys[j].RangeKey.Timestamp, + EncodedTimestampSuffix: partialRangeKeys[j].RangeKey.EncodedTimestampSuffix, }, Value: partialRangeKeys[j].Value, }) diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index 3dd9a3550611..64d68ca96220 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -1512,9 +1512,10 @@ func engineData(t *testing.T, r storage.Reader, desc roachpb.RangeDescriptor) [] i++ case 0: newPartial = append(newPartial, storage.MVCCRangeKey{ - StartKey: partialRangeKeys[j].StartKey, - EndKey: newKeys[i].EndKey.Clone(), - Timestamp: partialRangeKeys[j].Timestamp, + StartKey: partialRangeKeys[j].StartKey, + EndKey: newKeys[i].EndKey.Clone(), + Timestamp: partialRangeKeys[j].Timestamp, + EncodedTimestampSuffix: partialRangeKeys[j].EncodedTimestampSuffix, }) i++ j++ diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 803c7f243e44..01bdd098dad2 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -99,7 +99,7 @@ var ( // increment [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] [targetLockConflictBytes=] k= [inc=] // initput [t=] [ts=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] k= v= [raw] [failOnTombstones] // put [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] k= v= [raw] -// put_rangekey ts=[,] [localTs=[,]] k= end= +// put_rangekey ts=[,] [localTs=[,]] k= end= [syntheticBit] // put_blind_inline k= v= [prev=] // get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [skipLocked] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [maxKeys=] [targetBytes=] [allowEmpty] // scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [skipLocked] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [wholeRows[=]] [allowEmpty] @@ -1917,6 +1917,24 @@ func cmdPutRangeKey(e *evalCtx) error { var value storage.MVCCValue value.MVCCValueHeader.LocalTimestamp = hlc.ClockTimestamp(e.getTsWithName("localTs")) + // If the syntheticBit arg is present, manually construct a MVCC timestamp + // that includes the synthetic bit. Cockroach stopped writing these keys + // beginning in version 24.1. It's not possible to commit such a key through + // the PutMVCCRangeKey API, so we also need to manually encode the MVCC + // value and use PutEngineRangeKey. We keep the non-synthetic-bit case + // as-is, using PutMVCCRangeKey, since that's the codepath ordinary MVCC + // range key writes will use and we want to exercise it. See #129592. + if e.hasArg("syntheticBit") { + return e.withWriter("put_rangekey", func(rw storage.ReadWriter) error { + suffix := storage.EncodeMVCCTimestampSuffixWithSyntheticBitForTesting(rangeKey.Timestamp) + valueRaw, err := storage.EncodeMVCCValue(value) + if err != nil { + return errors.Wrapf(err, "failed to encode MVCC value for range key %s", rangeKey) + } + return rw.PutEngineRangeKey(rangeKey.StartKey, rangeKey.EndKey, suffix, valueRaw) + }) + } + return e.withWriter("put_rangekey", func(rw storage.ReadWriter) error { return rw.PutMVCCRangeKey(rangeKey, value) }) diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 77d72e2e015e..b74466a83bc5 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -14,6 +14,7 @@ import ( "bytes" "encoding/binary" "fmt" + "slices" "sort" "strings" @@ -385,6 +386,20 @@ type MVCCRangeKey struct { StartKey roachpb.Key EndKey roachpb.Key Timestamp hlc.Timestamp + // EncodedTimestampSuffix is an optional encoded representation of Timestamp + // as a Pebble "suffix". When reading range keys from the engine, the + // iterator copies the verbatim encoded timestamp here. There historically + // have been multiple representations of a timestamp that were intended to + // be logically equivalent. A bug in CockroachDB's pebble.Comparer + // implementation prevented some encodings from being considered equivalent. + // See #129592. + // + // To work around this wart within the comparer, we preserve a copy of the + // physical encoded timestamp we read off the engine. If a MVCCRangeKey with + // a non-empty EncodedTimestampSuffix is cleared via ClearMVCCRangeKey, the + // RangeKeyUnset tombstone is written with the verbatim + // EncodedTimestampSuffix. + EncodedTimestampSuffix []byte } // AsStack returns the range key as a range key stack with the given value. @@ -392,8 +407,9 @@ func (k MVCCRangeKey) AsStack(valueRaw []byte) MVCCRangeKeyStack { return MVCCRangeKeyStack{ Bounds: k.Bounds(), Versions: MVCCRangeKeyVersions{{ - Timestamp: k.Timestamp, - Value: valueRaw, + Timestamp: k.Timestamp, + Value: valueRaw, + EncodedTimestampSuffix: k.EncodedTimestampSuffix, }}, } } @@ -408,6 +424,7 @@ func (k MVCCRangeKey) Clone() MVCCRangeKey { // k is already a copy, but byte slices must be cloned. k.StartKey = k.StartKey.Clone() k.EndKey = k.EndKey.Clone() + k.EncodedTimestampSuffix = slices.Clone(k.EncodedTimestampSuffix) return k } @@ -507,6 +524,20 @@ type MVCCRangeKeyVersions []MVCCRangeKeyVersion type MVCCRangeKeyVersion struct { Timestamp hlc.Timestamp Value []byte + // EncodedTimestampSuffix is an optional encoded representation of Timestamp + // as a Pebble "suffix". When reading range keys from the engine, the + // iterator copies the verbatim encoded timestamp here. There historically + // have been multiple representations of a timestamp that were intended to + // be logically equivalent. A bug in CockroachDB's pebble.Comparer + // implementation prevented some encodings from being considered equivalent. + // See #129592. + // + // To work around this wart within the comparer, we preserve a copy of the + // physical encoded timestamp we read off the engine. If a MVCCRangeKey with + // a non-empty EncodedTimestampSuffix is cleared via ClearMVCCRangeKey, the + // RangeKeyUnset tombstone is written with the verbatim + // EncodedTimestampSuffix. + EncodedTimestampSuffix []byte } // CloneInto copies the version into the provided destination @@ -514,15 +545,17 @@ type MVCCRangeKeyVersion struct { func (v MVCCRangeKeyVersion) CloneInto(dst *MVCCRangeKeyVersion) { dst.Timestamp = v.Timestamp dst.Value = append(dst.Value[:0], v.Value...) + dst.EncodedTimestampSuffix = append(dst.EncodedTimestampSuffix[:0], v.EncodedTimestampSuffix...) } // AsRangeKey returns an MVCCRangeKey for the given version. Byte slices // are shared with the stack. func (s MVCCRangeKeyStack) AsRangeKey(v MVCCRangeKeyVersion) MVCCRangeKey { return MVCCRangeKey{ - StartKey: s.Bounds.Key, - EndKey: s.Bounds.EndKey, - Timestamp: v.Timestamp, + StartKey: s.Bounds.Key, + EndKey: s.Bounds.EndKey, + Timestamp: v.Timestamp, + EncodedTimestampSuffix: v.EncodedTimestampSuffix, } } @@ -707,6 +740,7 @@ func (v MVCCRangeKeyVersions) CloneInto(c *MVCCRangeKeyVersions) { for i := range v { (*c)[i].Timestamp = v[i].Timestamp (*c)[i].Value = append((*c)[i].Value[:0], v[i].Value...) + (*c)[i].EncodedTimestampSuffix = append((*c)[i].EncodedTimestampSuffix[:0], v[i].EncodedTimestampSuffix...) } } @@ -898,6 +932,9 @@ func (v MVCCRangeKeyVersion) Clone() MVCCRangeKeyVersion { if v.Value != nil { v.Value = append([]byte(nil), v.Value...) } + if v.EncodedTimestampSuffix != nil { + v.EncodedTimestampSuffix = append([]byte(nil), v.EncodedTimestampSuffix...) + } return v } @@ -910,3 +947,30 @@ func (v MVCCRangeKeyVersion) Equal(o MVCCRangeKeyVersion) bool { func (v MVCCRangeKeyVersion) String() string { return fmt.Sprintf("%s=%x", v.Timestamp, v.Value) } + +// EncodeMVCCTimestampSuffixWithSyntheticBitForTesting is a utility to encode +// the provided timestamp as a MVCC timestamp key suffix with the synthetic bit +// set. The synthetic bit is no longer encoded/decoded into the hlc.Timestamp +// but may exist in existing databases. This utility allows a test to construct +// a timestamp with the synthetic bit for testing appropriate handling of +// existing keys with the bit set. It should only be used in tests. See #129592. +// +// TODO(jackson): Remove this function when we've migrated all keys to unset the +// synthetic bit. +func EncodeMVCCTimestampSuffixWithSyntheticBitForTesting(ts hlc.Timestamp) []byte { + const mvccEncodedTimestampWithSyntheticBitLen = mvccEncodedTimeWallLen + + mvccEncodedTimeLogicalLen + + mvccEncodedTimeSyntheticLen + + mvccEncodedTimeLengthLen + suffix := make([]byte, mvccEncodedTimestampWithSyntheticBitLen) + encodeMVCCTimestampToBuf(suffix, ts) + suffix[len(suffix)-2] = 0x01 // Synthetic bit. + suffix[len(suffix)-1] = mvccEncodedTimestampWithSyntheticBitLen + if decodedTS, err := DecodeMVCCTimestampSuffix(suffix); err != nil { + panic(err) + } else if !ts.Equal(decodedTS) { + panic(errors.AssertionFailedf("manufactured MVCC timestamp with synthetic bit decoded to %s not %s", + ts, decodedTS)) + } + return suffix +} diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index 595e592d0740..69b996271e14 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -422,6 +422,8 @@ func TestMVCCRangeKeyClone(t *testing.T) { orig := MVCCRangeKeyStack{ Bounds: roachpb.Span{Key: roachpb.Key("abc"), EndKey: roachpb.Key("def")}, Versions: MVCCRangeKeyVersions{ + {Timestamp: hlc.Timestamp{WallTime: 5, Logical: 1}, Value: nil, + EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(hlc.Timestamp{WallTime: 5, Logical: 1})}, {Timestamp: hlc.Timestamp{WallTime: 3, Logical: 4}, Value: []byte{1, 2, 3}}, {Timestamp: hlc.Timestamp{WallTime: 1, Logical: 2}, Value: nil}, }, @@ -438,6 +440,10 @@ func TestMVCCRangeKeyClone(t *testing.T) { if len(orig.Versions[i].Value) > 0 { require.NotSame(t, &orig.Versions[i].Value[0], &clone.Versions[i].Value[0]) } + if len(orig.Versions[i].EncodedTimestampSuffix) > 0 { + require.NotSame(t, &orig.Versions[i].EncodedTimestampSuffix[0], + &clone.Versions[i].EncodedTimestampSuffix[0]) + } } } @@ -448,7 +454,8 @@ func TestMVCCRangeKeyCloneInto(t *testing.T) { Bounds: roachpb.Span{Key: roachpb.Key("abc"), EndKey: roachpb.Key("def")}, Versions: MVCCRangeKeyVersions{ {Timestamp: hlc.Timestamp{WallTime: 3, Logical: 4}, Value: []byte{1, 2, 3}}, - {Timestamp: hlc.Timestamp{WallTime: 1, Logical: 2}, Value: nil}, + {Timestamp: hlc.Timestamp{WallTime: 1, Logical: 2}, Value: nil, + EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(hlc.Timestamp{WallTime: 1, Logical: 2})}, }, } @@ -475,7 +482,7 @@ func TestMVCCRangeKeyCloneInto(t *testing.T) { Versions: MVCCRangeKeyVersions{ {Value: make([]byte, len(orig.Versions[0].Value)+1)}, {Value: make([]byte, len(orig.Versions[1].Value)+1)}, - {Value: make([]byte, 100)}, + {Value: make([]byte, 100), EncodedTimestampSuffix: make([]byte, 10)}, }, } @@ -524,6 +531,7 @@ func TestMVCCRangeKeyCloneInto(t *testing.T) { requireSliceIdentity(t, orig.Bounds.EndKey, clone.Bounds.EndKey, false) for i := range orig.Versions { requireSliceIdentity(t, orig.Versions[i].Value, clone.Versions[i].Value, false) + requireSliceIdentity(t, orig.Versions[i].EncodedTimestampSuffix, clone.Versions[i].EncodedTimestampSuffix, false) } // Assert whether the clone is reusing byte slices from the target. @@ -532,6 +540,7 @@ func TestMVCCRangeKeyCloneInto(t *testing.T) { for i := range tc.target.Versions { if i < len(clone.Versions) { requireSliceIdentity(t, tc.target.Versions[i].Value, clone.Versions[i].Value, tc.expectReused) + requireSliceIdentity(t, tc.target.Versions[i].EncodedTimestampSuffix, clone.Versions[i].EncodedTimestampSuffix, tc.expectReused) } } }) @@ -564,13 +573,14 @@ func TestMVCCRangeKeyString(t *testing.T) { func TestMVCCRangeKeyCompare(t *testing.T) { defer leaktest.AfterTest(t)() - ab1 := MVCCRangeKey{roachpb.Key("a"), roachpb.Key("b"), hlc.Timestamp{Logical: 1}} - ac1 := MVCCRangeKey{roachpb.Key("a"), roachpb.Key("c"), hlc.Timestamp{Logical: 1}} - ac2 := MVCCRangeKey{roachpb.Key("a"), roachpb.Key("c"), hlc.Timestamp{Logical: 2}} - bc0 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("c"), hlc.Timestamp{Logical: 0}} - bc1 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("c"), hlc.Timestamp{Logical: 1}} - bc3 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("c"), hlc.Timestamp{Logical: 3}} - bd4 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("d"), hlc.Timestamp{Logical: 4}} + ab1 := MVCCRangeKey{roachpb.Key("a"), roachpb.Key("b"), hlc.Timestamp{Logical: 1}, nil} + ac1 := MVCCRangeKey{roachpb.Key("a"), roachpb.Key("c"), hlc.Timestamp{Logical: 1}, nil} + ac2 := MVCCRangeKey{roachpb.Key("a"), roachpb.Key("c"), hlc.Timestamp{Logical: 2}, nil} + bc0 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("c"), hlc.Timestamp{Logical: 0}, nil} + bc1 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("c"), hlc.Timestamp{Logical: 1}, nil} + bc3 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("c"), hlc.Timestamp{Logical: 3}, nil} + bd4 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("d"), hlc.Timestamp{Logical: 4}, nil} + bd5 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("d"), hlc.Timestamp{Logical: 4}, []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4, 0, 14}} testcases := map[string]struct { a MVCCRangeKey @@ -588,6 +598,7 @@ func TestMVCCRangeKeyCompare(t *testing.T) { "set time gt empty": {bc1, bc0, 1}, // empty MVCC timestamps sort before non-empty "start time precedence": {ac2, bc3, -1}, // a before b, but 3 before 2; key takes precedence "time end precedence": {bd4, bc3, -1}, // c before d, but 4 before 3; time takes precedence + "ignore encoded ts": {bd4, bd5, 0}, // encoded timestamp is ignored } for name, tc := range testcases { t.Run(name, func(t *testing.T) { @@ -1041,9 +1052,10 @@ func pointKV(key string, ts int, value string) MVCCKeyValue { func rangeKey(start, end string, ts int) MVCCRangeKey { return MVCCRangeKey{ - StartKey: roachpb.Key(start), - EndKey: roachpb.Key(end), - Timestamp: wallTS(ts), + StartKey: roachpb.Key(start), + EndKey: roachpb.Key(end), + Timestamp: wallTS(ts), + EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(wallTS(ts)), } } diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index bbe325d9f37c..0c20236e0f21 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -5375,7 +5375,8 @@ func (d rangeTestData) populateEngine( } ts = v.point.Key.Timestamp } else { - require.NoError(t, MVCCDeleteRangeUsingTombstone(ctx, engine, ms, v.rangeTombstone.StartKey, + rw := mvccRangeKeyEncodedTimestampReadWriter{engine} + require.NoError(t, MVCCDeleteRangeUsingTombstone(ctx, rw, ms, v.rangeTombstone.StartKey, v.rangeTombstone.EndKey, v.rangeTombstone.Timestamp, hlc.ClockTimestamp{}, nil, nil, false, 0, 0, nil), "failed to insert range tombstone into engine (%s)", v.rangeTombstone.String()) ts = v.rangeTombstone.Timestamp @@ -5384,6 +5385,39 @@ func (d rangeTestData) populateEngine( return ts } +// mvccRangeKeyEncodedTimestampReadWriter wraps a ReadWriter, overriding +// PutMVCCRangeKey so that if MVCCRangeKey.EncodedTimestampSuffix is non-empty, +// the resulting range key uses the encoded suffix verbatim. This is a bit of a +// subtle and convoluted test construction, but it allows testing of range keys +// with the synthetic bit without needing to bring back knowledge of the +// synthetic bit to the various MVCC key encoding and decoding routines. +// +// Note that all production ReadWriter implementations only use +// EncodedTimestampSuffix in their ClearMVCCRangeKey implementations, not in +// their PutMVCCRangeKey implementations. +// +// TODO(jackson): Remove this when we've guaranteed that all range keys have +// timestamps without the synthetic bit. +type mvccRangeKeyEncodedTimestampReadWriter struct { + ReadWriter +} + +func (w mvccRangeKeyEncodedTimestampReadWriter) PutMVCCRangeKey( + rk MVCCRangeKey, v MVCCValue, +) error { + if len(rk.EncodedTimestampSuffix) == 0 { + return w.ReadWriter.PutMVCCRangeKey(rk, v) + } + valueRaw, err := EncodeMVCCValue(v) + if err != nil { + return errors.Wrapf(err, "failed to encode MVCC value for range key %s", rk) + } else if err := rk.Validate(); err != nil { + return err + } + return w.ReadWriter.PutEngineRangeKey( + rk.StartKey, rk.EndKey, rk.EncodedTimestampSuffix, valueRaw) +} + // pt creates a point update for key with default value. func pt(key roachpb.Key, ts hlc.Timestamp) rangeTestDataItem { val := roachpb.MakeValueFromString("testval").RawBytes @@ -5482,7 +5516,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyB, EndKey: keyC, Timestamp: ts4}, + {StartKey: keyB, EndKey: keyC, Timestamp: ts4, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts4)}, }, }, { @@ -5506,8 +5540,8 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, - {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, + {StartKey: keyA, EndKey: keyB, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5529,7 +5563,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyA, EndKey: keyC, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5541,7 +5575,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + {StartKey: keyA, EndKey: keyB, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5553,8 +5587,8 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, - {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, + {StartKey: keyA, EndKey: keyB, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5566,7 +5600,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, + {StartKey: keyB, EndKey: keyD, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5588,7 +5622,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + {StartKey: keyA, EndKey: keyB, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5611,7 +5645,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, + {StartKey: keyB, EndKey: keyC, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5624,7 +5658,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + {StartKey: keyA, EndKey: keyB, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5637,7 +5671,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyC, Timestamp: ts2}, + {StartKey: keyA, EndKey: keyC, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5650,7 +5684,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + {StartKey: keyA, EndKey: keyB, Timestamp: ts2, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts2)}, }, }, { @@ -5751,11 +5785,11 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyB, Timestamp: ts3}, - {StartKey: keyA, EndKey: keyB, Timestamp: ts1}, - {StartKey: keyB, EndKey: keyC, Timestamp: ts3}, - {StartKey: keyC, EndKey: keyD, Timestamp: ts3}, - {StartKey: keyC, EndKey: keyD, Timestamp: ts1}, + {StartKey: keyA, EndKey: keyB, Timestamp: ts3, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts3)}, + {StartKey: keyA, EndKey: keyB, Timestamp: ts1, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts1)}, + {StartKey: keyB, EndKey: keyC, Timestamp: ts3, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts3)}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts3, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts3)}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts1, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts1)}, }, }, { @@ -5768,7 +5802,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyD, Timestamp: ts3}, + {StartKey: keyA, EndKey: keyD, Timestamp: ts3, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts3)}, }, }, { @@ -5781,7 +5815,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyC, Timestamp: ts3}, + {StartKey: keyA, EndKey: keyC, Timestamp: ts3, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts3)}, }, }, { @@ -5794,7 +5828,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyA, EndKey: keyB, Timestamp: ts1}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyD, Timestamp: ts3}, + {StartKey: keyA, EndKey: keyD, Timestamp: ts3, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts3)}, }, }, { @@ -5810,8 +5844,8 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyD, EndKey: keyE, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyF, Timestamp: ts4}, - {StartKey: keyA, EndKey: keyF, Timestamp: ts3}, + {StartKey: keyA, EndKey: keyF, Timestamp: ts4, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts4)}, + {StartKey: keyA, EndKey: keyF, Timestamp: ts3, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts3)}, }, }, { @@ -5826,7 +5860,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ - {StartKey: keyA, EndKey: keyE, Timestamp: ts3}, + {StartKey: keyA, EndKey: keyE, Timestamp: ts3, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts3)}, }, }, { @@ -5842,11 +5876,26 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { after: []MVCCRangeKey{ // We only iterate data within range, so range keys would be // truncated. - {StartKey: keyB, EndKey: keyC, Timestamp: ts4}, + {StartKey: keyB, EndKey: keyC, Timestamp: ts4, EncodedTimestampSuffix: EncodeMVCCTimestampSuffix(ts4)}, }, rangeStart: keyB, rangeEnd: keyC, }, + { + name: "synthetic bit range keys", + before: rangeTestData{ + rangeTestDataItem{rangeTombstone: MVCCRangeKey{ + StartKey: keyA, + EndKey: keyC, + Timestamp: ts4, + EncodedTimestampSuffix: EncodeMVCCTimestampSuffixWithSyntheticBitForTesting(ts4), + }}, + }, + request: []kvpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyC, Timestamp: ts4}, + }, + after: []MVCCRangeKey{}, + }, } for _, d := range testData { diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 3c22e6c13fd0..cfc360309637 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1929,6 +1929,13 @@ func (p *Pebble) ClearMVCCRangeKey(rangeKey MVCCRangeKey) error { if err := rangeKey.Validate(); err != nil { return err } + // If the range key holds an encoded timestamp as it was read from storage, + // write the tombstone to clear it using the same encoding of the timestamp. + // See #129592. + if len(rangeKey.EncodedTimestampSuffix) > 0 { + return p.ClearEngineRangeKey( + rangeKey.StartKey, rangeKey.EndKey, rangeKey.EncodedTimestampSuffix) + } return p.ClearEngineRangeKey( rangeKey.StartKey, rangeKey.EndKey, EncodeMVCCTimestampSuffix(rangeKey.Timestamp)) } diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index e1ade9bb735c..c0681e1296c0 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -184,6 +184,13 @@ func (wb *writeBatch) ClearMVCCRangeKey(rangeKey MVCCRangeKey) error { if err := rangeKey.Validate(); err != nil { return err } + // If the range key holds an encoded timestamp as it was read from storage, + // write the tombstone to clear it using the same encoding of the timestamp. + // See #129592. + if len(rangeKey.EncodedTimestampSuffix) > 0 { + return wb.ClearEngineRangeKey( + rangeKey.StartKey, rangeKey.EndKey, rangeKey.EncodedTimestampSuffix) + } return wb.ClearEngineRangeKey( rangeKey.StartKey, rangeKey.EndKey, EncodeMVCCTimestampSuffix(rangeKey.Timestamp)) } @@ -211,6 +218,9 @@ func (wb *writeBatch) PutRawMVCCRangeKey(rangeKey MVCCRangeKey, value []byte) er if err := rangeKey.Validate(); err != nil { return err } + // NB: We deliberately do not use rangeKey.EncodedTimestampSuffix even if + // it's present, because we explicitly do NOT want to write range keys with + // the synthetic bit set. return wb.PutEngineRangeKey( rangeKey.StartKey, rangeKey.EndKey, EncodeMVCCTimestampSuffix(rangeKey.Timestamp), value) } @@ -751,8 +761,7 @@ func (p *pebbleBatch) ClearMVCCIteratorRange( startRaw := EncodeMVCCKey(MVCCKey{Key: rangeKeys.Bounds.Key}) endRaw := EncodeMVCCKey(MVCCKey{Key: rangeKeys.Bounds.EndKey}) for _, v := range rangeKeys.Versions { - if err := p.batch.RangeKeyUnset(startRaw, endRaw, - EncodeMVCCTimestampSuffix(v.Timestamp), nil); err != nil { + if err := p.batch.RangeKeyUnset(startRaw, endRaw, v.EncodedTimestampSuffix, nil); err != nil { return err } } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index 55a62165893f..6ee5b332e1e1 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -764,8 +764,9 @@ func (p *pebbleIterator) RangeKeys() MVCCRangeKeyStack { continue } stack.Versions = append(stack.Versions, MVCCRangeKeyVersion{ - Timestamp: timestamp, - Value: rangeKey.Value, + Timestamp: timestamp, + Value: rangeKey.Value, + EncodedTimestampSuffix: rangeKey.Suffix, }) } return stack diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 097beb83718a..c4689047f850 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -264,6 +264,13 @@ func (fw *SSTWriter) ClearMVCCRangeKey(rangeKey MVCCRangeKey) error { if err := rangeKey.Validate(); err != nil { return err } + // If the range key holds an encoded timestamp as it was read from storage, + // write the tombstone to clear it using the same encoding of the timestamp. + // See #129592. + if len(rangeKey.EncodedTimestampSuffix) > 0 { + return fw.ClearEngineRangeKey(rangeKey.StartKey, rangeKey.EndKey, + rangeKey.EncodedTimestampSuffix) + } return fw.ClearEngineRangeKey(rangeKey.StartKey, rangeKey.EndKey, EncodeMVCCTimestampSuffix(rangeKey.Timestamp)) } diff --git a/pkg/storage/testdata/mvcc_histories/range_key_clear_synthetic_bit b/pkg/storage/testdata/mvcc_histories/range_key_clear_synthetic_bit new file mode 100644 index 000000000000..ee43cbde18ee --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_key_clear_synthetic_bit @@ -0,0 +1,41 @@ +# Write MVCC range deletes, one with the synthetic bit and one without. + +run ok +put_rangekey k=a end=c ts=1 +put_rangekey k=l end=m ts=2 syntheticBit +---- +>> at end: +rangekey: {a-c}/[1.000000000,0=/] +rangekey: {l-m}/[2.000000000,0=/] + +# Use MVCCClearTimeRange to clear the tombstones. The range key with the +# synthetic bit should still be removed. + +run stats ok +clear_time_range k=a end=z ts=7 targetTs=0 +---- +>> clear_time_range k=a end=z ts=7 targetTs=0 +stats: range_key_count=-2 range_key_bytes=-26 range_val_count=-2 gc_bytes_age=-2561 +>> at end: + +stats: + +# Re-write MVCC range deletes, one with the synthetic bit and one without. + +run ok +put_rangekey k=a end=c ts=1 +put_rangekey k=l end=m ts=2 syntheticBit +---- +>> at end: +rangekey: {a-c}/[1.000000000,0=/] +rangekey: {l-m}/[2.000000000,0=/] + +# Clear the range. This will metamorphically use ClearMVCCIteratorRange which +# will unset individual range keys, otherwise using ClearMVCCRange which will +# write a single RANGEKEYDEL. + +run ok +clear_range k=a end=z +---- +>> at end: + diff --git a/pkg/testutils/storageutils/kv.go b/pkg/testutils/storageutils/kv.go index 7e5d52c3de04..ae620fc3ba0c 100644 --- a/pkg/testutils/storageutils/kv.go +++ b/pkg/testutils/storageutils/kv.go @@ -88,9 +88,10 @@ func RangeKey(start, end string, ts int) storage.MVCCRangeKey { // RangeKeyWithTS creates an MVCCRangeKey for the given string key and timestamp. func RangeKeyWithTS(start, end string, ts hlc.Timestamp) storage.MVCCRangeKey { return storage.MVCCRangeKey{ - StartKey: roachpb.Key(start), - EndKey: roachpb.Key(end), - Timestamp: ts, + StartKey: roachpb.Key(start), + EndKey: roachpb.Key(end), + Timestamp: ts, + EncodedTimestampSuffix: storage.EncodeMVCCTimestampSuffix(ts), } }