diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index dae9dc84986a..bb2337edcc82 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -122,47 +122,43 @@ func EngineKeyCompare(a, b []byte) int { aEnd := len(a) - 1 bEnd := len(b) - 1 if aEnd < 0 || bEnd < 0 { - // This should never happen unless there is some sort of corruption of - // the keys. + // Pebble sometimes passes empty "keys" and we have to tolerate them. return bytes.Compare(a, b) } - // Compute the index of the separator between the key and the version. If the - // separator is found to be at -1 for both keys, then we are comparing bare - // suffixes without a user key part. Pebble requires bare suffixes to be - // comparable with the same ordering as if they had a common user key. - aSep := aEnd - int(a[aEnd]) - bSep := bEnd - int(b[bEnd]) - if aSep == -1 && bSep == -1 { - aSep, bSep = 0, 0 // comparing bare suffixes - } - if aSep < 0 || bSep < 0 { - // This should never happen unless there is some sort of corruption of - // the keys. + // Last byte is the version length + 1 when there is a version, + // else it is 0. + aSuffixLen := int(a[len(a)-1]) + aSuffixStart := len(a) - aSuffixLen + bSuffixLen := int(b[len(b)-1]) + bSuffixStart := len(b) - bSuffixLen + if aSuffixStart < 0 || bSuffixStart < 0 { + if buildutil.CrdbTestBuild { + panic(errors.AssertionFailedf("malformed keys: %x, %x", a, b)) + } return bytes.Compare(a, b) } - // Compare the "user key" part of the key. - if c := bytes.Compare(a[:aSep], b[:bSep]); c != 0 { + // Compare the "prefix" part of the keys. + if c := bytes.Compare(a[:aSuffixStart], b[:bSuffixStart]); c != 0 { return c } - // Compare the version part of the key. Note that when the version is a - // timestamp, the timestamp encoding causes byte comparison to be equivalent - // to timestamp comparison. - aVer := a[aSep:aEnd] - bVer := b[bSep:bEnd] - if len(aVer) == 0 { - if len(bVer) == 0 { + // Empty suffixes come before non-empty suffixes. + if aSuffixLen == 0 { + if bSuffixLen == 0 { return 0 } return -1 - } else if len(bVer) == 0 { + } + if bSuffixLen == 0 { return 1 } - aVer = normalizeEngineKeyVersionForCompare(aVer) - bVer = normalizeEngineKeyVersionForCompare(bVer) - return bytes.Compare(bVer, aVer) + + return bytes.Compare( + normalizeEngineSuffixForCompare(b[bSuffixStart:]), + normalizeEngineSuffixForCompare(a[aSuffixStart:]), + ) } // EngineKeyEqual checks for equality of cockroach keys, including the version @@ -175,80 +171,89 @@ func EngineKeyEqual(a, b []byte) bool { aEnd := len(a) - 1 bEnd := len(b) - 1 if aEnd < 0 || bEnd < 0 { - // This should never happen unless there is some sort of corruption of - // the keys. + // Pebble sometimes passes empty "keys" and we have to tolerate them. return bytes.Equal(a, b) } // Last byte is the version length + 1 when there is a version, // else it is 0. - aVerLen := int(a[aEnd]) - bVerLen := int(b[bEnd]) - - // Fast-path. If the key version is empty or contains only a walltime - // component then normalizeEngineKeyVersionForCompare is a no-op, so we don't - // need to split the "user key" from the version suffix before comparing to - // compute equality. Instead, we can check for byte equality immediately. - const withWall = mvccEncodedTimeSentinelLen + mvccEncodedTimeWallLen - const withLockTableLen = mvccEncodedTimeSentinelLen + engineKeyVersionLockTableLen - if (aVerLen <= withWall && bVerLen <= withWall) || (aVerLen == withLockTableLen && bVerLen == withLockTableLen) { + aSuffixLen := int(a[len(a)-1]) + aSuffixStart := len(a) - aSuffixLen + bSuffixLen := int(b[len(b)-1]) + bSuffixStart := len(b) - bSuffixLen + if aSuffixStart < 0 || bSuffixStart < 0 { + if buildutil.CrdbTestBuild { + panic(errors.AssertionFailedf("malformed keys: %x, %x", a, b)) + } return bytes.Equal(a, b) } - // Compute the index of the separator between the key and the version. If the - // separator is found to be at -1 for both keys, then we are comparing bare - // suffixes without a user key part. Pebble requires bare suffixes to be - // comparable with the same ordering as if they had a common user key. - aSep := aEnd - aVerLen - bSep := bEnd - bVerLen - if aSep == -1 && bSep == -1 { - aSep, bSep = 0, 0 // comparing bare suffixes - } - if aSep < 0 || bSep < 0 { - // This should never happen unless there is some sort of corruption of - // the keys. + // Fast-path: normalizeEngineSuffixForCompare doesn't strip off bytes when the + // length is withWall or withLockTableLen. In this case, as well as cases with + // no prefix, we can check for byte equality immediately. + const withWall = mvccEncodedTimeSentinelLen + mvccEncodedTimeWallLen + const withLockTableLen = mvccEncodedTimeSentinelLen + engineKeyVersionLockTableLen + if (aSuffixLen <= withWall && bSuffixLen <= withWall) || + (aSuffixLen == withLockTableLen && bSuffixLen == withLockTableLen) || + aSuffixLen == 0 || bSuffixLen == 0 { return bytes.Equal(a, b) } - // Compare the "user key" part of the key. - if !bytes.Equal(a[:aSep], b[:bSep]) { + // Compare the "prefix" part of the keys. + if !bytes.Equal(a[:aSuffixStart], b[:bSuffixStart]) { return false } - // Compare the version part of the key. - aVer := a[aSep:aEnd] - bVer := b[bSep:bEnd] - aVer = normalizeEngineKeyVersionForCompare(aVer) - bVer = normalizeEngineKeyVersionForCompare(bVer) - return bytes.Equal(aVer, bVer) + return bytes.Equal( + normalizeEngineSuffixForCompare(a[aSuffixStart:]), + normalizeEngineSuffixForCompare(b[bSuffixStart:]), + ) } var zeroLogical [mvccEncodedTimeLogicalLen]byte +// normalizeEngineSuffixForCompare takes a non-empty key suffix (including the +// trailing version length byte) and returns a prefix of the buffer that should +// be used for byte-wise comparison. It trims the trailing version length byte +// and any other trailing bytes that need to be ignored (like a synthetic bit or +// zero logical component). +// //gcassert:inline -func normalizeEngineKeyVersionForCompare(a []byte) []byte { - // In general, the version could also be a non-timestamp version, but we know - // that engineKeyVersionLockTableLen+mvccEncodedTimeSentinelLen is a different - // constant than the above, so there is no danger here of stripping parts from - // a non-timestamp version. - const withWall = mvccEncodedTimeSentinelLen + mvccEncodedTimeWallLen - const withLogical = withWall + mvccEncodedTimeLogicalLen - const withSynthetic = withLogical + mvccEncodedTimeSyntheticLen - if len(a) == withSynthetic { +func normalizeEngineSuffixForCompare(a []byte) []byte { + // Check the trailing version length byte. + if buildutil.CrdbTestBuild && len(a) != int(a[len(a)-1]) { + panic(errors.AssertionFailedf("malformed suffix: %x", a)) + } + // Strip off the trailing version length byte. + a = a[:len(a)-1] + switch len(a) { + case engineKeyVersionWallLogicalAndSyntheticTimeLen: // Strip the synthetic bit component from the timestamp version. The // presence of the synthetic bit does not affect key ordering or equality. - a = a[:withLogical] - } - if len(a) == withLogical { + a = a[:engineKeyVersionWallAndLogicalTimeLen] + fallthrough + case engineKeyVersionWallAndLogicalTimeLen: // If the timestamp version contains a logical timestamp component that is // zero, strip the component. encodeMVCCTimestampToBuf will typically omit // the entire logical component in these cases as an optimization, but it // does not guarantee to never include a zero logical component. // Additionally, we can fall into this case after stripping off other // components of the key version earlier on in this function. - if bytes.Equal(a[withWall:], zeroLogical[:]) { - a = a[:withWall] + if bytes.Equal(a[engineKeyVersionWallTimeLen:], zeroLogical[:]) { + a = a[:engineKeyVersionWallTimeLen] } + fallthrough + case engineKeyVersionWallTimeLen: + // Nothing to do. + + case engineKeyVersionLockTableLen: + // We rely on engineKeyVersionLockTableLen being different from the other + // lengths above to ensure that we don't strip parts from a non-timestamp + // version. + + default: + // It would have been nice to panic here in test builds, but the fuzz tests + // trigger it. } return a } @@ -610,8 +615,8 @@ func DefaultPebbleOptions() *pebble.Options { } opts.Experimental.ShortAttributeExtractor = shortAttributeExtractorForValues opts.Experimental.RequiredInPlaceValueBound = pebble.UserKeyPrefixBound{ - Lower: keys.LocalRangeLockTablePrefix, - Upper: keys.LocalRangeLockTablePrefix.PrefixEnd(), + Lower: EncodeMVCCKey(MVCCKey{Key: keys.LocalRangeLockTablePrefix}), + Upper: EncodeMVCCKey(MVCCKey{Key: keys.LocalRangeLockTablePrefix.PrefixEnd()}), } for i := 0; i < len(opts.Levels); i++ { diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 22c4448a74cd..7d3144b38781 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -11,6 +11,7 @@ package storage import ( + "bytes" "context" "fmt" "math" @@ -45,38 +46,129 @@ func TestEngineComparer(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - keyAMetadata := MVCCKey{ - Key: []byte("a"), - } - keyA2 := MVCCKey{ - Key: []byte("a"), - Timestamp: hlc.Timestamp{WallTime: 2}, + // encodeKey encodes a key. For the version, it supports arbitrary bytes or + // hlc.Timestamp, using either the EngineKey or MVCCKey or encoder. + encodeKey := func(key roachpb.Key, version any) []byte { + switch t := version.(type) { + case []byte: + ek := EngineKey{Key: key, Version: t} + return ek.Encode() + case hlc.Timestamp: + return EncodeMVCCKey(MVCCKey{Key: key, Timestamp: t}) + default: + panic(t) + } } - keyA1 := MVCCKey{ - Key: []byte("a"), - Timestamp: hlc.Timestamp{WallTime: 1}, + encodeVersion := func(version any) []byte { + kBare := encodeKey(roachpb.Key("foo"), hlc.Timestamp{}) + k := encodeKey(roachpb.Key("foo"), version) + if !bytes.HasPrefix(k, kBare) { + panic(fmt.Sprintf("expected %s to have prefix %s", k, kBare)) + } + return bytes.TrimPrefix(k, kBare) } - keyB2 := MVCCKey{ - Key: []byte("b"), - Timestamp: hlc.Timestamp{WallTime: 2}, + + appendBytesToTimestamp := func(ts hlc.Timestamp, bytes []byte) []byte { + suffix := encodeVersion(ts) + // Strip off sentinel byte. + version := suffix[:len(suffix)-1] + return append(version[:len(version):len(version)], bytes...) } - require.Equal(t, -1, EngineComparer.Compare(EncodeMVCCKey(keyAMetadata), EncodeMVCCKey(keyA1)), + keyAMetadata := encodeKey(roachpb.Key("a"), hlc.Timestamp{}) + keyA2 := encodeKey(roachpb.Key("a"), hlc.Timestamp{WallTime: 2}) + keyA1 := encodeKey(roachpb.Key("a"), hlc.Timestamp{WallTime: 1}) + keyB2 := encodeKey(roachpb.Key("b"), hlc.Timestamp{WallTime: 2}) + + require.Equal(t, -1, EngineComparer.Compare(keyAMetadata, keyA1), "expected key metadata to sort first") - require.Equal(t, -1, EngineComparer.Compare(EncodeMVCCKey(keyA2), EncodeMVCCKey(keyA1)), + require.Equal(t, -1, EngineComparer.Compare(keyA2, keyA1), "expected higher timestamp to sort first") - require.Equal(t, -1, EngineComparer.Compare(EncodeMVCCKey(keyA2), EncodeMVCCKey(keyB2)), + require.Equal(t, -1, EngineComparer.Compare(keyA2, keyB2), "expected lower key to sort first") suffix := func(key []byte) []byte { return key[EngineComparer.Split(key):] } - require.Equal(t, -1, EngineComparer.Compare(suffix(EncodeMVCCKey(keyA2)), suffix(EncodeMVCCKey(keyA1))), + require.Equal(t, -1, EngineComparer.Compare(suffix(keyA2), suffix(keyA1)), "expected bare suffix with higher timestamp to sort first") - for _, k := range []MVCCKey{keyAMetadata, keyA2, keyA1, keyB2} { - b := EncodeMVCCKey(k) + for _, b := range [][]byte{keyAMetadata, keyA2, keyA1, keyB2} { require.Equal(t, 2, EngineComparer.Split(b)) } + + ts1 := hlc.Timestamp{} + require.Len(t, encodeVersion(ts1), 0) + ts2 := hlc.Timestamp{WallTime: 2, Logical: 1} + ts3 := hlc.Timestamp{WallTime: 2} + ts4 := hlc.Timestamp{WallTime: 1, Logical: 1} + ts5 := hlc.Timestamp{WallTime: 1} + + syntheticBit := []byte{1} + ts2a := appendBytesToTimestamp(ts2, syntheticBit) + ts3a := appendBytesToTimestamp(ts3, zeroLogical[:]) + ts3b := appendBytesToTimestamp(ts3, append(zeroLogical[:], syntheticBit...)) + + // We group versions by equality and in the expected ordering. + orderedVersions := [][]any{ + {ts1}, // Empty version sorts first. + {ts2, ts2a}, // Higher timestamps sort before lower timestamps. + {ts3, ts3a, ts3b}, + {ts4}, + {ts5}, + } + + // Compare suffixes. + for i := range orderedVersions { + for j := range orderedVersions { + for _, v1 := range orderedVersions[i] { + for _, v2 := range orderedVersions[j] { + result := EngineComparer.Compare(encodeVersion(v1), encodeVersion(v2)) + if expected := compareInts(i, j); result != expected { + t.Fatalf("Compare(%x, %x) = %d, expected %d", encodeVersion(v1), encodeVersion(v2), result, expected) + } + } + } + } + } + + lock1 := bytes.Repeat([]byte{1}, engineKeyVersionLockTableLen) + lock2 := bytes.Repeat([]byte{2}, engineKeyVersionLockTableLen) + require.Equal(t, 0, EngineComparer.Compare(encodeVersion(lock1), encodeVersion(lock1))) + require.Equal(t, 0, EngineComparer.Compare(encodeVersion(lock2), encodeVersion(lock2))) + require.Equal(t, +1, EngineComparer.Compare(encodeVersion(lock1), encodeVersion(lock2))) + require.Equal(t, -1, EngineComparer.Compare(encodeVersion(lock2), encodeVersion(lock1))) + + keys := []roachpb.Key{ + roachpb.Key(""), + roachpb.Key("a"), + roachpb.Key("bcd"), + roachpb.Key("fg"), + } + + // We group keys by equality and in the expected ordering. + var orderedKeys [][][]byte + for _, k := range keys { + orderedKeys = append(orderedKeys, + [][]byte{encodeKey(k, ts1)}, + [][]byte{encodeKey(k, ts2), encodeKey(k, ts2a)}, + [][]byte{encodeKey(k, ts3), encodeKey(k, ts3a), encodeKey(k, ts3b)}, + [][]byte{encodeKey(k, ts4)}, + [][]byte{encodeKey(k, ts5)}, + ) + } + // Compare keys. + for i := range orderedKeys { + for j := range orderedKeys { + for _, k1 := range orderedKeys[i] { + for _, k2 := range orderedKeys[j] { + result := EngineComparer.Compare(k1, k2) + if expected := compareInts(i, j); result != expected { + t.Fatalf("Compare(%x, %x) = %d, expected %d", k1, k2, result, expected) + } + } + } + } + } } func TestPebbleIterReuse(t *testing.T) { @@ -1465,3 +1557,13 @@ func TestPebbleLoggingSlowReads(t *testing.T) { } require.Less(t, 0, slowCount) } + +func compareInts(i, j int) int { + if i < j { + return -1 + } + if i > j { + return +1 + } + return 0 +}