Skip to content

Commit 200f9cf

Browse files
committed
overlap: rework ingest LSM overlap check code
Instead of using level iterators, we directly check overlap with files individually. This requires less machinery and there is no chance of opening files that are outside of the bounds of interest. We put the new code in a new `internal/overlap` package and `overlapChecker` is now a very small wrapper which interfaces with `overlap.Checker`.
1 parent 7cdb5ee commit 200f9cf

File tree

18 files changed

+811
-242
lines changed

18 files changed

+811
-242
lines changed

db.go

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3007,7 +3007,13 @@ func (d *DB) checkVirtualBounds(m *fileMetadata) {
30073007

30083008
// DebugString returns a debugging string describing the LSM.
30093009
func (d *DB) DebugString() string {
3010+
return d.DebugCurrentVersion().DebugString()
3011+
}
3012+
3013+
// DebugCurrentVersion returns the current LSM tree metadata. Should only be
3014+
// used for testing/debugging.
3015+
func (d *DB) DebugCurrentVersion() *manifest.Version {
30103016
d.mu.Lock()
30113017
defer d.mu.Unlock()
3012-
return d.mu.versions.currentVersion().DebugString()
3018+
return d.mu.versions.currentVersion()
30133019
}

flushable.go

Lines changed: 30 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -394,3 +394,33 @@ func determineOverlapAllIters(
394394
}
395395
return false, nil
396396
}
397+
398+
func determineOverlapPointIterator(
399+
cmp base.Compare, bounds base.UserKeyBounds, iter internalIterator,
400+
) (bool, error) {
401+
kv := iter.SeekGE(bounds.Start, base.SeekGEFlagsNone)
402+
if kv == nil {
403+
return false, iter.Error()
404+
}
405+
return bounds.End.IsUpperBoundForInternalKey(cmp, kv.K), nil
406+
}
407+
408+
func determineOverlapKeyspanIterator(
409+
cmp base.Compare, bounds base.UserKeyBounds, iter keyspan.FragmentIterator,
410+
) (bool, error) {
411+
// NB: The spans surfaced by the fragment iterator are non-overlapping.
412+
span, err := iter.SeekGE(bounds.Start)
413+
if err != nil {
414+
return false, err
415+
}
416+
for ; span != nil; span, err = iter.Next() {
417+
if !bounds.End.IsUpperBoundFor(cmp, span.Start) {
418+
// The span starts after our bounds.
419+
return false, nil
420+
}
421+
if !span.Empty() {
422+
return true, nil
423+
}
424+
}
425+
return false, err
426+
}

ingest.go

