diff --git a/internal/base/internal.go b/internal/base/internal.go index 7d7a12d2af..c168396bb0 100644 --- a/internal/base/internal.go +++ b/internal/base/internal.go @@ -247,6 +247,12 @@ func (t InternalKeyTrailer) Kind() InternalKeyKind { return InternalKeyKind(t & 0xff) } +// IsExclusiveSentinel returns true if the trailer is a sentinel for an +// exclusive boundary. +func (t InternalKeyTrailer) IsExclusiveSentinel() bool { + return t.SeqNum() == SeqNumMax +} + // InternalKey is a key used for the in-memory and on-disk partial DBs that // make up a pebble DB. // diff --git a/internal/manifest/bench_test.go b/internal/manifest/bench_test.go new file mode 100644 index 0000000000..f32b5ed945 --- /dev/null +++ b/internal/manifest/bench_test.go @@ -0,0 +1,49 @@ +// Copyright 2025 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 manifest_test + +import ( + "math/rand/v2" + "testing" + "time" + + "github.com/cockroachdb/pebble/cockroachkvs" + "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/manifest" +) + +func BenchmarkLevelIteratorSeekGE(b *testing.B) { + const countTables = 10_000 + fileAlloc := make([]manifest.TableMetadata, countTables) + files := make([]*manifest.TableMetadata, countTables) + rng := rand.New(rand.NewPCG(0, uint64(time.Now().UnixNano()))) + keys, _ := cockroachkvs.RandomKVs(rng, 2*countTables, cockroachkvs.KeyGenConfig{ + PrefixAlphabetLen: 26, + PrefixLenShared: 2, + RoachKeyLen: 16, + AvgKeysPerPrefix: 1, + BaseWallTime: uint64(time.Now().UnixNano()), + PercentLogical: 0, + PercentEmptySuffix: 0, + PercentLockSuffix: 0, + }, 0) + for i := 0; i < countTables; i++ { + fileAlloc[i] = manifest.TableMetadata{ + FileNum: base.FileNum(i), + } + fileAlloc[i].ExtendPointKeyBounds(cockroachkvs.Compare, + base.MakeInternalKey(keys[i*2], base.SeqNum(i), base.InternalKeyKindSet), + base.MakeInternalKey(keys[i*2+1], base.SeqNum(i), base.InternalKeyKindSet)) + fileAlloc[i].InitPhysicalBacking() + files[i] = &fileAlloc[i] + } + + lm := manifest.MakeLevelMetadata(cockroachkvs.Compare, 0, files) + iter := lm.Iter() + b.ResetTimer() + for i := 0; i < b.N; i++ { + _ = iter.SeekGE(cockroachkvs.Compare, keys[i%len(keys)]) + } +} diff --git a/internal/manifest/btree.go b/internal/manifest/btree.go index d6c452a7ea..8a630b17ad 100644 --- a/internal/manifest/btree.go +++ b/internal/manifest/btree.go @@ -1095,31 +1095,101 @@ func (i *iterator) ascend() { i.pos = f.pos } -// seek repositions the iterator over the first file for which fn returns -// true, mirroring the semantics of the standard library's sort.Search -// function. Like sort.Search, seek requires the iterator's B-Tree to be -// ordered such that fn returns false for some (possibly empty) prefix of the -// tree's files, and then true for the (possibly empty) remainder. -func (i *iterator) seek(fn func(*TableMetadata) bool) { +// seekSeqNumL0 seeks an iterator over L0 files (ordered by sequence number) to +// the provided table metadata if it exists. +func (i *iterator) seekSeqNumL0(m *TableMetadata) { i.reset() if i.r == nil { return } - for { // Logic copied from sort.Search. + // + // INVARIANT A: items[j-1].cmpSeqNum(m) < 0 + // INVARIANT B: items[k].cmpSeqNum(m) >= 0 j, k := 0, int(i.n.count) for j < k { h := int(uint(j+k) >> 1) // avoid overflow when computing h + // j ≤ h < k + if i.n.items[h].cmpSeqNum(m) < 0 { + j = h + 1 // preserves INVARIANT A + } else { + k = h // preserves INVARIANT B + } + } + i.pos = int16(j) + if i.n.leaf { + if i.pos == i.n.count { + i.next() + } + return + } + i.descend(i.n, i.pos) + } +} +// seekLargest repositions the iterator over the first table whose largest key +// is an upper bound for the given user key. seekLargest requires the iterator's +// B-Tree to be ordered by user keys (i.e, L1+ or a single sublevel of L0). +func (i *iterator) seekLargest(cmp base.Compare, userKey []byte) { + i.reset() + if i.r == nil { + return + } + for { + // Logic copied from sort.Search. + // + // INVARIANT A: items[j-1].Largest().IsUpperBoundFor(cmp, userKey) == false + // INVARIANT B: items[k].Largest().IsUpperBoundFor(cmp, userKey) == true + j, k := 0, int(i.n.count) + for j < k { + h := int(uint(j+k) >> 1) // avoid overflow when computing h // j ≤ h < k - if !fn(i.n.items[h]) { - j = h + 1 // preserves f(j-1) == false + ik := &i.n.items[h].PointKeyBounds + if i.n.items[h].boundTypeLargest == boundTypeRangeKey { + ik = i.n.items[h].RangeKeyBounds + } + c := cmp(userKey, ik.LargestUserKey()) + if c > 0 || (c == 0 && ik.largestTrailer.IsExclusiveSentinel()) { + j = h + 1 // preserves INVARIANT A } else { - k = h // preserves f(k) == true + k = h // preserves INVARIANT B } } + i.pos = int16(j) + if i.n.leaf { + if i.pos == i.n.count { + i.next() + } + return + } + i.descend(i.n, i.pos) + } +} +// seekSmallest repositions the iterator over the first table whose smallest key +// is a lower bound for the given user key. seekSmallest requires the iterator's +// B-Tree to be ordered by user keys (i.e, L1+ or a single sublevel of L0). +func (i *iterator) seekSmallest(cmp base.Compare, userKey []byte) { + i.reset() + if i.r == nil { + return + } + for { + // Logic copied from sort.Search. + // + // INVARIANT A: items[j-1].Smallest().UserKey < userKey + // INVARIANT B: items[k].Smallest().UserKey >= 0 + j, k := 0, int(i.n.count) + for j < k { + h := int(uint(j+k) >> 1) // avoid overflow when computing h + // j ≤ h < k + if cmp(i.n.items[h].Smallest().UserKey, userKey) < 0 { + j = h + 1 // preserves INVARIANT A + } else { + k = h // preserves INVARIANT B + } + } i.pos = int16(j) if i.n.leaf { if i.pos == i.n.count { diff --git a/internal/manifest/level_metadata.go b/internal/manifest/level_metadata.go index 9947683373..6168a60dc8 100644 --- a/internal/manifest/level_metadata.go +++ b/internal/manifest/level_metadata.go @@ -146,9 +146,8 @@ func (lm *LevelMetadata) Find(cmp base.Compare, m *TableMetadata) LevelSlice { if lm.level == 0 { // We only need to look at the portion of files that are "equal" to m with // respect to the L0 ordering. - f := iter.seek(func(f *TableMetadata) bool { - return f.cmpSeqNum(m) >= 0 - }) + iter.iter.seekSeqNumL0(m) + f := iter.constrainToIteratorBounds() for ; f != nil && f.cmpSeqNum(m) == 0; f = iter.Next() { if f == m { return iter.Take().slice @@ -157,7 +156,7 @@ func (lm *LevelMetadata) Find(cmp base.Compare, m *TableMetadata) LevelSlice { } else { // For levels other than L0, UserKeyBounds in the level are non-overlapping // so we only need to check one file. - if f := iter.SeekGE(cmp, m.UserKeyBounds().Start); f == m { + if f := iter.SeekGE(cmp, m.Smallest().UserKey); f == m { return iter.Take().slice } } @@ -572,9 +571,8 @@ func (i *LevelIterator) SeekGE(cmp Compare, userKey []byte) *TableMetadata { return nil } i.assertNotL0Cmp() - m := i.seek(func(m *TableMetadata) bool { - return m.Largest().IsUpperBoundFor(cmp, userKey) - }) + i.iter.seekLargest(cmp, userKey) + m := i.constrainToIteratorBounds() if i.filter != KeyTypePointAndRange && m != nil { b, ok := m.LargestBound(i.filter) if !ok || !b.IsUpperBoundFor(cmp, userKey) { @@ -598,9 +596,8 @@ func (i *LevelIterator) SeekLT(cmp Compare, userKey []byte) *TableMetadata { return nil } i.assertNotL0Cmp() - i.seek(func(m *TableMetadata) bool { - return cmp(m.Smallest().UserKey, userKey) >= 0 - }) + i.iter.seekSmallest(cmp, userKey) + _ = i.constrainToIteratorBounds() m := i.Prev() // Although i.Prev() guarantees that the current file contains keys of the // relevant type, it doesn't guarantee that the keys of the relevant type @@ -682,16 +679,12 @@ func (i *LevelIterator) skipFilteredBackward(meta *TableMetadata) *TableMetadata return meta } -// seek repositions the iterator over the first file for which fn returns true, -// mirroring the semantics of the standard library's sort.Search function: fn -// returns false for some (possibly empty) prefix of the tree's files, and then -// true for the (possibly empty) remainder. -func (i *LevelIterator) seek(fn func(*TableMetadata) bool) *TableMetadata { - i.iter.seek(fn) - - // i.iter.seek seeked in the unbounded underlying B-Tree. If the iterator - // has start or end bounds, we may have exceeded them. Reset to the bounds - // if necessary. +// constrainToIteratorBounds adjusts the iterator position to ensure it's +// positioned within the iterator's bounds. +func (i *LevelIterator) constrainToIteratorBounds() *TableMetadata { + // i.iter.{seekLargest,seekSmallest,seekSeqNumL0} all seek in the unbounded + // underlying B-Tree. If the iterator has start or end bounds, we may have + // exceeded them. Reset to the bounds if necessary. // // NB: The LevelIterator and LevelSlice semantics require that a bounded // LevelIterator/LevelSlice containing files x0, x1, ..., xn behave