diff --git a/pkg/storage/engine_key.go b/pkg/storage/engine_key.go index 6fe0fccaae48..fcf3b89dbfc2 100644 --- a/pkg/storage/engine_key.go +++ b/pkg/storage/engine_key.go @@ -42,6 +42,7 @@ type EngineKey struct { // their particular use case, that demultiplex on the various lengths below. // If adding another length to this list, remember to search for code // referencing these lengths and fix it. +// TODO(nvanbenschoten): unify these constants with those in mvcc_key.go. const ( engineKeyNoVersion = 0 engineKeyVersionWallTimeLen = 8 diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index 093ee7e88101..4e8a2ea3d5b0 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -15,9 +15,11 @@ import ( "encoding/hex" "fmt" "math" + "math/rand" "reflect" "sort" "testing" + "testing/quick" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -71,6 +73,7 @@ func TestMVCCKeyCompare(t *testing.T) { b0 := MVCCKey{roachpb.Key("b"), hlc.Timestamp{Logical: 0}} b1 := MVCCKey{roachpb.Key("b"), hlc.Timestamp{Logical: 1}} b2 := MVCCKey{roachpb.Key("b"), hlc.Timestamp{Logical: 2}} + b2S := MVCCKey{roachpb.Key("b"), hlc.Timestamp{Logical: 2, Synthetic: true}} testcases := map[string]struct { a MVCCKey @@ -85,14 +88,76 @@ func TestMVCCKeyCompare(t *testing.T) { "empty time lt set": {b0, b1, -1}, // empty MVCC timestamps sort before non-empty "set time gt empty": {b1, b0, 1}, // empty MVCC timestamps sort before non-empty "key time precedence": {a1, b2, -1}, // a before b, but 2 before 1; key takes precedence + "synthetic equal": {b2, b2S, 0}, // synthetic bit does not affect ordering } for name, tc := range testcases { t.Run(name, func(t *testing.T) { require.Equal(t, tc.expect, tc.a.Compare(tc.b)) + require.Equal(t, tc.expect == 0, tc.a.Equal(tc.b)) + require.Equal(t, tc.expect < 0, tc.a.Less(tc.b)) + require.Equal(t, tc.expect > 0, tc.b.Less(tc.a)) + + // Comparators on encoded keys should be identical. + aEnc, bEnc := EncodeMVCCKey(tc.a), EncodeMVCCKey(tc.b) + require.Equal(t, tc.expect, EngineKeyCompare(aEnc, bEnc)) + require.Equal(t, tc.expect == 0, EngineKeyEqual(aEnc, bEnc)) }) } } +func TestMVCCKeyCompareRandom(t *testing.T) { + defer leaktest.AfterTest(t)() + + f := func(aGen, bGen randMVCCKey) bool { + a, b := MVCCKey(aGen), MVCCKey(bGen) + aEnc, bEnc := EncodeMVCCKey(a), EncodeMVCCKey(b) + + cmp := a.Compare(b) + cmpEnc := EngineKeyCompare(aEnc, bEnc) + eq := a.Equal(b) + eqEnc := EngineKeyEqual(aEnc, bEnc) + lessAB := a.Less(b) + lessBA := b.Less(a) + + if cmp != cmpEnc { + t.Logf("cmp (%v) != cmpEnc (%v)", cmp, cmpEnc) + return false + } + if eq != eqEnc { + t.Logf("eq (%v) != eqEnc (%v)", eq, eqEnc) + return false + } + if (cmp == 0) != eq { + t.Logf("(cmp == 0) (%v) != eq (%v)", cmp == 0, eq) + return false + } + if (cmp < 0) != lessAB { + t.Logf("(cmp < 0) (%v) != lessAB (%v)", cmp < 0, lessAB) + return false + } + if (cmp > 0) != lessBA { + t.Logf("(cmp > 0) (%v) != lessBA (%v)", cmp > 0, lessBA) + return false + } + return true + } + require.NoError(t, quick.Check(f, nil)) +} + +// randMVCCKey is a quick.Generator for MVCCKey. +type randMVCCKey MVCCKey + +func (k randMVCCKey) Generate(r *rand.Rand, size int) reflect.Value { + k.Key = []byte([...]string{"a", "b", "c"}[r.Intn(3)]) + k.Timestamp.WallTime = r.Int63n(5) + k.Timestamp.Logical = r.Int31n(5) + if !k.Timestamp.IsEmpty() { + // NB: the zero timestamp cannot be synthetic. + k.Timestamp.Synthetic = r.Intn(2) != 0 + } + return reflect.ValueOf(k) +} + func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { defer leaktest.AfterTest(t)() @@ -113,7 +178,6 @@ func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { "all": {"foo", hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535, Synthetic: true}, "666f6f0016cf10bc050557410000ffff010e"}, } for name, tc := range testcases { - tc := tc t.Run(name, func(t *testing.T) { // Test Encode/DecodeMVCCKey. @@ -172,6 +236,46 @@ func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { } } +func TestDecodeUnnormalizedMVCCKey(t *testing.T) { + defer leaktest.AfterTest(t)() + + testcases := map[string]struct { + encoded string // hex-encoded + expected MVCCKey + equalToNormal bool + }{ + "zero logical": { + encoded: "666f6f0016cf10bc05055741000000000d", + expected: MVCCKey{Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 0}}, + equalToNormal: true, + }, + "zero walltime and logical": { + encoded: "666f6f000000000000000000000000000d", + expected: MVCCKey{Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 0, Logical: 0}}, + // We could normalize this form in EngineKeyEqual and EngineKeyCompare, + // but doing so is not worth losing the fast-path byte comparison between + // keys that only contain (at most) a walltime. + equalToNormal: false, + }, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + encoded, err := hex.DecodeString(tc.encoded) + require.NoError(t, err) + + decoded, err := DecodeMVCCKey(encoded) + require.NoError(t, err) + require.Equal(t, tc.expected, decoded) + + // Re-encode the key into its normal form. + reencoded := EncodeMVCCKey(decoded) + require.NotEqual(t, encoded, reencoded) + require.Equal(t, tc.equalToNormal, EngineKeyEqual(encoded, reencoded)) + require.Equal(t, tc.equalToNormal, EngineKeyCompare(encoded, reencoded) == 0) + }) + } +} + func TestDecodeMVCCKeyErrors(t *testing.T) { defer leaktest.AfterTest(t)() @@ -185,7 +289,6 @@ func TestDecodeMVCCKeyErrors(t *testing.T) { "invalid timestamp length suffix": {"ab00ffffffffffffffff0f", "invalid encoded mvcc key: ab00ffffffffffffffff0f"}, } for name, tc := range testcases { - tc := tc t.Run(name, func(t *testing.T) { encoded, err := hex.DecodeString(tc.encoded) require.NoError(t, err) @@ -208,7 +311,6 @@ func TestDecodeMVCCTimestampSuffixErrors(t *testing.T) { "invalid length suffix": {"ffffffffffffffff0f", "bad timestamp: found length suffix 15, actual length 9"}, } for name, tc := range testcases { - tc := tc t.Run(name, func(t *testing.T) { encoded, err := hex.DecodeString(tc.encoded) require.NoError(t, err) diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 2457a6798697..ccc54793d8ce 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -117,17 +117,106 @@ func EngineKeyCompare(a, b []byte) int { // 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. - aTS := a[aSep:aEnd] - bTS := b[bSep:bEnd] - if len(aTS) == 0 { - if len(bTS) == 0 { + aVer := a[aSep:aEnd] + bVer := b[bSep:bEnd] + if len(aVer) == 0 { + if len(bVer) == 0 { return 0 } return -1 - } else if len(bTS) == 0 { + } else if len(bVer) == 0 { return 1 } - return bytes.Compare(bTS, aTS) + aVer = normalizeEngineKeyVersionForCompare(aVer) + bVer = normalizeEngineKeyVersionForCompare(bVer) + return bytes.Compare(bVer, aVer) +} + +// EngineKeyEqual checks for equality of cockroach keys, including the version +// (which could be MVCC timestamps). +func EngineKeyEqual(a, b []byte) bool { + // NB: For performance, this routine manually splits the key into the + // user-key and version components rather than using DecodeEngineKey. In + // most situations, use DecodeEngineKey or GetKeyPartFromEngineKey or + // SplitMVCCKey instead of doing this. + 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. + 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 + if aVerLen <= withWall && bVerLen <= withWall { + 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 "user key" part of the key. + if !bytes.Equal(a[:aSep], b[:bSep]) { + 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) +} + +var zeroLogical [mvccEncodedTimeLogicalLen]byte + +//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 { + // 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 { + // 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] + } + } + return a } // EngineComparer is a pebble.Comparer object that implements MVCC-specific @@ -135,6 +224,8 @@ func EngineKeyCompare(a, b []byte) int { var EngineComparer = &pebble.Comparer{ Compare: EngineKeyCompare, + Equal: EngineKeyEqual, + AbbreviatedKey: func(k []byte) uint64 { key, ok := GetKeyPartFromEngineKey(k) if !ok { diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index d087eb1715fe..44409107043d 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -14,7 +14,6 @@ import ( "bytes" "context" "fmt" - "io/ioutil" "math" "math/rand" "path/filepath" @@ -602,26 +601,42 @@ func TestPebbleIterConsistency(t *testing.T) { } func BenchmarkMVCCKeyCompare(b *testing.B) { + keys := makeRandEncodedKeys() + b.ResetTimer() + for i, j := 0, 0; i < b.N; i, j = i+1, j+3 { + _ = EngineKeyCompare(keys[i%len(keys)], keys[j%len(keys)]) + } +} + +func BenchmarkMVCCKeyEqual(b *testing.B) { + keys := makeRandEncodedKeys() + b.ResetTimer() + for i, j := 0, 0; i < b.N; i, j = i+1, j+3 { + _ = EngineKeyEqual(keys[i%len(keys)], keys[j%len(keys)]) + } +} + +func makeRandEncodedKeys() [][]byte { rng := rand.New(rand.NewSource(timeutil.Now().Unix())) keys := make([][]byte, 1000) for i := range keys { k := MVCCKey{ - Key: randutil.RandBytes(rng, 8), + Key: []byte("shared" + [...]string{"a", "b", "c"}[rng.Intn(3)]), Timestamp: hlc.Timestamp{ - WallTime: int64(rng.Intn(5)), + WallTime: rng.Int63n(5), }, } + if rng.Int31n(5) == 0 { + // 20% of keys have a logical component. + k.Timestamp.Logical = rng.Int31n(4) + 1 + } + if rng.Int31n(1000) == 0 && !k.Timestamp.IsEmpty() { + // 0.1% of keys have a synthetic component. + k.Timestamp.Synthetic = true + } keys[i] = EncodeMVCCKey(k) } - - b.ResetTimer() - var c int - for i, j := 0, 0; i < b.N; i, j = i+1, j+3 { - c = EngineKeyCompare(keys[i%len(keys)], keys[j%len(keys)]) - } - if testing.Verbose() { - fmt.Fprint(ioutil.Discard, c) - } + return keys } type testValue struct { diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit index 5f1252ea8085..41f854262a02 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit @@ -200,15 +200,17 @@ scan t=txn1 k=k1 localUncertaintyLimit=5,0 ---- scan: "k1"-"k1\x00" -> -run ok +run error get t=txn1 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] -run ok +run error scan t=txn1 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k3 localUncertaintyLimit=5,0 @@ -220,15 +222,17 @@ scan t=txn1 k=k3 localUncertaintyLimit=5,0 ---- scan: "k3"-"k3\x00" -> -run ok +run error get t=txn1 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] -run ok +run error scan t=txn1 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k5 localUncertaintyLimit=5,0 @@ -240,15 +244,17 @@ scan t=txn1 k=k5 localUncertaintyLimit=5,0 ---- scan: "k5"-"k5\x00" -> -run ok +run error get t=txn1 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] -run ok +run error scan t=txn1 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k7 localUncertaintyLimit=5,0 @@ -260,15 +266,17 @@ scan t=txn1 k=k7 localUncertaintyLimit=5,0 ---- scan: "k7"-"k7\x00" -> -run ok +run error get t=txn1 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] -run ok +run error scan t=txn1 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index adebc98eb49f..8d0e18c98659 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1996,6 +1996,7 @@ func TestLint(t *testing.T) { "../../sql/colfetcher", "../../sql/row", "../../kv/kvclient/rangecache", + "../../storage", ); err != nil { t.Fatal(err) }