diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index e3339338bb87..f0596ccee6ac 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -91,22 +91,33 @@ var ( // check_for_acquire_lock t= k= str= [maxLockConflicts=] [targetLockConflictBytes=] // acquire_lock t= k= str= [maxLockConflicts=] [targetLockConflictBytes=] // -// cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] [targetLockConflictBytes=] k= v= [raw] [cond=] -// del [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] [targetLockConflictBytes=] k= -// del_range [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] [targetLockConflictBytes=] k= end= [max=] [returnKeys] -// del_range_ts [ts=[,]] [localTs=[,]] [maxLockConflicts=] [targetLockConflictBytes=] k= end= [idempotent] [noCoveredStats] -// del_range_pred [ts=[,]] [localTs=[,]] [maxLockConflicts=] [targetLockConflictBytes=] k= end= [startTime=,max=,maxBytes=,rangeThreshold=] -// 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] +// cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] +// [targetLockConflictBytes=] k= v= [raw] [cond=] +// del [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] +// [targetLockConflictBytes=] k= +// del_range [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] +// [targetLockConflictBytes=] k= end= [max=] [returnKeys] +// del_range_ts [ts=[,]] [localTs=[,]] [maxLockConflicts=] [targetLockConflictBytes=] k= end= [idempotent] +// [noCoveredStats] +// del_range_pred [ts=[,]] [localTs=[,]] [maxLockConflicts=] [targetLockConflictBytes=] k= end= +// [startTime=,max=,maxBytes=,rangeThreshold=] +// 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_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] -// export [k=] [end=] [ts=[,]] [kTs=[,]] [startTs=[,]] [maxLockConflicts=] [targetLockConflictBytes=] [allRevisions] [targetSize=] [maxSize=] [stopMidKey] [fingerprint] +// 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] +// export [k=] [end=] [ts=[,]] [kTs=[,]] [startTs=[,]] [maxLockConflicts=] [targetLockConflictBytes=] +// [allRevisions] [targetSize=] [maxSize=] [stopMidKey] [fingerprint] // -// iter_new [k=] [end=] [prefix] [kind=key|keyAndIntents] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] [maskBelow=[,]] [minTimestamp=[,]] [maxTimestamp=[,]] -// iter_new_incremental [k=] [end=] [startTs=[,]] [endTs=[,]] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] [maskBelow=[,]] [intents=error|aggregate|emit] +// iter_new [k=] [end=] [prefix] [kind=key|keyAndIntents] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] [maskBelow=[,]] +// [minTimestamp=[,]] [maxTimestamp=[,]] +// iter_new_incremental [k=] [end=] [startTs=[,]] [endTs=[,]] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] +// [maskBelow=[,]] [intents=error|aggregate|emit] // iter_seek_ge k= [ts=[,]] // iter_seek_lt k= [ts=[,]] // iter_next @@ -128,7 +139,7 @@ var ( // replace_point_tombstones_with_range_tombstones k= [end=] // // sst_put [ts=[,]] [localTs=[,]] k= [v=] -// sst_put_rangekey ts=[,] [localTS=[,]] k= end= +// sst_put_rangekey ts=[,] [localTS=[,]] k= end= [syntheticBit] // sst_clear_range k= end= // sst_clear_rangekey k= end= ts=[,] // sst_finish @@ -1917,6 +1928,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") { + suffix := storage.EncodeMVCCTimestampSuffixWithSyntheticBitForTesting(rangeKey.Timestamp) + return e.withWriter("put_rangekey", func(rw storage.ReadWriter) error { + 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 429767fb08fb..7a88b5bd4fb9 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 08e89c773030..1dd54388d0b1 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -5400,7 +5400,9 @@ func (d rangeTestData) populateEngine( } ts = v.point.Key.Timestamp } else { - require.NoError(t, MVCCDeleteRangeUsingTombstone(ctx, engine, ms, v.rangeTombstone.StartKey, + rw := mvccRangeKeyEncodedTimestampReadWriter{engine} + // rw := 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 @@ -5409,6 +5411,35 @@ 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 varbatim. 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. +// +// 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 @@ -5507,7 +5538,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)}, }, }, { @@ -5531,8 +5562,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)}, }, }, { @@ -5554,7 +5585,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)}, }, }, { @@ -5566,7 +5597,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)}, }, }, { @@ -5578,8 +5609,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)}, }, }, { @@ -5591,7 +5622,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)}, }, }, { @@ -5613,7 +5644,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)}, }, }, { @@ -5636,7 +5667,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)}, }, }, { @@ -5649,7 +5680,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)}, }, }, { @@ -5662,7 +5693,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)}, }, }, { @@ -5675,7 +5706,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)}, }, }, { @@ -5776,11 +5807,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)}, }, }, { @@ -5793,7 +5824,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)}, }, }, { @@ -5806,7 +5837,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)}, }, }, { @@ -5819,7 +5850,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)}, }, }, { @@ -5835,8 +5866,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)}, }, }, { @@ -5851,7 +5882,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)}, }, }, { @@ -5867,11 +5898,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 e4b08c0ba977..b22784433511 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1843,6 +1843,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..861760382b42 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)) } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index 1dcae6b71d55..b2ae38fdcd07 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -745,8 +745,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 bb16f696d4f1..3ba97567bbba 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..31578dd24712 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_key_clear_synthetic_bit @@ -0,0 +1,21 @@ +# 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: