Skip to content

Commit 1643b8d

Browse files
committed
db: represent compaction bounds as UserKeyBounds
Use the base.UserKeyBounds type to represent the bounds of a compaction across the various compaction types: manifest.L0Compaction, compactionInfo, compaction and pickedCompaction. Add a UserKeyBounds.Union method that returns the minimum bounds that include all the keys of both bounds and use it in the calculation of compaction bounds.
1 parent bb173d2 commit 1643b8d

13 files changed

+132
-163
lines changed

compaction.go

Lines changed: 17 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,6 @@ import (
88
"bytes"
99
"context"
1010
"fmt"
11-
"iter"
1211
"math"
1312
"runtime/pprof"
1413
"slices"
@@ -263,8 +262,7 @@ type compaction struct {
263262
bytesWritten atomic.Int64
264263

265264
// The boundaries of the input data.
266-
smallest InternalKey
267-
largest InternalKey
265+
bounds base.UserKeyBounds
268266

269267
// A list of fragment iterators to close when the compaction finishes. Used by
270268
// input iteration to keep rangeDelIters open for the lifetime of the
@@ -364,10 +362,6 @@ func (c *compaction) makeInfo(jobID JobID) CompactionInfo {
364362
return info
365363
}
366364

367-
func (c *compaction) userKeyBounds() base.UserKeyBounds {
368-
return base.UserKeyBoundsFromInternal(c.smallest, c.largest)
369-
}
370-
371365
type getValueSeparation func(JobID, *compaction, sstable.TableFormat) compact.ValueSeparation
372366

373367
// newCompaction constructs a compaction from the provided picked compaction.
@@ -390,8 +384,7 @@ func newCompaction(
390384
comparer: opts.Comparer,
391385
formatKey: opts.Comparer.FormatKey,
392386
inputs: pc.inputs,
393-
smallest: pc.smallest,
394-
largest: pc.largest,
387+
bounds: pc.bounds,
395388
logger: opts.Logger,
396389
version: pc.version,
397390
beganAt: beganAt,
@@ -434,10 +427,10 @@ func newCompaction(
434427
// Compute the set of outputLevel+1 files that overlap this compaction (these
435428
// are the grandparent sstables).
436429
if c.outputLevel.level+1 < numLevels {
437-
c.grandparents = c.version.Overlaps(c.outputLevel.level+1, c.userKeyBounds())
430+
c.grandparents = c.version.Overlaps(c.outputLevel.level+1, c.bounds)
438431
}
439432
c.delElision, c.rangeKeyElision = compact.SetupTombstoneElision(
440-
c.cmp, c.version, pc.l0Organizer, c.outputLevel.level, base.UserKeyBoundsFromInternal(c.smallest, c.largest),
433+
c.cmp, c.version, pc.l0Organizer, c.outputLevel.level, c.bounds,
441434
)
442435
c.kind = pc.kind
443436

@@ -568,11 +561,10 @@ func newDeleteOnlyCompaction(
568561
c.version.Ref()
569562

570563
// Set c.smallest, c.largest.
571-
files := make([]iter.Seq[*manifest.TableMetadata], 0, len(inputs))
564+
cmp := opts.Comparer.Compare
572565
for _, in := range inputs {
573-
files = append(files, in.files.All())
566+
c.bounds = manifest.ExtendKeyRange(cmp, c.bounds, in.files.All())
574567
}
575-
c.smallest, c.largest = manifest.KeyRange(opts.Comparer.Compare, files...)
576568
return c
577569
}
578570

@@ -713,20 +705,15 @@ func newFlush(
713705

714706
c.l0Limits = l0Organizer.FlushSplitKeys()
715707

716-
smallestSet, largestSet := false, false
717708
updatePointBounds := func(iter internalIterator) {
718709
if kv := iter.First(); kv != nil {
719-
if !smallestSet ||
720-
base.InternalCompare(c.cmp, c.smallest, kv.K) > 0 {
721-
smallestSet = true
722-
c.smallest = kv.K.Clone()
710+
if c.bounds.Start == nil || c.cmp(c.bounds.Start, kv.K.UserKey) > 0 {
711+
c.bounds.Start = slices.Clone(kv.K.UserKey)
723712
}
724713
}
725714
if kv := iter.Last(); kv != nil {
726-
if !largestSet ||
727-
base.InternalCompare(c.cmp, c.largest, kv.K) < 0 {
728-
largestSet = true
729-
c.largest = kv.K.Clone()
715+
if c.bounds.End.Key == nil || !c.bounds.End.IsUpperBoundForInternalKey(c.cmp, kv.K) {
716+
c.bounds.End = base.UserKeyExclusiveIf(slices.Clone(kv.K.UserKey), kv.K.IsExclusiveSentinel())
730717
}
731718
}
732719
}
@@ -738,20 +725,12 @@ func newFlush(
738725
if s, err := iter.First(); err != nil {
739726
return err
740727
} else if s != nil {
741-
if key := s.SmallestKey(); !smallestSet ||
742-
base.InternalCompare(c.cmp, c.smallest, key) > 0 {
743-
smallestSet = true
744-
c.smallest = key.Clone()
745-
}
728+
c.bounds = c.bounds.Union(c.cmp, s.Bounds().Clone())
746729
}
747730
if s, err := iter.Last(); err != nil {
748731
return err
749732
} else if s != nil {
750-
if key := s.LargestKey(); !largestSet ||
751-
base.InternalCompare(c.cmp, c.largest, key) < 0 {
752-
largestSet = true
753-
c.largest = key.Clone()
754-
}
733+
c.bounds = c.bounds.Union(c.cmp, s.Bounds().Clone())
755734
}
756735
return nil
757736
}
@@ -776,7 +755,7 @@ func newFlush(
776755
if opts.FlushSplitBytes > 0 {
777756
c.maxOutputFileSize = uint64(opts.Levels[0].TargetFileSize)
778757
c.maxOverlapBytes = maxGrandparentOverlapBytes(opts, 0)
779-
c.grandparents = c.version.Overlaps(baseLevel, c.userKeyBounds())
758+
c.grandparents = c.version.Overlaps(baseLevel, c.bounds)
780759
adjustGrandparentOverlapBytesForFlush(c, flushingBytes)
781760
}
782761

@@ -1665,6 +1644,7 @@ func (d *DB) flush1() (bytesFlushed uint64, err error) {
16651644
// c.kind == compactionKindIngestedFlushable && we could have deleted files due
16661645
// to ingest-time splits or excises.
16671646
ingestFlushable := c.flushing[0].flushable.(*ingestedFlushable)
1647+
exciseBounds := ingestFlushable.exciseSpan.UserKeyBounds()
16681648
for c2 := range d.mu.compact.inProgress {
16691649
// Check if this compaction overlaps with the excise span. Note that just
16701650
// checking if the inputs individually overlap with the excise span
@@ -1673,7 +1653,7 @@ func (d *DB) flush1() (bytesFlushed uint64, err error) {
16731653
// doing a [c,d) excise at the same time as this compaction, we will have
16741654
// to error out the whole compaction as we can't guarantee it hasn't/won't
16751655
// write a file overlapping with the excise span.
1676-
if ingestFlushable.exciseSpan.OverlapsInternalKeyRange(d.cmp, c2.smallest, c2.largest) {
1656+
if c2.bounds.Overlaps(d.cmp, &exciseBounds) {
16771657
c2.cancel.Store(true)
16781658
}
16791659
}
@@ -2452,7 +2432,7 @@ func (d *DB) compactionPprofLabels(c *compaction) pprof.LabelSet {
24522432
level = fmt.Sprintf("L%d", c.outputLevel.level)
24532433
}
24542434
if kc := d.opts.Experimental.UserKeyCategories; kc.Len() > 0 {
2455-
cat := kc.CategorizeKeyRange(c.smallest.UserKey, c.largest.UserKey)
2435+
cat := kc.CategorizeKeyRange(c.bounds.Start, c.bounds.End.Key)
24562436
return pprof.Labels("pebble", activity, "output-level", level, "key-type", cat)
24572437
}
24582438
return pprof.Labels("pebble", activity, "output-level", level)
@@ -3270,7 +3250,7 @@ func (d *DB) compactAndWrite(
32703250
iter := compact.NewIter(cfg, pointIter, rangeDelIter, rangeKeyIter)
32713251

32723252
runnerCfg := compact.RunnerConfig{
3273-
CompactionBounds: base.UserKeyBoundsFromInternal(c.smallest, c.largest),
3253+
CompactionBounds: c.bounds,
32743254
L0SplitKeys: c.l0Limits,
32753255
Grandparents: c.grandparents,
32763256
MaxGrandparentOverlapBytes: c.maxOverlapBytes,

0 commit comments

Comments
 (0)