diff --git a/comparer.go b/comparer.go index c92cd7930e..5f31d240e8 100644 --- a/comparer.go +++ b/comparer.go @@ -29,3 +29,10 @@ type Comparer = base.Comparer // DefaultComparer exports the base.DefaultComparer variable. var DefaultComparer = base.DefaultComparer + +// CheckComparer is a mini test suite that verifies a comparer implementation. +// +// It takes strictly ordered (according to the comparator) lists of prefixes and +// suffixes. Both lists must contain the empty slice. It is recommended that +// both lists have at least three elements. +var CheckComparer = base.CheckComparer diff --git a/db.go b/db.go index 826374660f..2760f2ccd7 100644 --- a/db.go +++ b/db.go @@ -1152,7 +1152,7 @@ func finishInitializingIter(ctx context.Context, buf *iterAlloc) *Iterator { } if dbi.opts.rangeKeys() { - dbi.rangeKeyMasking.init(dbi, dbi.comparer.Compare, dbi.comparer.Split) + dbi.rangeKeyMasking.init(dbi, &dbi.comparer) // When iterating over both point and range keys, don't create the // range-key iterator stack immediately if we can avoid it. This diff --git a/external_iterator.go b/external_iterator.go index 08eecca095..72c7a14797 100644 --- a/external_iterator.go +++ b/external_iterator.go @@ -194,7 +194,7 @@ func finishInitializingExternal(ctx context.Context, it *Iterator) error { it.iter = it.pointIter if it.opts.rangeKeys() { - it.rangeKeyMasking.init(it, it.comparer.Compare, it.comparer.Split) + it.rangeKeyMasking.init(it, &it.comparer) var rangeKeyIters []keyspan.FragmentIterator if it.rangeKey == nil { // We could take advantage of the lack of overlaps in range keys within diff --git a/internal/base/comparer.go b/internal/base/comparer.go index 4673f64800..934ba820a9 100644 --- a/internal/base/comparer.go +++ b/internal/base/comparer.go @@ -8,29 +8,50 @@ import ( "bytes" "encoding/binary" "fmt" + "slices" "strconv" "unicode/utf8" + + "github.com/cockroachdb/errors" ) +// CompareSuffixes compares two key suffixes and returns -1, 0, or +1. +// +// The empty slice suffix must be 'less than' any non-empty suffix. +// +// A full key k is composed of a prefix k[:Split(k)] and suffix k[Split(k):]. +// Suffixes are compared to break ties between equal prefixes. +type CompareSuffixes func(a, b []byte) int + // Compare returns -1, 0, or +1 depending on whether a is 'less than', 'equal -// to' or 'greater than' b. The empty slice must be 'less than' any non-empty -// slice. +// to' or 'greater than' b. // -// Compare is used to compare user keys, such as those passed as arguments to -// the various DB methods, as well as those returned from Separator, Successor, -// and Split. It is also used to compare key suffixes, i.e. the remainder of the -// key after Split. +// Both a and b must be valid keys. // -// The comparison of the prefix parts must be a simple byte-wise compare. In -// other words, if a and be don't have suffixes, then -// Compare(a, b) = bytes.Compare(a, b). +// A key a is less than b if a's prefix is byte-wise less than b's prefix, or if +// the prefixes are equal and a's suffix is less than b's suffix (according to +// CompareSuffixes). // -// In general, if prefix(a) = a[:Split(a)] and suffix(a) = a[Split(a):], then +// In other words, if prefix(a) = a[:Split(a)] and suffix(a) = a[Split(a):]: // // Compare(a, b) = bytes.Compare(prefix(a), prefix(b)) if not 0, or -// bytes.Compare(suffix(a), suffix(b)) otherwise. +// CompareSuffixes(suffix(a), suffix(b)) otherwise. +// +// Compare defaults to using the formula above but it can be customized if there +// is a (potentially faster) specialization. type Compare func(a, b []byte) int +// defaultCompare implements Compare in terms of Split and CompareSuffixes, as +// mentioned above. +func defaultCompare(split Split, compareSuffixes CompareSuffixes, a, b []byte) int { + an := split(a) + bn := split(b) + if prefixCmp := bytes.Compare(a[:an], b[:bn]); prefixCmp != 0 { + return prefixCmp + } + return compareSuffixes(a[an:], b[bn:]) +} + // Equal returns true if a and b are equivalent. // // For a given Compare, Equal(a,b)=true iff Compare(a,b)=0; that is, Equal is a @@ -145,8 +166,7 @@ var DefaultSplit Split = func(key []byte) int { return len(key) } // Comparer defines a total ordering over the space of []byte keys: a 'less // than' relationship. type Comparer struct { - // These fields must always be specified. - Compare Compare + // The following must always be specified. AbbreviatedKey AbbreviatedKey Separator Separator Successor Successor @@ -154,13 +174,20 @@ type Comparer struct { // ImmediateSuccessor must be specified if range keys are used. ImmediateSuccessor ImmediateSuccessor + // Split defaults to a trivial implementation that returns the full key length + // if it is not specified. + Split Split + + // CompareSuffixes defaults to bytes.Compare if it is not specified. + CompareSuffixes CompareSuffixes + + // Compare defaults to a generic implementation that uses Split, + // bytes.Compare, and CompareSuffixes if it is not specified. + Compare Compare // Equal defaults to using Compare() == 0 if it is not specified. Equal Equal // FormatKey defaults to the DefaultFormatter if it is not specified. FormatKey FormatKey - // Split defaults to a trivial implementation that returns the full key length - // if it is not specified. - Split Split // FormatValue is optional. FormatValue FormatValue @@ -182,23 +209,37 @@ func (c *Comparer) EnsureDefaults() *Comparer { if c == nil { return DefaultComparer } - if c.Compare == nil || c.AbbreviatedKey == nil || c.Separator == nil || c.Successor == nil || c.Name == "" { + if c.AbbreviatedKey == nil || c.Separator == nil || c.Successor == nil || c.Name == "" { panic("invalid Comparer: mandatory field not set") } - if c.Equal != nil && c.Split != nil && c.FormatKey != nil { + if c.CompareSuffixes != nil && c.Compare != nil && c.Equal != nil && c.Split != nil && c.FormatKey != nil { return c } n := &Comparer{} *n = *c - if n.Equal == nil { - cmp := n.Compare - n.Equal = func(a, b []byte) bool { - return cmp(a, b) == 0 - } - } + if n.Split == nil { n.Split = DefaultSplit } + if n.CompareSuffixes == nil && n.Compare == nil && n.Equal == nil { + n.CompareSuffixes = bytes.Compare + n.Compare = bytes.Compare + n.Equal = bytes.Equal + } else { + if n.CompareSuffixes == nil { + n.CompareSuffixes = bytes.Compare + } + if n.Compare == nil { + n.Compare = func(a, b []byte) int { + return defaultCompare(n.Split, n.CompareSuffixes, a, b) + } + } + if n.Equal == nil { + n.Equal = func(a, b []byte) bool { + return n.Compare(a, b) == 0 + } + } + } if n.FormatKey == nil { n.FormatKey = DefaultFormatter } @@ -208,8 +249,9 @@ func (c *Comparer) EnsureDefaults() *Comparer { // DefaultComparer is the default implementation of the Comparer interface. // It uses the natural ordering, consistent with bytes.Compare. var DefaultComparer = &Comparer{ - Compare: bytes.Compare, - Equal: bytes.Equal, + CompareSuffixes: bytes.Compare, + Compare: bytes.Compare, + Equal: bytes.Equal, AbbreviatedKey: func(key []byte) uint64 { if len(key) >= 8 { @@ -338,31 +380,25 @@ func MakeAssertComparer(c Comparer) Comparer { return Comparer{ Compare: func(a []byte, b []byte) int { res := c.Compare(a, b) - an := c.Split(a) - aPrefix, aSuffix := a[:an], a[an:] - bn := c.Split(b) - bPrefix, bSuffix := b[:bn], b[bn:] - if prefixCmp := bytes.Compare(aPrefix, bPrefix); prefixCmp == 0 { - if suffixCmp := c.Compare(aSuffix, bSuffix); suffixCmp != res { - panic(AssertionFailedf("%s: Compare with equal prefixes not consistent with Compare of suffixes: Compare(%q, %q)=%d, Compare(%q, %q)=%d", - c.Name, a, b, res, aSuffix, bSuffix, suffixCmp, - )) - } - } else if prefixCmp != res { - panic(AssertionFailedf("%s: Compare did not perform byte-wise comparison of prefixes", c.Name)) + // Verify that Compare is consistent with the default implementation. + if expected := defaultCompare(c.Split, c.CompareSuffixes, a, b); res != expected { + panic(AssertionFailedf("%s: Compare(%s, %s)=%d, expected %d", + c.Name, c.FormatKey(a), c.FormatKey(b), res, expected)) } return res }, Equal: func(a []byte, b []byte) bool { eq := c.Equal(a, b) - if cmp := c.Compare(a, b); eq != (cmp == 0) { + // Verify that Equal is consistent with Compare. + if expected := c.Compare(a, b); eq != (expected == 0) { panic("Compare and Equal are not consistent") } return eq }, // TODO(radu): add more checks. + CompareSuffixes: c.CompareSuffixes, AbbreviatedKey: c.AbbreviatedKey, Separator: c.Separator, Successor: c.Successor, @@ -373,3 +409,57 @@ func MakeAssertComparer(c Comparer) Comparer { Name: c.Name, } } + +// CheckComparer is a mini test suite that verifies a comparer implementation. +// +// It takes lists of valid prefixes and suffixes. It is recommended that both +// lists have at least three elements. +func CheckComparer(c *Comparer, prefixes [][]byte, suffixes [][]byte) error { + // Empty slice is always a valid suffix. + suffixes = append(suffixes, nil) + + // Verify the suffixes have a consistent ordering. + slices.SortFunc(suffixes, c.CompareSuffixes) + if !slices.IsSortedFunc(suffixes, c.CompareSuffixes) { + return errors.Errorf("CompareSuffixes is inconsistent") + } + + // Check the split function. + for _, p := range prefixes { + for _, s := range suffixes { + key := slices.Concat(p, s) + if n := c.Split(key); n != len(p) { + return errors.Errorf("incorrect Split result %d on '%x' (prefix '%x' suffix '%x')", n, key, p, s) + } + } + } + + // Check the Compare/Equals functions on all possible combinations. + for _, ap := range prefixes { + for _, as := range suffixes { + a := slices.Concat(ap, as) + for _, bp := range prefixes { + for _, bs := range suffixes { + b := slices.Concat(bp, bs) + result := c.Compare(a, b) + + expected := bytes.Compare(ap, bp) + if expected == 0 { + expected = c.CompareSuffixes(as, bs) + } + + if (result == 0) != c.Equal(a, b) { + return errors.Errorf("Equal(%s, %s) doesn't agree with Compare", c.FormatKey(a), c.FormatKey(b)) + } + + if result != expected { + return errors.Errorf("Compare(%s, %s)=%d, expected %d", c.FormatKey(a), c.FormatKey(b), result, expected) + } + } + } + } + } + + // TODO(radu): check more methods. + return nil +} diff --git a/internal/base/comparer_test.go b/internal/base/comparer_test.go index ae49a314ed..cb1d7ea9b7 100644 --- a/internal/base/comparer_test.go +++ b/internal/base/comparer_test.go @@ -55,6 +55,12 @@ func TestDefAppendSeparator(t *testing.T) { } } +func TestDefaultComparer(t *testing.T) { + if err := CheckComparer(DefaultComparer, [][]byte{{}, []byte("abc"), []byte("d"), []byte("ef")}, [][]byte{{}}); err != nil { + t.Error(err) + } +} + func TestAbbreviatedKey(t *testing.T) { rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) randBytes := func(size int) []byte { diff --git a/internal/compact/iterator.go b/internal/compact/iterator.go index f17e88e7be..b5c710ee91 100644 --- a/internal/compact/iterator.go +++ b/internal/compact/iterator.go @@ -148,7 +148,8 @@ import ( // exported function, and before a subsequent call to Next advances the iterator // and mutates the contents of the returned key and value. type Iter struct { - cmp base.Compare + cmp base.Compare + suffixCmp base.CompareSuffixes cfg IterConfig @@ -308,8 +309,9 @@ func NewIter( ) *Iter { cfg.ensureDefaults() i := &Iter{ - cmp: cfg.Comparer.Compare, - cfg: cfg, + cmp: cfg.Comparer.Compare, + suffixCmp: cfg.Comparer.CompareSuffixes, + cfg: cfg, // We don't want a nil keyBuf because if the first key we encounter is // empty, it would become nil. keyBuf: make([]byte, 8), @@ -329,7 +331,7 @@ func NewIter( i.frontiers.Init(i.cmp) i.delElider.Init(i.cmp, cfg.TombstoneElision) i.rangeDelCompactor = MakeRangeDelSpanCompactor(i.cmp, i.cfg.Comparer.Equal, cfg.Snapshots, cfg.TombstoneElision) - i.rangeKeyCompactor = MakeRangeKeySpanCompactor(i.cmp, i.cfg.Comparer.Equal, cfg.Snapshots, cfg.RangeKeyElision) + i.rangeKeyCompactor = MakeRangeKeySpanCompactor(i.cmp, i.suffixCmp, cfg.Snapshots, cfg.RangeKeyElision) i.lastRangeDelSpanFrontier.Init(&i.frontiers, nil, i.lastRangeDelSpanFrontierReached) return i } diff --git a/internal/compact/spans.go b/internal/compact/spans.go index 8ca8c13a43..d8d5bd47e9 100644 --- a/internal/compact/spans.go +++ b/internal/compact/spans.go @@ -83,18 +83,18 @@ func (c *RangeDelSpanCompactor) Compact(span, output *keyspan.Span) { // for at most one "compacted" span. type RangeKeySpanCompactor struct { cmp base.Compare - equal base.Equal + suffixCmp base.CompareSuffixes snapshots Snapshots elider rangeTombstoneElider } // MakeRangeKeySpanCompactor creates a new compactor for range key spans. func MakeRangeKeySpanCompactor( - cmp base.Compare, equal base.Equal, snapshots Snapshots, elision TombstoneElision, + cmp base.Compare, suffixCmp base.CompareSuffixes, snapshots Snapshots, elision TombstoneElision, ) RangeKeySpanCompactor { c := RangeKeySpanCompactor{ cmp: cmp, - equal: equal, + suffixCmp: suffixCmp, snapshots: snapshots, } c.elider.Init(cmp, elision) @@ -131,7 +131,7 @@ func (c *RangeKeySpanCompactor) Compact(span, output *keyspan.Span) { } if y > start { keysDst := output.Keys[usedLen:cap(output.Keys)] - rangekey.Coalesce(c.cmp, c.equal, span.Keys[start:y], &keysDst) + rangekey.Coalesce(c.suffixCmp, span.Keys[start:y], &keysDst) if y == len(span.Keys) { // This is the last snapshot stripe. Unsets and deletes can be elided. keysDst = c.elideInLastStripe(span.Start, span.End, keysDst) @@ -143,7 +143,7 @@ func (c *RangeKeySpanCompactor) Compact(span, output *keyspan.Span) { } if y < len(span.Keys) { keysDst := output.Keys[usedLen:cap(output.Keys)] - rangekey.Coalesce(c.cmp, c.equal, span.Keys[y:], &keysDst) + rangekey.Coalesce(c.suffixCmp, span.Keys[y:], &keysDst) keysDst = c.elideInLastStripe(span.Start, span.End, keysDst) usedLen += len(keysDst) output.Keys = append(output.Keys, keysDst...) diff --git a/internal/compact/spans_test.go b/internal/compact/spans_test.go index a42fe65d02..2a45c28d50 100644 --- a/internal/compact/spans_test.go +++ b/internal/compact/spans_test.go @@ -71,7 +71,7 @@ func TestRangeKeySpanCompactor(t *testing.T) { c = MakeRangeKeySpanCompactor( base.DefaultComparer.Compare, - base.DefaultComparer.Equal, + base.DefaultComparer.CompareSuffixes, s, ElideTombstonesOutsideOf(keyRanges), ) diff --git a/internal/crdbtest/crdbtest.go b/internal/crdbtest/crdbtest.go index 588779f468..57763362f5 100644 --- a/internal/crdbtest/crdbtest.go +++ b/internal/crdbtest/crdbtest.go @@ -8,6 +8,7 @@ package crdbtest import ( "bytes" + "cmp" "encoding/binary" "github.com/cockroachdb/errors" @@ -24,8 +25,10 @@ const MaxSuffixLen = max(withLockTableLen, withSynthetic, withLogical, withWall) // Comparer is a base.Comparer for CockroachDB keys. var Comparer = base.Comparer{ - Compare: Compare, - Equal: Equal, + Split: Split, + CompareSuffixes: CompareSuffixes, + Compare: Compare, + Equal: Equal, AbbreviatedKey: func(k []byte) uint64 { key, ok := getKeyPartFromEngineKey(k) if !ok { @@ -84,16 +87,13 @@ var Comparer = base.Comparer{ // zero-length version. return append(append(dst, a...), 0) }, - Split: Split, - Name: "cockroach_comparator", + Name: "cockroach_comparator", } // EncodeMVCCKey encodes a MVCC key into dst, growing dst as necessary. func EncodeMVCCKey(dst []byte, key []byte, walltime uint64, logical uint32) []byte { if cap(dst) < len(key)+withSynthetic { - newKey := make([]byte, len(key), len(key)+withSynthetic) - copy(newKey, key) - key = newKey + dst = make([]byte, 0, len(key)+withSynthetic) } dst = append(dst[:0], key...) return EncodeTimestamp(dst, walltime, logical) @@ -150,25 +150,23 @@ func DecodeTimestamp(mvccKey []byte) ([]byte, []byte, uint64, uint32) { // Split implements base.Split for CockroachDB keys. func Split(key []byte) int { - keyLen := len(key) - if keyLen == 0 { - return 0 + if len(key) == 0 { + panic(errors.AssertionFailedf("empty key")) } // Last byte is the version length + 1 when there is a version, else it is // 0. - versionLen := int(key[keyLen-1]) - keyPartEnd := keyLen - 1 - versionLen - if keyPartEnd < 0 { - return keyLen + versionLen := int(key[len(key)-1]) + if versionLen >= len(key) { + panic(errors.AssertionFailedf("empty key")) } - return keyPartEnd + 1 + return len(key) - versionLen } // Compare compares cockroach keys, including the version (which could be MVCC // timestamps). func Compare(a, b []byte) int { if len(a) == 0 || len(b) == 0 { - return bytes.Compare(a, b) + panic(errors.AssertionFailedf("malformed key: %x, %x", a, b)) } // NB: For performance, this routine manually splits the key into the @@ -189,13 +187,8 @@ func Compare(a, b []byte) int { // 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) + if aSep < 0 || bSep < 0 || a[aSep] != 0 || b[bSep] != 0 { + panic(errors.AssertionFailedf("malformed key: %x, %x", a, b)) } // Compare the "user key" part of the key. if c := bytes.Compare(a[:aSep], b[:bSep]); c != 0 { @@ -205,19 +198,19 @@ func Compare(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. - 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 CompareSuffixes(a[aSep+1:], b[bSep+1:]) +} + +// CompareSuffixes compares suffixes (normally timestamps). +func CompareSuffixes(a, b []byte) int { + if len(a) == 0 || len(b) == 0 { + // Empty suffixes come before non-empty suffixes. + return cmp.Compare(len(a), len(b)) } - aVer = normalizeEngineKeyVersionForCompare(aVer) - bVer = normalizeEngineKeyVersionForCompare(bVer) - return bytes.Compare(bVer, aVer) + return bytes.Compare( + normalizeEngineKeyVersionForCompare(b), + normalizeEngineKeyVersionForCompare(a), + ) } // Equal implements base.Equal for Cockroach keys. @@ -247,23 +240,17 @@ func Equal(a, b []byte) bool { // 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 } + if aVerLen == 0 || bVerLen == 0 { + return aVerLen == bVerLen + } // Compare the version part of the key. - aVer := a[aSep:aEnd] - bVer := b[bSep:bEnd] + aVer := a[aSep+1:] + bVer := b[bSep+1:] aVer = normalizeEngineKeyVersionForCompare(aVer) bVer = normalizeEngineKeyVersionForCompare(bVer) return bytes.Equal(aVer, bVer) @@ -272,24 +259,30 @@ func Equal(a, b []byte) bool { var zeroLogical [4]byte func normalizeEngineKeyVersionForCompare(a []byte) []byte { + // Check sentinel byte. + if len(a) != int(a[len(a)-1]) { + panic(errors.AssertionFailedf("malformed suffix: %x", a)) + } + // Strip off sentinel byte. + a = a[:len(a)-1] // 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. - if len(a) == withSynthetic { + if len(a) == withSynthetic-1 { // 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] + a = a[:withLogical-1] } - if len(a) == withLogical { + if len(a) == withLogical-1 { // 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[withWall-1:], zeroLogical[:]) { + a = a[:withWall-1] } } return a diff --git a/internal/crdbtest/crdbtest_test.go b/internal/crdbtest/crdbtest_test.go new file mode 100644 index 0000000000..e8adc69cc4 --- /dev/null +++ b/internal/crdbtest/crdbtest_test.go @@ -0,0 +1,56 @@ +// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package crdbtest + +import ( + "bytes" + "slices" + "testing" + + "github.com/cockroachdb/pebble/internal/base" +) + +func TestComparer(t *testing.T) { + prefixes := [][]byte{ + EncodeMVCCKey(nil, []byte("abc"), 0, 0), + EncodeMVCCKey(nil, []byte("d"), 0, 0), + EncodeMVCCKey(nil, []byte("ef"), 0, 0), + } + + suffixes := [][]byte{{}} + for walltime := 3; walltime > 0; walltime-- { + for logical := 2; logical >= 0; logical-- { + key := EncodeMVCCKey(nil, []byte("foo"), uint64(walltime), uint32(logical)) + suffix := key[Comparer.Split(key):] + suffixes = append(suffixes, suffix) + + if len(suffix) == withWall { + // Append a suffix that encodes a zero logical value that should be + // ignored in comparisons. + newSuffix := slices.Concat(suffix[:withWall-1], zeroLogical[:], []byte{withLogical}) + if Comparer.CompareSuffixes(suffix, newSuffix) != 0 { + t.Fatalf("expected suffixes %x and %x to be equal", suffix, newSuffix) + } + suffixes = append(suffixes, newSuffix) + suffix = newSuffix + } + if len(suffix) != withLogical { + t.Fatalf("unexpected suffix %x", suffix) + } + // Append a synthetic bit that should be ignored in comparisons. + newSuffix := slices.Concat(suffix[:withLogical-1], []byte{1}, []byte{withSynthetic}) + if Comparer.CompareSuffixes(suffix, newSuffix) != 0 { + t.Fatalf("expected suffixes %x and %x to be equal", suffix, newSuffix) + } + suffixes = append(suffixes, newSuffix) + } + } + // Add some lock table suffixes. + suffixes = append(suffixes, append(bytes.Repeat([]byte{1}, withLockTableLen-1), withLockTableLen)) + suffixes = append(suffixes, append(bytes.Repeat([]byte{2}, withLockTableLen-1), withLockTableLen)) + if err := base.CheckComparer(&Comparer, prefixes, suffixes); err != nil { + t.Error(err) + } +} diff --git a/internal/keyspan/defragment.go b/internal/keyspan/defragment.go index 35dda16de7..df8b7dac58 100644 --- a/internal/keyspan/defragment.go +++ b/internal/keyspan/defragment.go @@ -29,27 +29,29 @@ const keysReuseMaxCapacity = 100 type DefragmentMethod interface { // ShouldDefragment takes two abutting spans and returns whether the two // spans should be combined into a single, defragmented Span. - ShouldDefragment(equal base.Equal, left, right *Span) bool + ShouldDefragment(suffixCmp base.CompareSuffixes, left, right *Span) bool } // The DefragmentMethodFunc type is an adapter to allow the use of ordinary // functions as DefragmentMethods. If f is a function with the appropriate // signature, DefragmentMethodFunc(f) is a DefragmentMethod that calls f. -type DefragmentMethodFunc func(equal base.Equal, left, right *Span) bool +type DefragmentMethodFunc func(suffixCmp base.CompareSuffixes, left, right *Span) bool // ShouldDefragment calls f(equal, left, right). -func (f DefragmentMethodFunc) ShouldDefragment(equal base.Equal, left, right *Span) bool { - return f(equal, left, right) +func (f DefragmentMethodFunc) ShouldDefragment( + suffixCmp base.CompareSuffixes, left, right *Span, +) bool { + return f(suffixCmp, left, right) } -// DefragmentInternal configures a DefragmentingIter to defragment spans -// only if they have identical keys. It requires spans' keys to be sorted in -// trailer descending order. +// DefragmentInternal configures a DefragmentingIter to defragment spans only if +// they have identical keys. It requires spans' keys to be sorted in trailer +// descending order. // // This defragmenting method is intended for use in compactions that may see // internal range keys fragments that may now be joined, because the state that // required their fragmentation has been dropped. -var DefragmentInternal DefragmentMethod = DefragmentMethodFunc(func(equal base.Equal, a, b *Span) bool { +var DefragmentInternal DefragmentMethod = DefragmentMethodFunc(func(suffixCmp base.CompareSuffixes, a, b *Span) bool { if a.KeysOrder != ByTrailerDesc || b.KeysOrder != ByTrailerDesc { panic("pebble: span keys unexpectedly not in trailer descending order") } @@ -60,7 +62,7 @@ var DefragmentInternal DefragmentMethod = DefragmentMethodFunc(func(equal base.E if a.Keys[i].Trailer != b.Keys[i].Trailer { return false } - if !equal(a.Keys[i].Suffix, b.Keys[i].Suffix) { + if suffixCmp(a.Keys[i].Suffix, b.Keys[i].Suffix) != 0 { return false } if !bytes.Equal(a.Keys[i].Value, b.Keys[i].Value) { @@ -452,7 +454,7 @@ func (i *DefragmentingIter) Prev() (*Span, error) { // DefragmentMethod and ensures both spans are NOT empty; not defragmenting empty // spans is an optimization that lets us load fewer sstable blocks. func (i *DefragmentingIter) checkEqual(left, right *Span) bool { - return (!left.Empty() && !right.Empty()) && i.method.ShouldDefragment(i.equal, i.iterSpan, &i.curr) + return (!left.Empty() && !right.Empty()) && i.method.ShouldDefragment(i.comparer.CompareSuffixes, i.iterSpan, &i.curr) } // defragmentForward defragments spans in the forward direction, starting from diff --git a/internal/keyspan/defragment_test.go b/internal/keyspan/defragment_test.go index c43e194273..8e01781694 100644 --- a/internal/keyspan/defragment_test.go +++ b/internal/keyspan/defragment_test.go @@ -22,7 +22,7 @@ func TestDefragmentingIter(t *testing.T) { comparer := testkeys.Comparer cmp := comparer.Compare internalEqual := DefragmentInternal - alwaysEqual := DefragmentMethodFunc(func(_ base.Equal, _, _ *Span) bool { return true }) + alwaysEqual := DefragmentMethodFunc(func(_ base.CompareSuffixes, _, _ *Span) bool { return true }) staticReducer := StaticDefragmentReducer collectReducer := func(cur, next []Key) []Key { c := append(cur, next...) diff --git a/internal/keyspan/interleaving_iter_test.go b/internal/keyspan/interleaving_iter_test.go index 4d6ff575ac..143404cd6a 100644 --- a/internal/keyspan/interleaving_iter_test.go +++ b/internal/keyspan/interleaving_iter_test.go @@ -30,7 +30,7 @@ func TestInterleavingIter_Masking(t *testing.T) { type maskingHooks struct { log io.Writer - cmp base.Compare + suffixCmp base.CompareSuffixes split base.Split threshold []byte maskSuffix []byte @@ -55,10 +55,10 @@ func (m *maskingHooks) SpanChanged(s *Span) { if s.Keys[i].Suffix == nil { continue } - if m.cmp(s.Keys[i].Suffix, m.threshold) < 0 { + if m.suffixCmp(s.Keys[i].Suffix, m.threshold) < 0 { continue } - if m.maskSuffix == nil || m.cmp(m.maskSuffix, s.Keys[i].Suffix) > 0 { + if m.maskSuffix == nil || m.suffixCmp(m.maskSuffix, s.Keys[i].Suffix) > 0 { m.maskSuffix = s.Keys[i].Suffix } } @@ -66,7 +66,7 @@ func (m *maskingHooks) SpanChanged(s *Span) { func (m *maskingHooks) SkipPoint(userKey []byte) bool { pointSuffix := userKey[m.split(userKey):] - return m.maskSuffix != nil && len(pointSuffix) > 0 && m.cmp(m.maskSuffix, pointSuffix) < 0 + return m.maskSuffix != nil && len(pointSuffix) > 0 && m.suffixCmp(m.maskSuffix, pointSuffix) < 0 } func runInterleavingIterTest(t *testing.T, filename string) { @@ -76,9 +76,9 @@ func runInterleavingIterTest(t *testing.T, filename string) { var iter InterleavingIter var buf bytes.Buffer hooks := maskingHooks{ - log: &buf, - cmp: testkeys.Comparer.Compare, - split: testkeys.Comparer.Split, + log: &buf, + suffixCmp: testkeys.Comparer.CompareSuffixes, + split: testkeys.Comparer.Split, } var prevKV *base.InternalKV diff --git a/internal/keyspan/keyspanimpl/merging_iter.go b/internal/keyspan/keyspanimpl/merging_iter.go index 9fd8796050..edcd0600ed 100644 --- a/internal/keyspan/keyspanimpl/merging_iter.go +++ b/internal/keyspan/keyspanimpl/merging_iter.go @@ -1047,7 +1047,7 @@ func (m *MergingIter) synthesizeKeys(dir int8) (bool, *keyspan.Span, error) { Keys: m.keys, KeysOrder: keyspan.ByTrailerDesc, } - if err := m.transformer.Transform(m.comparer.Compare, m.span, &m.span); err != nil { + if err := m.transformer.Transform(m.comparer.CompareSuffixes, m.span, &m.span); err != nil { return false, nil, err } return found, &m.span, nil diff --git a/internal/keyspan/span.go b/internal/keyspan/span.go index c2464b4c5c..8f41ef964d 100644 --- a/internal/keyspan/span.go +++ b/internal/keyspan/span.go @@ -93,9 +93,9 @@ func (k Key) Kind() base.InternalKeyKind { // Equal returns true if this Key is equal to the given key. Two keys are said // to be equal if the two Keys have equal trailers, suffix and value. Suffix // comparison uses the provided base.Compare func. Value comparison is bytewise. -func (k Key) Equal(equal base.Equal, b Key) bool { +func (k Key) Equal(suffixCmp base.CompareSuffixes, b Key) bool { return k.Trailer == b.Trailer && - equal(k.Suffix, b.Suffix) && + suffixCmp(k.Suffix, b.Suffix) == 0 && bytes.Equal(k.Value, b.Value) } @@ -464,9 +464,9 @@ func SortKeysByTrailer(keys []Key) { } // SortKeysBySuffix sorts a Keys slice by suffix. -func SortKeysBySuffix(cmp base.Compare, keys []Key) { +func SortKeysBySuffix(suffixCmp base.CompareSuffixes, keys []Key) { slices.SortFunc(keys, func(a, b Key) int { - return cmp(a.Suffix, b.Suffix) + return suffixCmp(a.Suffix, b.Suffix) }) } diff --git a/internal/keyspan/transformer.go b/internal/keyspan/transformer.go index df7c096c36..037a2e30b6 100644 --- a/internal/keyspan/transformer.go +++ b/internal/keyspan/transformer.go @@ -11,21 +11,21 @@ type Transformer interface { // Transform takes a Span as input and writes the transformed Span to the // provided output *Span pointer. The output Span's Keys slice may be reused // by Transform to reduce allocations. - Transform(cmp base.Compare, in Span, out *Span) error + Transform(suffixCmp base.CompareSuffixes, in Span, out *Span) error } // The TransformerFunc type is an adapter to allow the use of ordinary functions // as Transformers. If f is a function with the appropriate signature, // TransformerFunc(f) is a Transformer that calls f. -type TransformerFunc func(base.Compare, Span, *Span) error +type TransformerFunc func(base.CompareSuffixes, Span, *Span) error // Transform calls f(cmp, in, out). -func (tf TransformerFunc) Transform(cmp base.Compare, in Span, out *Span) error { - return tf(cmp, in, out) +func (tf TransformerFunc) Transform(suffixCmp base.CompareSuffixes, in Span, out *Span) error { + return tf(suffixCmp, in, out) } // NoopTransform is a Transformer that performs no mutations. -var NoopTransform Transformer = TransformerFunc(func(_ base.Compare, s Span, dst *Span) error { +var NoopTransform Transformer = TransformerFunc(func(_ base.CompareSuffixes, s Span, dst *Span) error { dst.Start, dst.End = s.Start, s.End dst.Keys = append(dst.Keys[:0], s.Keys...) return nil @@ -34,7 +34,7 @@ var NoopTransform Transformer = TransformerFunc(func(_ base.Compare, s Span, dst // VisibleTransform filters keys that are invisible at the provided snapshot // sequence number. func VisibleTransform(snapshot base.SeqNum) Transformer { - return TransformerFunc(func(_ base.Compare, s Span, dst *Span) error { + return TransformerFunc(func(_ base.CompareSuffixes, s Span, dst *Span) error { dst.Start, dst.End = s.Start, s.End dst.Keys = dst.Keys[:0] for _, k := range s.Keys { @@ -58,8 +58,8 @@ type TransformerIter struct { // Transformer is applied on every Span returned by this iterator. Transformer Transformer - // Comparer in use for this keyspace. - Compare base.Compare + // Suffix comparer in use for this keyspace. + SuffixCmp base.CompareSuffixes span Span } @@ -73,7 +73,7 @@ func (t *TransformerIter) applyTransform(span *Span) (*Span, error) { End: t.span.End[:0], Keys: t.span.Keys[:0], } - if err := t.Transformer.Transform(t.Compare, *span, &t.span); err != nil { + if err := t.Transformer.Transform(t.SuffixCmp, *span, &t.span); err != nil { return nil, err } return &t.span, nil diff --git a/internal/rangekey/coalesce.go b/internal/rangekey/coalesce.go index a23a4a5ff4..c89e7542f5 100644 --- a/internal/rangekey/coalesce.go +++ b/internal/rangekey/coalesce.go @@ -49,10 +49,10 @@ import ( // set or unset but not both. // // The resulting dst Keys slice is sorted by InternalKeyTrailer. -func Coalesce(cmp base.Compare, eq base.Equal, keys []keyspan.Key, dst *[]keyspan.Key) { +func Coalesce(suffixCmp base.CompareSuffixes, keys []keyspan.Key, dst *[]keyspan.Key) { // TODO(jackson): Currently, Coalesce doesn't actually perform the sequence // number promotion described in the comment above. - *dst = CoalesceInto(cmp, eq, (*dst)[:0], math.MaxUint64, keys) + *dst = CoalesceInto(suffixCmp, (*dst)[:0], math.MaxUint64, keys) // Update the span with the (potentially reduced) keys slice. coalesce left // the keys in *dst sorted by suffix. Re-sort them by trailer. keyspan.SortKeysByTrailer(*dst) @@ -61,7 +61,7 @@ func Coalesce(cmp base.Compare, eq base.Equal, keys []keyspan.Key, dst *[]keyspa // CoalesceInto is a variant of Coalesce which outputs the results into dst // without sorting them. func CoalesceInto( - cmp base.Compare, equal base.Equal, dst []keyspan.Key, snapshot base.SeqNum, keys []keyspan.Key, + suffixCmp base.CompareSuffixes, dst []keyspan.Key, snapshot base.SeqNum, keys []keyspan.Key, ) []keyspan.Key { dst = dst[:0] // First, enforce visibility and RangeKeyDelete mechanics. We only need to @@ -100,7 +100,7 @@ func CoalesceInto( // that with the highest InternalKeyTrailer (because the input `keys` was sorted by // trailer descending). slices.SortStableFunc(dst, func(a, b keyspan.Key) int { - return cmp(a.Suffix, b.Suffix) + return suffixCmp(a.Suffix, b.Suffix) }) // Grab a handle of the full sorted slice, before reslicing @@ -121,7 +121,7 @@ func CoalesceInto( shadowing bool ) for i := range sorted { - if i > 0 && equal(prevSuffix, sorted[i].Suffix) { + if i > 0 && suffixCmp(prevSuffix, sorted[i].Suffix) == 0 { // Skip; this key is shadowed by the predecessor that had a larger // InternalKeyTrailer. If this is the first shadowed key, set shadowing=true // and reslice keysBySuffix.keys to hold the entire unshadowed @@ -165,23 +165,23 @@ type ForeignSSTTransformer struct { // Transform implements the Transformer interface. func (f *ForeignSSTTransformer) Transform( - cmp base.Compare, s keyspan.Span, dst *keyspan.Span, + suffixCmp base.CompareSuffixes, s keyspan.Span, dst *keyspan.Span, ) error { // Apply shadowing of keys. dst.Start = s.Start dst.End = s.End f.sortBuf = f.sortBuf[:0] - f.sortBuf = CoalesceInto(cmp, f.Equal, f.sortBuf, math.MaxUint64, s.Keys) + f.sortBuf = CoalesceInto(suffixCmp, f.sortBuf, math.MaxUint64, s.Keys) keys := f.sortBuf dst.Keys = dst.Keys[:0] for i := range keys { switch keys[i].Kind() { case base.InternalKeyKindRangeKeySet: - if invariants.Enabled && len(dst.Keys) > 0 && cmp(dst.Keys[len(dst.Keys)-1].Suffix, keys[i].Suffix) > 0 { + if invariants.Enabled && len(dst.Keys) > 0 && suffixCmp(dst.Keys[len(dst.Keys)-1].Suffix, keys[i].Suffix) > 0 { panic("pebble: keys unexpectedly not in ascending suffix order") } case base.InternalKeyKindRangeKeyUnset: - if invariants.Enabled && len(dst.Keys) > 0 && cmp(dst.Keys[len(dst.Keys)-1].Suffix, keys[i].Suffix) > 0 { + if invariants.Enabled && len(dst.Keys) > 0 && suffixCmp(dst.Keys[len(dst.Keys)-1].Suffix, keys[i].Suffix) > 0 { panic("pebble: keys unexpectedly not in ascending suffix order") } case base.InternalKeyKindRangeKeyDelete: diff --git a/internal/rangekey/coalesce_test.go b/internal/rangekey/coalesce_test.go index 177969dc29..b76a6b45e2 100644 --- a/internal/rangekey/coalesce_test.go +++ b/internal/rangekey/coalesce_test.go @@ -5,7 +5,6 @@ package rangekey import ( - "bytes" "fmt" "testing" @@ -15,22 +14,17 @@ import ( ) func TestCoalesce(t *testing.T) { - var buf bytes.Buffer - eq := testkeys.Comparer.Equal - cmp := testkeys.Comparer.Compare - datadriven.RunTest(t, "testdata/coalesce", func(t *testing.T, td *datadriven.TestData) string { switch td.Cmd { case "coalesce": - buf.Reset() span := keyspan.ParseSpan(td.Input) coalesced := keyspan.Span{ Start: span.Start, End: span.End, } - Coalesce(cmp, eq, span.Keys, &coalesced.Keys) - fmt.Fprintln(&buf, coalesced) - return buf.String() + Coalesce(testkeys.Comparer.CompareSuffixes, span.Keys, &coalesced.Keys) + return coalesced.String() + default: return fmt.Sprintf("unrecognized command %q", td.Cmd) } diff --git a/internal/rangekeystack/user_iterator.go b/internal/rangekeystack/user_iterator.go index cbcd3f7177..232dba6de1 100644 --- a/internal/rangekeystack/user_iterator.go +++ b/internal/rangekeystack/user_iterator.go @@ -143,11 +143,13 @@ func (ui *UserIteratorConfig) SetBounds(lower, upper []byte) { // only contain RangeKeySets describing the state visible at the provided // sequence number, and hold their Keys sorted by Suffix (except if internalKeys // is true, then keys remain sorted by trailer. -func (ui *UserIteratorConfig) Transform(cmp base.Compare, s keyspan.Span, dst *keyspan.Span) error { +func (ui *UserIteratorConfig) Transform( + suffixCmp base.CompareSuffixes, s keyspan.Span, dst *keyspan.Span, +) error { // Apply shadowing of keys. dst.Start = s.Start dst.End = s.End - ui.bufs.sortBuf = rangekey.CoalesceInto(cmp, ui.comparer.Equal, ui.bufs.sortBuf[:0], ui.snapshot, s.Keys) + ui.bufs.sortBuf = rangekey.CoalesceInto(suffixCmp, ui.bufs.sortBuf[:0], ui.snapshot, s.Keys) if ui.internalKeys { if s.KeysOrder != keyspan.ByTrailerDesc { panic("unexpected key ordering in UserIteratorTransform with internalKeys = true") @@ -163,12 +165,12 @@ func (ui *UserIteratorConfig) Transform(cmp base.Compare, s keyspan.Span, dst *k for i := range keys { switch keys[i].Kind() { case base.InternalKeyKindRangeKeySet: - if invariants.Enabled && len(dst.Keys) > 0 && cmp(dst.Keys[len(dst.Keys)-1].Suffix, keys[i].Suffix) > 0 { + if invariants.Enabled && len(dst.Keys) > 0 && suffixCmp(dst.Keys[len(dst.Keys)-1].Suffix, keys[i].Suffix) > 0 { panic("pebble: keys unexpectedly not in ascending suffix order") } dst.Keys = append(dst.Keys, keys[i]) case base.InternalKeyKindRangeKeyUnset: - if invariants.Enabled && len(dst.Keys) > 0 && cmp(dst.Keys[len(dst.Keys)-1].Suffix, keys[i].Suffix) > 0 { + if invariants.Enabled && len(dst.Keys) > 0 && suffixCmp(dst.Keys[len(dst.Keys)-1].Suffix, keys[i].Suffix) > 0 { panic("pebble: keys unexpectedly not in ascending suffix order") } // Skip. @@ -193,7 +195,9 @@ func (ui *UserIteratorConfig) Transform(cmp base.Compare, s keyspan.Span, dst *k // defragmenter checks for equality between set suffixes and values (ignoring // sequence numbers). It's intended for use during user iteration, when the // wrapped keyspan iterator is merging spans across all levels of the LSM. -func (ui *UserIteratorConfig) ShouldDefragment(equal base.Equal, a, b *keyspan.Span) bool { +func (ui *UserIteratorConfig) ShouldDefragment( + suffixCmp base.CompareSuffixes, a, b *keyspan.Span, +) bool { // This method is not called with internalKeys = true. if ui.internalKeys { panic("unexpected call to ShouldDefragment with internalKeys = true") @@ -217,12 +221,12 @@ func (ui *UserIteratorConfig) ShouldDefragment(equal base.Equal, a, b *keyspan.S b.Keys[i].Kind() != base.InternalKeyKindRangeKeySet { panic("pebble: unexpected non-RangeKeySet during defragmentation") } - if i > 0 && (ui.comparer.Compare(a.Keys[i].Suffix, a.Keys[i-1].Suffix) < 0 || - ui.comparer.Compare(b.Keys[i].Suffix, b.Keys[i-1].Suffix) < 0) { + if i > 0 && (suffixCmp(a.Keys[i].Suffix, a.Keys[i-1].Suffix) < 0 || + suffixCmp(b.Keys[i].Suffix, b.Keys[i-1].Suffix) < 0) { panic("pebble: range keys not ordered by suffix during defragmentation") } } - if !equal(a.Keys[i].Suffix, b.Keys[i].Suffix) { + if suffixCmp(a.Keys[i].Suffix, b.Keys[i].Suffix) != 0 { ret = false break } diff --git a/internal/rangekeystack/user_iterator_test.go b/internal/rangekeystack/user_iterator_test.go index 145b21900c..134acc1c7e 100644 --- a/internal/rangekeystack/user_iterator_test.go +++ b/internal/rangekeystack/user_iterator_test.go @@ -25,7 +25,6 @@ import ( ) func TestIter(t *testing.T) { - eq := testkeys.Comparer.Equal cmp := testkeys.Comparer.Compare var iter keyspanimpl.MergingIter var buf bytes.Buffer @@ -46,8 +45,8 @@ func TestIter(t *testing.T) { for _, line := range lines { spans = append(spans, keyspan.ParseSpan(line)) } - transform := keyspan.TransformerFunc(func(cmp base.Compare, s keyspan.Span, dst *keyspan.Span) error { - dst.Keys = rangekey.CoalesceInto(cmp, eq, dst.Keys[:0], visibleSeqNum, s.Keys) + transform := keyspan.TransformerFunc(func(suffixCmp base.CompareSuffixes, s keyspan.Span, dst *keyspan.Span) error { + dst.Keys = rangekey.CoalesceInto(suffixCmp, dst.Keys[:0], visibleSeqNum, s.Keys) // Update the span with the (potentially reduced) keys slice. // CoalesceInto() left the keys sorted by suffix. Re-sort them by // trailer. @@ -376,7 +375,7 @@ func BenchmarkTransform(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - err := ui.Transform(testkeys.Comparer.Compare, keyspan.Span{Keys: keys}, &dst) + err := ui.Transform(testkeys.Comparer.CompareSuffixes, keyspan.Span{Keys: keys}, &dst) if err != nil { b.Fatal(err) } diff --git a/internal/testkeys/testkeys.go b/internal/testkeys/testkeys.go index afaf6b2f1f..b3ed5c70b9 100644 --- a/internal/testkeys/testkeys.go +++ b/internal/testkeys/testkeys.go @@ -44,8 +44,9 @@ var MaxSuffixLen = 1 + len(fmt.Sprintf("%d", int64(math.MaxInt64))) // Comparer is the comparer for test keys generated by this package. var Comparer = &base.Comparer{ - Compare: compare, - Equal: func(a, b []byte) bool { return compare(a, b) == 0 }, + CompareSuffixes: compareTimestamps, + Compare: compare, + Equal: func(a, b []byte) bool { return compare(a, b) == 0 }, AbbreviatedKey: func(k []byte) uint64 { return base.DefaultComparer.AbbreviatedKey(k[:split(k)]) }, @@ -109,35 +110,42 @@ var Comparer = &base.Comparer{ // value is smaller. func compare(a, b []byte) int { ai, bi := split(a), split(b) + if ai == 0 && len(a) > 0 { + panic(fmt.Sprintf("Compare called with bare suffix %s", a)) + } + if bi == 0 && len(b) > 0 { + panic(fmt.Sprintf("Compare called with bare suffix %s", b)) + } if v := bytes.Compare(a[:ai], b[:bi]); v != 0 { return v } - - if len(a[ai:]) == 0 { - if len(b[bi:]) == 0 { - return 0 - } - return -1 - } else if len(b[bi:]) == 0 { - return +1 - } return compareTimestamps(a[ai:], b[bi:]) } func split(a []byte) int { i := bytes.LastIndexByte(a, suffixDelim) - if i >= 0 { + if i == 0 { + panic("split called on bare prefix") + } + if i > 0 { return i } return len(a) } func compareTimestamps(a, b []byte) int { - ai, err := parseUintBytes(bytes.TrimPrefix(a, []byte{suffixDelim}), 10, 64) + if len(a) == 0 || len(b) == 0 { + // The empty suffix sorts first. + return cmp.Compare(len(a), len(b)) + } + if a[0] != suffixDelim || b[0] != suffixDelim { + panic(fmt.Sprintf("invalid suffixes %q %q", a, b)) + } + ai, err := parseUintBytes(a[1:], 10, 64) if err != nil { panic(fmt.Sprintf("invalid test mvcc timestamp %q", a)) } - bi, err := parseUintBytes(bytes.TrimPrefix(b, []byte{suffixDelim}), 10, 64) + bi, err := parseUintBytes(b[1:], 10, 64) if err != nil { panic(fmt.Sprintf("invalid test mvcc timestamp %q", b)) } diff --git a/internal/testkeys/testkeys_test.go b/internal/testkeys/testkeys_test.go index 7573eeb75f..75f9a22106 100644 --- a/internal/testkeys/testkeys_test.go +++ b/internal/testkeys/testkeys_test.go @@ -11,6 +11,7 @@ import ( "testing" "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/pebble/internal/base" "github.com/stretchr/testify/require" "golang.org/x/exp/rand" ) @@ -157,12 +158,15 @@ func TestSuffix(t *testing.T) { } } - // Suffixes should be comparable on their own too. + // Test CompareSuffixes. a, b := make([]byte, MaxSuffixLen), make([]byte, MaxSuffixLen) for ts := int64(2); ts < 150; ts++ { an := WriteSuffix(a, ts-1) bn := WriteSuffix(b, ts) - assertCmp(+1, a[:an], b[:bn]) + got := Comparer.CompareSuffixes(a[:an], b[:bn]) + if want := +1; got != want { + t.Errorf("CompareSuffixes(%q, %q) = %d, want %d", a, b, got, want) + } } } @@ -294,3 +298,11 @@ func TestOverflowPanic(t *testing.T) { }() keyCount(6, len(alpha)) } + +func TestComparer(t *testing.T) { + if err := base.CheckComparer(Comparer, + [][]byte{[]byte("abc"), []byte("d"), []byte("ef")}, + [][]byte{{}, []byte("@3"), []byte("@2"), []byte("@1")}); err != nil { + t.Error(err) + } +} diff --git a/iterator.go b/iterator.go index e37de7174c..ee85d4d027 100644 --- a/iterator.go +++ b/iterator.go @@ -2199,7 +2199,7 @@ func (i *Iterator) saveRangeKey() { if invariants.Enabled { if s.Keys[j].Kind() != base.InternalKeyKindRangeKeySet { panic("pebble: user iteration encountered non-RangeKeySet key kind") - } else if j > 0 && i.cmp(s.Keys[j].Suffix, s.Keys[j-1].Suffix) < 0 { + } else if j > 0 && i.comparer.CompareSuffixes(s.Keys[j].Suffix, s.Keys[j-1].Suffix) < 0 { panic("pebble: user iteration encountered range keys not in suffix order") } } @@ -2685,7 +2685,7 @@ func (i *Iterator) SetOptions(o *IterOptions) { if boundsEqual && o.KeyTypes == i.opts.KeyTypes && (i.pointIter != nil || !i.opts.pointKeys()) && (i.rangeKey != nil || !i.opts.rangeKeys() || i.opts.KeyTypes == IterKeyTypePointsAndRanges) && - i.equal(o.RangeKeyMasking.Suffix, i.opts.RangeKeyMasking.Suffix) && + i.comparer.CompareSuffixes(o.RangeKeyMasking.Suffix, i.opts.RangeKeyMasking.Suffix) == 0 && o.UseL6Filters == i.opts.UseL6Filters { // The options are identical, so we can likely use the fast path. In // addition to all the above constraints, we cannot use the fast path if diff --git a/metamorphic/build.go b/metamorphic/build.go index e18e0f60aa..3973929cd7 100644 --- a/metamorphic/build.go +++ b/metamorphic/build.go @@ -135,9 +135,7 @@ func writeSSTForIngestion( End: outputKey(span.End), Keys: make([]keyspan.Key, 0, len(span.Keys)), } - rangekey.Coalesce( - t.opts.Comparer.Compare, t.opts.Comparer.Equal, span.Keys, &collapsed.Keys, - ) + rangekey.Coalesce(t.opts.Comparer.CompareSuffixes, span.Keys, &collapsed.Keys) for i := range collapsed.Keys { collapsed.Keys[i].Trailer = base.MakeTrailer(0, collapsed.Keys[i].Kind()) } diff --git a/range_keys.go b/range_keys.go index 52640a4158..b50f562404 100644 --- a/range_keys.go +++ b/range_keys.go @@ -225,9 +225,10 @@ func containsAnyRangeKeys(iter manifest.LevelIterator) bool { // result is ignored, and the block is read. type rangeKeyMasking struct { - cmp base.Compare - split base.Split - filter BlockPropertyFilterMask + cmp base.Compare + suffixCmp base.CompareSuffixes + split base.Split + filter BlockPropertyFilterMask // maskActiveSuffix holds the suffix of a range key currently acting as a // mask, hiding point keys with suffixes greater than it. maskActiveSuffix // is only ever non-nil if IterOptions.RangeKeyMasking.Suffix is non-nil. @@ -243,9 +244,10 @@ type rangeKeyMasking struct { parent *Iterator } -func (m *rangeKeyMasking) init(parent *Iterator, cmp base.Compare, split base.Split) { - m.cmp = cmp - m.split = split +func (m *rangeKeyMasking) init(parent *Iterator, c *base.Comparer) { + m.cmp = c.Compare + m.suffixCmp = c.CompareSuffixes + m.split = c.Split if parent.opts.RangeKeyMasking.Filter != nil { m.filter = parent.opts.RangeKeyMasking.Filter() } @@ -270,10 +272,10 @@ func (m *rangeKeyMasking) SpanChanged(s *keyspan.Span) { if s.Keys[j].Suffix == nil { continue } - if m.cmp(s.Keys[j].Suffix, m.parent.opts.RangeKeyMasking.Suffix) < 0 { + if m.suffixCmp(s.Keys[j].Suffix, m.parent.opts.RangeKeyMasking.Suffix) < 0 { continue } - if len(m.maskActiveSuffix) == 0 || m.cmp(m.maskActiveSuffix, s.Keys[j].Suffix) > 0 { + if len(m.maskActiveSuffix) == 0 || m.suffixCmp(m.maskActiveSuffix, s.Keys[j].Suffix) > 0 { m.maskSpan = s m.maskActiveSuffix = append(m.maskActiveSuffix[:0], s.Keys[j].Suffix...) } @@ -352,7 +354,7 @@ func (m *rangeKeyMasking) SkipPoint(userKey []byte) bool { // the InterleavingIter). Skip the point key if the range key's suffix is // greater than the point key's suffix. pointSuffix := userKey[m.split(userKey):] - if len(pointSuffix) > 0 && m.cmp(m.maskActiveSuffix, pointSuffix) < 0 { + if len(pointSuffix) > 0 && m.suffixCmp(m.maskActiveSuffix, pointSuffix) < 0 { m.parent.stats.RangeKeyStats.SkippedPoints++ return true } diff --git a/sstable/comparer.go b/sstable/comparer.go index 66a20b51e7..62b5273090 100644 --- a/sstable/comparer.go +++ b/sstable/comparer.go @@ -9,6 +9,9 @@ import "github.com/cockroachdb/pebble/internal/base" // Compare exports the base.Compare type. type Compare = base.Compare +// CompareSuffixes exports the base.CompareSuffixes type. +type CompareSuffixes = base.CompareSuffixes + // Equal exports the base.Equal type. type Equal = base.Equal diff --git a/sstable/data_test.go b/sstable/data_test.go index cdcad0831e..6c03acac88 100644 --- a/sstable/data_test.go +++ b/sstable/data_test.go @@ -293,7 +293,7 @@ func runIterCmd( for iter.Valid() { k := iter.Key().UserKey suffix := k[testkeys.Comparer.Split(k):] - if len(suffix) == 0 || testkeys.Comparer.Compare(suffix, maskingSuffix) <= 0 { + if len(suffix) == 0 || testkeys.Comparer.CompareSuffixes(suffix, maskingSuffix) <= 0 { return } if direction > 0 { diff --git a/sstable/raw_writer.go b/sstable/raw_writer.go index 74074758a3..d7aa8ec710 100644 --- a/sstable/raw_writer.go +++ b/sstable/raw_writer.go @@ -131,6 +131,7 @@ type RawWriter struct { indexBlockOptions flushDecisionOptions // The following fields are copied from Options. compare Compare + suffixCmp CompareSuffixes split Split formatKey base.FormatKey compression block.Compression @@ -750,7 +751,7 @@ func (w *RawWriter) makeAddPointDecisionV3( cmpUser = cmpPrefix if cmpPrefix == 0 { // Need to compare suffixes to compute cmpUser. - cmpUser = w.compare(prevPointUserKey[prevPointKeyInfo.prefixLen:], + cmpUser = w.suffixCmp(prevPointUserKey[prevPointKeyInfo.prefixLen:], key.UserKey[w.lastPointKeyInfo.prefixLen:]) } } else { @@ -1029,7 +1030,7 @@ func (w *RawWriter) encodeFragmentedRangeKeySpan(span keyspan.Span) { // Sort the keys by suffix. Iteration doesn't *currently* depend on it, but // we may want to in the future. w.rangeKeySpan = span - keyspan.SortKeysBySuffix(w.compare, w.rangeKeySpan.Keys) + keyspan.SortKeysBySuffix(w.suffixCmp, w.rangeKeySpan.Keys) if w.err == nil { w.err = w.EncodeSpan(w.rangeKeySpan) @@ -1907,6 +1908,7 @@ func NewRawWriter(writable objstorage.Writable, o WriterOptions) *RawWriter { sizeClassAwareThreshold: (o.IndexBlockSize*o.SizeClassAwareThreshold + 99) / 100, }, compare: o.Comparer.Compare, + suffixCmp: o.Comparer.CompareSuffixes, split: o.Comparer.Split, formatKey: o.Comparer.FormatKey, compression: o.Compression, diff --git a/sstable/reader.go b/sstable/reader.go index cbec87d46a..029fe0c07a 100644 --- a/sstable/reader.go +++ b/sstable/reader.go @@ -101,6 +101,7 @@ type Reader struct { logger base.LoggerAndTracer Compare Compare + SuffixCmp CompareSuffixes Equal Equal FormatKey base.FormatKey Split Split @@ -979,11 +980,13 @@ func NewReader(ctx context.Context, f objstorage.Readable, o ReaderOptions) (*Re if r.Properties.ComparerName == "" || o.Comparer.Name == r.Properties.ComparerName { r.Compare = o.Comparer.Compare + r.SuffixCmp = o.Comparer.CompareSuffixes r.Equal = o.Comparer.Equal r.FormatKey = o.Comparer.FormatKey r.Split = o.Comparer.Split } else if comparer, ok := o.Comparers[r.Properties.ComparerName]; ok { r.Compare = comparer.Compare + r.SuffixCmp = comparer.CompareSuffixes r.Equal = comparer.Equal r.FormatKey = comparer.FormatKey r.Split = comparer.Split diff --git a/sstable/reader_virtual.go b/sstable/reader_virtual.go index 771cb6a35b..37dd6770a7 100644 --- a/sstable/reader_virtual.go +++ b/sstable/reader_virtual.go @@ -190,7 +190,7 @@ func (v *VirtualReader) NewRawRangeKeyIter( transformIter := &keyspan.TransformerIter{ FragmentIterator: iter, Transformer: transform, - Compare: v.reader.Compare, + SuffixCmp: v.reader.SuffixCmp, } iter = transformIter } diff --git a/table_stats.go b/table_stats.go index 7d9d402ee6..cfd1af0f46 100644 --- a/table_stats.go +++ b/table_stats.go @@ -850,7 +850,7 @@ func newCombinedDeletionKeyspanIter( ) (keyspan.FragmentIterator, error) { // The range del iter and range key iter are each wrapped in their own // defragmenting iter. For each iter, abutting spans can always be merged. - var equal = keyspan.DefragmentMethodFunc(func(_ base.Equal, a, b *keyspan.Span) bool { return true }) + var equal = keyspan.DefragmentMethodFunc(func(_ base.CompareSuffixes, a, b *keyspan.Span) bool { return true }) // Reduce keys by maintaining a slice of at most length two, corresponding to // the largest and smallest keys in the defragmented span. This maintains the // contract that the emitted slice is sorted by (SeqNum, Kind) descending. @@ -879,7 +879,7 @@ func newCombinedDeletionKeyspanIter( smallest = last } } - if largest.Equal(comparer.Equal, smallest) { + if largest.Equal(comparer.CompareSuffixes, smallest) { current = append(current[:0], largest) } else { current = append(current[:0], largest, smallest) @@ -891,7 +891,7 @@ func newCombinedDeletionKeyspanIter( // merging iter to join the keyspaces into a single keyspace. The separate // iters are only added if the particular key kind is present. mIter := &keyspanimpl.MergingIter{} - var transform = keyspan.TransformerFunc(func(cmp base.Compare, in keyspan.Span, out *keyspan.Span) error { + var transform = keyspan.TransformerFunc(func(_ base.CompareSuffixes, in keyspan.Span, out *keyspan.Span) error { if in.KeysOrder != keyspan.ByTrailerDesc { panic("pebble: combined deletion iter encountered keys in non-trailer descending order") } diff --git a/testdata/ingest b/testdata/ingest index 354e50e978..48930711d1 100644 --- a/testdata/ingest +++ b/testdata/ingest @@ -54,7 +54,7 @@ Virtual tables: 0 (0B) Local tables size: 569B Compression types: snappy: 1 Block cache: 6 entries (945B) hit rate: 30.8% -Table cache: 1 entries (728B) hit rate: 50.0% +Table cache: 1 entries (736B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 diff --git a/testdata/metrics b/testdata/metrics index d583567c6b..5760286df0 100644 --- a/testdata/metrics +++ b/testdata/metrics @@ -75,7 +75,7 @@ Virtual tables: 0 (0B) Local tables size: 589B Compression types: snappy: 1 Block cache: 3 entries (484B) hit rate: 0.0% -Table cache: 1 entries (728B) hit rate: 0.0% +Table cache: 1 entries (736B) hit rate: 0.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 1 @@ -214,7 +214,7 @@ Virtual tables: 0 (0B) Local tables size: 595B Compression types: snappy: 1 Block cache: 3 entries (484B) hit rate: 33.3% -Table cache: 1 entries (728B) hit rate: 66.7% +Table cache: 1 entries (736B) hit rate: 66.7% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 1 @@ -488,7 +488,7 @@ Virtual tables: 0 (0B) Local tables size: 4.3KB Compression types: snappy: 7 Block cache: 12 entries (1.9KB) hit rate: 9.1% -Table cache: 1 entries (728B) hit rate: 53.8% +Table cache: 1 entries (736B) hit rate: 53.8% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -551,7 +551,7 @@ Virtual tables: 0 (0B) Local tables size: 6.1KB Compression types: snappy: 10 Block cache: 12 entries (1.9KB) hit rate: 9.1% -Table cache: 1 entries (728B) hit rate: 53.8% +Table cache: 1 entries (736B) hit rate: 53.8% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -822,7 +822,7 @@ Virtual tables: 0 (0B) Local tables size: 0B Compression types: snappy: 1 Block cache: 1 entries (440B) hit rate: 0.0% -Table cache: 1 entries (728B) hit rate: 0.0% +Table cache: 1 entries (736B) hit rate: 0.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -869,7 +869,7 @@ Virtual tables: 0 (0B) Local tables size: 0B Compression types: snappy: 2 Block cache: 6 entries (996B) hit rate: 0.0% -Table cache: 1 entries (728B) hit rate: 50.0% +Table cache: 1 entries (736B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -917,7 +917,7 @@ Virtual tables: 0 (0B) Local tables size: 589B Compression types: snappy: 3 Block cache: 6 entries (996B) hit rate: 0.0% -Table cache: 1 entries (728B) hit rate: 50.0% +Table cache: 1 entries (736B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0