Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
130453: logictest: revert incorrect test assertion update r=rafiss a=michae2

(Deja vu: this is #121556 all over again.)

103bd54 incorrectly updated the test expectations, likely because the `--rewrite` flag was used on an assertion that has the retry directive.

This commit undoes that change.

Fixes: #130405

Release note: None

130572: storage: GC range keys by unsetting identical suffixes r=jbowens a=jbowens

In CockroachDB's key encoding some keys have multiple logically equivalent but physically distinct encodings. Most notably, in CockroachDB versions 23.2 and earlier keys written to global tables encoded MVCC timestamps with a 'synthetic bit.' In #101938 CockroachDB stopped encoding and decoding this synthetic bit, transparently ignoring it.

In #129592 we observed the existence of a bug in the CockroachDB comparator when comparing two MVCC timestamp suffixes, specifically outside the context of a full MVCC key. The comparator failed to consider a timestamp with the synthetic bit and a timestamp without the synthetic bit as logically equivalent. There are limited instances where Pebble uses the comparator to compare "bare suffixes," and all instances are constrained to the implementation of range keys.

In #129592 it was observed that the comparator bug could prevent the garbage collection of MVCC delete range tombstones (the single use of range keys within CockroachDB). A cluster running 23.2 or earlier may write a MVCC delete range tombstone with a timestamp encoding the synthetic bit. If the cluster subsequently upgraded to 24.1 or later, the code path to clear range keys stopped understanding synthetic bits and wrote range key unset tombstones without the synthetic bit set. Due to the comparator bug, Pebble did not consider these timestamp suffixes equal and the unset was ineffective.

We initially attempted to fix this issue by fixing the comparator, but inadvertently introduced the possibility of replica divergence #130533 by changing the semantics of LSM state below raft.

This commit works around this comparator bug by adapting ClearMVCCRangeKey to write range key unsets using the verbatim suffix that was read from the storage engine. To avoid reverting #101938 and re-introducing knowledge of the synthetic bit, the MVCCRangeKey data structures are adapted to retain a copy of the encoded timestamp suffix when reading range keys from storage engine iterators. If later an attempt is made to clear the range key through ClearMVCCRangeKey, this encoded timestamp suffix is used instead of re-encoding the timestamp. Through avoiding the decoding/encoding roundtrip, ClearMVCCRangeKey ensures that the suffixes it writes are identical to the range keys that exist on disk, even if they encode a synthetic bit.

Release note (bug fix): Fixes a bug that could result in the inability to garbage collect a MVCC range tombstone within a global table.
Epic: none
Informs #129592.

130906: sql: deflake TestValidationWithProtectedTS r=rafiss a=rafiss

This test does not work if ranges get split, so we disable the split queue.

fixes #130715
Release note: None

Co-authored-by: Michael Erickson <[email protected]>
Co-authored-by: Jackson Owens <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
4 people committed Sep 18, 2024
4 parents 1e27189 + ef2f74c + 75dcbec + 9cb541d commit 68fc953
Show file tree
Hide file tree
Showing 18 changed files with 285 additions and 65 deletions.
8 changes: 5 additions & 3 deletions pkg/ccl/crosscluster/replicationutils/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,9 +124,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,
})
}
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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})
Expand Down
8 changes: 5 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_delete_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
7 changes: 4 additions & 3 deletions pkg/kv/kvserver/gc/gc_random_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
})
Expand Down
7 changes: 4 additions & 3 deletions pkg/kv/kvserver/gc/gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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++
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/backfill_protected_timestamp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,9 @@ func TestValidationWithProtectedTS(t *testing.T) {
SQLEvalContext: &eval.TestingKnobs{
ForceProductionValues: true,
},
Store: &kvserver.StoreTestingKnobs{
DisableSplitQueue: true,
},
SQLExecutor: &sql.ExecutorTestingKnobs{
BeforeExecute: func(ctx context.Context, sql string, descriptors *descs.Collection) {
if indexScanQuery.MatchString(sql) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -205,8 +205,8 @@ ORDER BY start_key
/Table/106 14400
/Table/107 14400
/Table/110 90001
/Table/111 90001
/Table/112 14400
/Table/111 1
/Table/112 1

subtest transactional_schemachanges

Expand Down
20 changes: 19 additions & 1 deletion pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ var (
// increment [t=<name>] [ts=<int>[,<int>]] [localTs=<int>[,<int>]] [resolve [status=<txnstatus>]] [ambiguousReplay] [maxLockConflicts=<int>] [targetLockConflictBytes=<int>] k=<key> [inc=<val>]
// initput [t=<name>] [ts=<int>[,<int>]] [resolve [status=<txnstatus>]] [ambiguousReplay] [maxLockConflicts=<int>] k=<key> v=<string> [raw] [failOnTombstones]
// put [t=<name>] [ts=<int>[,<int>]] [localTs=<int>[,<int>]] [resolve [status=<txnstatus>]] [ambiguousReplay] [maxLockConflicts=<int>] k=<key> v=<string> [raw]
// put_rangekey ts=<int>[,<int>] [localTs=<int>[,<int>]] k=<key> end=<key>
// put_rangekey ts=<int>[,<int>] [localTs=<int>[,<int>]] k=<key> end=<key> [syntheticBit]
// put_blind_inline k=<key> v=<string> [prev=<string>]
// get [t=<name>] [ts=<int>[,<int>]] [resolve [status=<txnstatus>]] k=<key> [inconsistent] [skipLocked] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=<int>[,<int>]] [globalUncertaintyLimit=<int>[,<int>]] [maxKeys=<int>] [targetBytes=<int>] [allowEmpty]
// scan [t=<name>] [ts=<int>[,<int>]] [resolve [status=<txnstatus>]] k=<key> [end=<key>] [inconsistent] [skipLocked] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=<int>[,<int>]] [globalUncertaintyLimit=<int>[,<int>]] [max=<max>] [targetbytes=<target>] [wholeRows[=<int>]] [allowEmpty]
Expand Down Expand Up @@ -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)
})
Expand Down
74 changes: 69 additions & 5 deletions pkg/storage/mvcc_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"bytes"
"encoding/binary"
"fmt"
"slices"
"sort"
"strings"

Expand Down Expand Up @@ -385,15 +386,30 @@ 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.
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,
}},
}
}
Expand All @@ -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
}

Expand Down Expand Up @@ -507,22 +524,38 @@ 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
// MVCCRangeKeyVersion, reusing and overwriting its value slice.
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,
}
}

Expand Down Expand Up @@ -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...)
}
}

Expand Down Expand Up @@ -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
}

Expand All @@ -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
}
36 changes: 24 additions & 12 deletions pkg/storage/mvcc_key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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},
},
Expand All @@ -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])
}
}
}

Expand All @@ -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})},
},
}

Expand All @@ -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)},
},
}

Expand Down Expand Up @@ -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.
Expand All @@ -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)
}
}
})
Expand Down Expand Up @@ -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
Expand All @@ -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) {
Expand Down Expand Up @@ -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)),
}
}

Expand Down
Loading

0 comments on commit 68fc953

Please sign in to comment.