Skip to content

Commit

Permalink
Revert "release-24.2: storage: fix comparison of suffixes"
Browse files Browse the repository at this point in the history
This reverts commit 5daa9f6.

Release note: None.
Release justification: Reverts a commit that could result in replica inconsistency.
Epic: None.
  • Loading branch information
jbowens committed Sep 12, 2024
1 parent b95f17a commit 8c242e7
Show file tree
Hide file tree
Showing 2 changed files with 95 additions and 190 deletions.
154 changes: 77 additions & 77 deletions pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package storage

import (
"bytes"
"cmp"
"context"
"encoding/binary"
"encoding/json"
Expand Down Expand Up @@ -400,37 +399,47 @@ func EngineKeyCompare(a, b []byte) int {
aEnd := len(a) - 1
bEnd := len(b) - 1
if aEnd < 0 || bEnd < 0 {
// Pebble sometimes passes empty "keys" and we have to tolerate them.
// This should never happen unless there is some sort of corruption of
// the keys.
return bytes.Compare(a, b)
}

// 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))
}
// 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.
return bytes.Compare(a, b)
}

// Compare the "prefix" part of the keys.
if c := bytes.Compare(a[:aSuffixStart], b[:bSuffixStart]); c != 0 {
// Compare the "user key" part of the key.
if c := bytes.Compare(a[:aSep], b[:bSep]); c != 0 {
return c
}

if aSuffixLen == 0 || bSuffixLen == 0 {
// Empty suffixes come before non-empty suffixes.
return cmp.Compare(aSuffixLen, bSuffixLen)
// 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 {
return 0
}
return -1
} else if len(bVer) == 0 {
return 1
}

return bytes.Compare(
normalizeEngineSuffixForCompare(b[bSuffixStart:]),
normalizeEngineSuffixForCompare(a[aSuffixStart:]),
)
aVer = normalizeEngineKeyVersionForCompare(aVer)
bVer = normalizeEngineKeyVersionForCompare(bVer)
return bytes.Compare(bVer, aVer)
}

// EngineKeyEqual checks for equality of cockroach keys, including the version
Expand All @@ -443,89 +452,80 @@ func EngineKeyEqual(a, b []byte) bool {
aEnd := len(a) - 1
bEnd := len(b) - 1
if aEnd < 0 || bEnd < 0 {
// Pebble sometimes passes empty "keys" and we have to tolerate them.
// This should never happen unless there is some sort of corruption of
// the keys.
return bytes.Equal(a, b)
}

// 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.Equal(a, b)
}
aVerLen := int(a[aEnd])
bVerLen := int(b[bEnd])

// 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.
// 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 (aSuffixLen <= withWall && bSuffixLen <= withWall) ||
(aSuffixLen == withLockTableLen && bSuffixLen == withLockTableLen) ||
aSuffixLen == 0 || bSuffixLen == 0 {
if (aVerLen <= withWall && bVerLen <= withWall) || (aVerLen == withLockTableLen && bVerLen == withLockTableLen) {
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.
return bytes.Equal(a, b)
}

// Compare the "prefix" part of the keys.
if !bytes.Equal(a[:aSuffixStart], b[:bSuffixStart]) {
// Compare the "user key" part of the key.
if !bytes.Equal(a[:aSep], b[:bSep]) {
return false
}

return bytes.Equal(
normalizeEngineSuffixForCompare(a[aSuffixStart:]),
normalizeEngineSuffixForCompare(b[bSuffixStart:]),
)
// 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)
}

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 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:
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 {
// Strip the synthetic bit component from the timestamp version. The
// presence of the synthetic bit does not affect key ordering or equality.
a = a[:engineKeyVersionWallAndLogicalTimeLen]
fallthrough
case engineKeyVersionWallAndLogicalTimeLen:
a = a[:withLogical]
}
if len(a) == withLogical {
// 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[engineKeyVersionWallTimeLen:], zeroLogical[:]) {
a = a[:engineKeyVersionWallTimeLen]
if bytes.Equal(a[withWall:], zeroLogical[:]) {
a = a[:withWall]
}
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
}
Expand Down
131 changes: 18 additions & 113 deletions pkg/storage/pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,14 +11,11 @@
package storage

import (
"bytes"
"cmp"
"context"
"fmt"
"math/rand"
"path/filepath"
"runtime"
"slices"
"strings"
"sync/atomic"
"testing"
Expand Down Expand Up @@ -52,130 +49,38 @@ func TestEngineComparer(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

// 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)
}
keyAMetadata := MVCCKey{
Key: []byte("a"),
}
encodeVersion := func(version any) []byte {
kBare := encodeKey(roachpb.Key("foo"), hlc.Timestamp{})
k := encodeKey(roachpb.Key("foo"), version)
result, ok := bytes.CutPrefix(k, kBare)
if !ok {
panic(fmt.Sprintf("expected %s to have prefix %s", k, kBare))
}
return result
keyA2 := MVCCKey{
Key: []byte("a"),
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 slices.Concat(version, bytes)
keyA1 := MVCCKey{
Key: []byte("a"),
Timestamp: hlc.Timestamp{WallTime: 1},
}
keyB2 := MVCCKey{
Key: []byte("b"),
Timestamp: hlc.Timestamp{WallTime: 2},
}

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),
require.Equal(t, -1, EngineComparer.Compare(EncodeMVCCKey(keyAMetadata), EncodeMVCCKey(keyA1)),
"expected key metadata to sort first")
require.Equal(t, -1, EngineComparer.Compare(keyA2, keyA1),
require.Equal(t, -1, EngineComparer.Compare(EncodeMVCCKey(keyA2), EncodeMVCCKey(keyA1)),
"expected higher timestamp to sort first")
require.Equal(t, -1, EngineComparer.Compare(keyA2, keyB2),
require.Equal(t, -1, EngineComparer.Compare(EncodeMVCCKey(keyA2), EncodeMVCCKey(keyB2)),
"expected lower key to sort first")

suffix := func(key []byte) []byte {
return key[EngineComparer.Split(key):]
}
require.Equal(t, -1, EngineComparer.Compare(suffix(keyA2), suffix(keyA1)),
require.Equal(t, -1, EngineComparer.Compare(suffix(EncodeMVCCKey(keyA2)), suffix(EncodeMVCCKey(keyA1))),
"expected bare suffix with higher timestamp to sort first")
for _, b := range [][]byte{keyAMetadata, keyA2, keyA1, keyB2} {
for _, k := range []MVCCKey{keyAMetadata, keyA2, keyA1, keyB2} {
b := EncodeMVCCKey(k)
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, slices.Concat(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 := cmp.Compare(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 := cmp.Compare(i, j); result != expected {
t.Fatalf("Compare(%x, %x) = %d, expected %d", k1, k2, result, expected)
}
}
}
}
}
}

func TestPebbleIterReuse(t *testing.T) {
Expand Down

0 comments on commit 8c242e7

Please sign in to comment.