Lines changed: 13 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ import (
1515
"github.com/cockroachdb/pebble/internal/base"
1616
"github.com/cockroachdb/pebble/internal/invariants"
1717
"github.com/cockroachdb/pebble/internal/manifest"
18+
"github.com/cockroachdb/pebble/internal/overlap"
1819
"github.com/cockroachdb/pebble/internal/private"
1920
"github.com/cockroachdb/pebble/objstorage"
2021
"github.com/cockroachdb/pebble/objstorage/remote"
@@ -832,7 +833,7 @@ func ingestUpdateSeqNum(
832833
func ingestTargetLevel(
833834
ctx context.Context,
834835
cmp base.Compare,
835-
lsmOverlap lsmOverlap,
836+
lsmOverlap overlap.WithLSM,
836837
baseLevel int,
837838
compactions map[*compaction]struct{},
838839
meta *fileMetadata,
@@ -904,32 +905,32 @@ func ingestTargetLevel(
904905
// existing point that falls within the ingested table bounds as being "data
905906
// overlap".
906907

907-
if lsmOverlap[0].result == dataOverlap {
908+
if lsmOverlap[0].Result == overlap.Data {
908909
return 0, nil, nil
909910
}
910911
targetLevel = 0
911912
splitFile = nil
912913
for level := baseLevel; level < numLevels; level++ {
913914
var candidateSplitFile *fileMetadata
914-
switch lsmOverlap[level].result {
915-
case dataOverlap:
915+
switch lsmOverlap[level].Result {
916+
case overlap.Data:
916917
// We cannot ingest into or under this level; return the best target level
917918
// so far.
918919
return targetLevel, splitFile, nil
919920

920-
case noDataOverlap:
921-
if !suggestSplit || lsmOverlap[level].splitFile == nil {
921+
case overlap.OnlyBoundary:
922+
if !suggestSplit || lsmOverlap[level].SplitFile == nil {
922923
// We can ingest under this level, but not into this level.
923924
continue
924925
}
925926
// We can ingest into this level if we split this file.
926-
candidateSplitFile = lsmOverlap[level].splitFile
927+
candidateSplitFile = lsmOverlap[level].SplitFile
927928

928-
case noBoundaryOverlap:
929+
case overlap.None:
929930
// We can ingest into this level.
930931

931932
default:
932-
return 0, nil, base.AssertionFailedf("unexpected lsmOverlap result: %v", lsmOverlap[level].result)
933+
return 0, nil, base.AssertionFailedf("unexpected WithLevel.Result: %v", lsmOverlap[level].Result)
933934
}
934935

935936
// Check boundary overlap with any ongoing compactions. We consider an
@@ -2192,15 +2193,15 @@ func (d *DB) ingestApply(
21922193
// We check overlap against the LSM without holding DB.mu. Note that we
21932194
// are still holding the log lock, so the version cannot change.
21942195
// TODO(radu): perform this check optimistically outside of the log lock.
2195-
var overlap lsmOverlap
2196-
overlap, err = func() (lsmOverlap, error) {
2196+
var lsmOverlap overlap.WithLSM
2197+
lsmOverlap, err = func() (overlap.WithLSM, error) {
21972198
d.mu.Unlock()
21982199
defer d.mu.Lock()
21992200
return overlapChecker.DetermineLSMOverlap(ctx, m.UserKeyBounds())
22002201
}()
22012202
if err == nil {
22022203
f.Level, splitFile, err = ingestTargetLevel(
2203-
ctx, d.cmp, overlap, baseLevel, d.mu.compact.inProgress, m, shouldIngestSplit,
2204+
ctx, d.cmp, lsmOverlap, baseLevel, d.mu.compact.inProgress, m, shouldIngestSplit,
22042205
)
22052206
}
22062207
}

internal/base/key_bounds.go

Lines changed: 18 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -73,6 +73,23 @@ func (eb UserKeyBoundary) IsUpperBoundForInternalKey(cmp Compare, key InternalKe
7373
return c < 0 || (c == 0 && (eb.Kind == Inclusive || key.IsExclusiveSentinel()))
7474
}
7575

76+
// CompareUpperBounds compares two UserKeyBoundaries as upper bounds (e.g. when
77+
// they are used for UserKeyBounds.End).
78+
func (eb UserKeyBoundary) CompareUpperBounds(cmp Compare, other UserKeyBoundary) int {
79+
switch c := cmp(eb.Key, other.Key); {
80+
case c != 0:
81+
return c
82+
case eb.Kind == other.Kind:
83+
return 0
84+
case eb.Kind == Inclusive:
85+
// eb is inclusive, other is exclusive.
86+
return 1
87+
default:
88+
// eb is exclusive, other is inclusive.
89+
return -1
90+
}
91+
}
92+
7693
// UserKeyBounds is a user key interval with an inclusive start boundary and
7794
// with an end boundary that can be either inclusive or exclusive.
7895
type UserKeyBounds struct {
@@ -132,8 +149,7 @@ func (b *UserKeyBounds) ContainsBounds(cmp Compare, other *UserKeyBounds) bool {
132149
if cmp(b.Start, other.Start) > 0 {
133150
return false
134151
}
135-
c := cmp(other.End.Key, b.End.Key)
136-
return c < 0 || (c == 0 && (b.End.Kind == Inclusive || other.End.Kind == Exclusive))
152+
return other.End.CompareUpperBounds(cmp, b.End) <= 0
137153
}
138154

139155
// ContainsUserKey returns true if the user key is within the bounds.

internal/base/key_bounds_test.go

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,42 @@ import (
1010
"github.com/stretchr/testify/require"
1111
)
1212

13+
func TestUserKeyBoundary(t *testing.T) {
14+
cmp := DefaultComparer.Compare
15+
a := []byte("a")
16+
b := []byte("b")
17+
18+
aExclusive := UserKeyExclusive(a)
19+
require.False(t, aExclusive.IsUpperBoundFor(cmp, a))
20+
require.False(t, aExclusive.IsUpperBoundFor(cmp, b))
21+
22+
aInclusive := UserKeyInclusive(a)
23+
require.True(t, aInclusive.IsUpperBoundFor(cmp, a))
24+
require.False(t, aInclusive.IsUpperBoundFor(cmp, b))
25+
26+
bExclusive := UserKeyExclusive(b)
27+
require.True(t, bExclusive.IsUpperBoundFor(cmp, a))
28+
require.False(t, bExclusive.IsUpperBoundFor(cmp, b))
29+
30+
bInclusive := UserKeyInclusive(b)
31+
require.True(t, bInclusive.IsUpperBoundFor(cmp, a))
32+
require.True(t, bInclusive.IsUpperBoundFor(cmp, b))
33+
34+
ordered := []UserKeyBoundary{aExclusive, aInclusive, bExclusive, bInclusive}
35+
for i := range ordered {
36+
for j := range ordered {
37+
expected := 0
38+
if i < j {
39+
expected = -1
40+
} else if i > j {
41+
expected = 1
42+
}
43+
require.Equalf(t, expected, ordered[i].CompareUpperBounds(cmp, ordered[j]),
44+
"%v, %v", ordered[i], ordered[j])
45+
}
46+
}
47+
}
48+
1349
func TestUserKeyBounds(t *testing.T) {
1450
var ukb UserKeyBounds
1551
cmp := DefaultComparer.Compare

internal/base/test_utils.go

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@ package base
66

77
import (
88
"context"
9+
"fmt"
910
"io"
1011
"strconv"
1112
"strings"
@@ -249,3 +250,14 @@ func (f *FakeIter) SetBounds(lower, upper []byte) {
249250

250251
// SetContext is part of the InternalIterator interface.
251252
func (f *FakeIter) SetContext(_ context.Context) {}
253+
254+
// ParseUserKeyBounds parses UserKeyBounds from a string representation of the
255+
// form "[foo, bar]" or "[foo, bar)".
256+
func ParseUserKeyBounds(s string) UserKeyBounds {
257+
first, last, s := s[0], s[len(s)-1], s[1:len(s)-1]
258+
start, end, ok := strings.Cut(s, ", ")
259+
if !ok || first != '[' || (last != ']' && last != ')') {
260+
panic(fmt.Sprintf("invalid bounds %q", s))
261+
}
262+
return UserKeyBoundsEndExclusiveIf([]byte(start), []byte(end), last == ')')
263+
}

internal/manifest/l0_sublevels_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -289,7 +289,7 @@ func TestL0Sublevels(t *testing.T) {
289289
SortBySmallest(fileMetas[i], base.DefaultComparer.Compare)
290290
}
291291

292-
levelMetadata := makeLevelMetadata(base.DefaultComparer.Compare, 0, fileMetas[0])
292+
levelMetadata := MakeLevelMetadata(base.DefaultComparer.Compare, 0, fileMetas[0])
293293
if initialize {
294294
if addL0FilesOpt {
295295
SortBySeqNum(addedL0Files)
@@ -550,7 +550,7 @@ func TestAddL0FilesEquivalence(t *testing.T) {
550550
continue
551551
}
552552

553-
levelMetadata := makeLevelMetadata(testkeys.Comparer.Compare, 0, fileMetas)
553+
levelMetadata := MakeLevelMetadata(testkeys.Comparer.Compare, 0, fileMetas)
554554
var err error
555555

556556
if s2 == nil {

internal/manifest/level_metadata.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,8 @@ func (lm *LevelMetadata) release() (obsolete []*FileBacking) {
4141
return lm.tree.Release()
4242
}
4343

44-
func makeLevelMetadata(cmp Compare, level int, files []*FileMetadata) LevelMetadata {
44+
// MakeLevelMetadata creates a LevelMetadata with the given files.
45+
func MakeLevelMetadata(cmp Compare, level int, files []*FileMetadata) LevelMetadata {
4546
bcmp := btreeCmpSeqNum
4647
if level > 0 {
4748
bcmp = btreeCmpSmallestKey(cmp)

internal/manifest/version.go

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -301,6 +301,21 @@ func (m *FileMetadata) UserKeyBounds() base.UserKeyBounds {
301301
return base.UserKeyBoundsFromInternal(m.Smallest, m.Largest)
302302
}
303303

304+
// UserKeyBoundsByType returns the user key bounds for the given key types.
305+
// Note that the returned bounds are invalid when requesting KeyTypePoint but
306+
// HasPointKeys is false, or when requesting KeyTypeRange and HasRangeKeys is
307+
// false.
308+
func (m *FileMetadata) UserKeyBoundsByType(keyType KeyType) base.UserKeyBounds {
309+
switch keyType {
310+
case KeyTypePoint:
311+
return base.UserKeyBoundsFromInternal(m.SmallestPointKey, m.LargestPointKey)
312+
case KeyTypeRange:
313+
return base.UserKeyBoundsFromInternal(m.SmallestRangeKey, m.LargestRangeKey)
314+
default:
315+
return base.UserKeyBoundsFromInternal(m.Smallest, m.Largest)
316+
}
317+
}
318+
304319
// SyntheticSeqNum returns a SyntheticSeqNum which is set when SmallestSeqNum
305320
// equals LargestSeqNum.
306321
func (m *FileMetadata) SyntheticSeqNum() sstable.SyntheticSeqNum {

internal/manifest/version_edit.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -963,12 +963,12 @@ func (b *BulkVersionEdit) Apply(
963963

964964
for level := range v.Levels {
965965
if curr == nil || curr.Levels[level].tree.root == nil {
966-
v.Levels[level] = makeLevelMetadata(comparer.Compare, level, nil /* files */)
966+
v.Levels[level] = MakeLevelMetadata(comparer.Compare, level, nil /* files */)
967967
} else {
968968
v.Levels[level] = curr.Levels[level].clone()
969969
}
970970
if curr == nil || curr.RangeKeyLevels[level].tree.root == nil {
971-
v.RangeKeyLevels[level] = makeLevelMetadata(comparer.Compare, level, nil /* files */)
971+
v.RangeKeyLevels[level] = MakeLevelMetadata(comparer.Compare, level, nil /* files */)
972972
} else {
973973
v.RangeKeyLevels[level] = curr.RangeKeyLevels[level].clone()
974974
}

0 commit comments

Comments
 (0)