Skip to content

Commit

Permalink
manifest: store comparer in Version
Browse files Browse the repository at this point in the history
The `Version` and `VersionEdit` code has a ton of `Compare` and
`FormatKey` arguments passed around. We simplify all this code by
storing a `*base.Comparer` reference in `Version` and `versionSet`.
  • Loading branch information
RaduBerinde committed Feb 26, 2024
1 parent 403ee52 commit aed0b13
Show file tree
Hide file tree
Showing 25 changed files with 167 additions and 178 deletions.
10 changes: 5 additions & 5 deletions compaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -733,7 +733,7 @@ func newCompaction(
// Compute the set of outputLevel+1 files that overlap this compaction (these
// are the grandparent sstables).
if c.outputLevel.level+1 < numLevels {
c.grandparents = c.version.Overlaps(c.outputLevel.level+1, c.cmp,
c.grandparents = c.version.Overlaps(c.outputLevel.level+1,
c.smallest.UserKey, c.largest.UserKey, c.largest.IsExclusiveSentinel())
}
c.setupInuseKeyRanges()
Expand Down Expand Up @@ -972,7 +972,7 @@ func newFlush(
if opts.FlushSplitBytes > 0 {
c.maxOutputFileSize = uint64(opts.Level(0).TargetFileSize)
c.maxOverlapBytes = maxGrandparentOverlapBytes(opts, 0)
c.grandparents = c.version.Overlaps(baseLevel, c.cmp, c.smallest.UserKey,
c.grandparents = c.version.Overlaps(baseLevel, c.smallest.UserKey,
c.largest.UserKey, c.largest.IsExclusiveSentinel())
adjustGrandparentOverlapBytesForFlush(c, flushingBytes)
}
Expand Down Expand Up @@ -1002,7 +1002,7 @@ func (c *compaction) setupInuseKeyRanges() {
// calculateInuseKeyRanges will return a series of sorted spans. Overlapping
// or abutting spans have already been merged.
c.inuseKeyRanges = c.version.CalculateInuseKeyRanges(
c.cmp, level, numLevels-1, c.smallest.UserKey, c.largest.UserKey,
level, numLevels-1, c.smallest.UserKey, c.largest.UserKey,
)
// Check if there's a single in-use span that encompasses the entire key
// range of the compaction. This is an optimization to avoid key comparisons
Expand Down Expand Up @@ -2176,7 +2176,7 @@ func (d *DB) maybeScheduleDownloadCompaction(env compactionEnv, maxConcurrentCom
var err error
var level int
for i := range v.Levels {
overlaps := v.Overlaps(i, d.cmp, download.start, download.end, true /* exclusiveEnd */)
overlaps := v.Overlaps(i, download.start, download.end, true /* exclusiveEnd */)
iter := overlaps.Iter()
provider := d.objProvider
for f := iter.First(); f != nil; f = iter.Next() {
Expand Down Expand Up @@ -2565,7 +2565,7 @@ func checkDeleteCompactionHints(
// The hint h will be resolved and dropped, regardless of whether
// there are any tables that can be deleted.
for l := h.tombstoneLevel + 1; l < numLevels; l++ {
overlaps := v.Overlaps(l, cmp, h.start, h.end, true /* exclusiveEnd */)
overlaps := v.Overlaps(l, h.start, h.end, true /* exclusiveEnd */)
iter := overlaps.Iter()
for m := iter.First(); m != nil; m = iter.Next() {
if m.IsCompacting() || !h.canDelete(cmp, m, snapshots) || files[m] {
Expand Down
15 changes: 7 additions & 8 deletions compaction_picker.go
Original file line number Diff line number Diff line change
Expand Up @@ -420,7 +420,7 @@ func (pc *pickedCompaction) setupInputs(
// sstables. No need to do this for intra-L0 compactions; outputLevel.files is
// left empty for those.
if startLevel.level != pc.outputLevel.level {
pc.outputLevel.files = pc.version.Overlaps(pc.outputLevel.level, pc.cmp, pc.smallest.UserKey,
pc.outputLevel.files = pc.version.Overlaps(pc.outputLevel.level, pc.smallest.UserKey,
pc.largest.UserKey, pc.largest.IsExclusiveSentinel())
if anyTablesCompacting(pc.outputLevel.files) {
return false
Expand Down Expand Up @@ -514,7 +514,7 @@ func (pc *pickedCompaction) grow(
if pc.outputLevel.files.Empty() {
return false
}
grow0 := pc.version.Overlaps(startLevel.level, pc.cmp, sm.UserKey,
grow0 := pc.version.Overlaps(startLevel.level, sm.UserKey,
la.UserKey, la.IsExclusiveSentinel())
if anyTablesCompacting(grow0) {
return false
Expand All @@ -528,7 +528,7 @@ func (pc *pickedCompaction) grow(
// We need to include the outputLevel iter because without it, in a multiLevel scenario,
// sm1 and la1 could shift the output level keyspace when pc.outputLevel.files is set to grow1.
sm1, la1 := manifest.KeyRange(pc.cmp, grow0.Iter(), pc.outputLevel.files.Iter())
grow1 := pc.version.Overlaps(pc.outputLevel.level, pc.cmp, sm1.UserKey,
grow1 := pc.version.Overlaps(pc.outputLevel.level, sm1.UserKey,
la1.UserKey, la1.IsExclusiveSentinel())
if anyTablesCompacting(grow1) {
return false
Expand Down Expand Up @@ -1560,7 +1560,7 @@ func pickAutoLPositive(
if pc.startLevel.level == 0 {
cmp := opts.Comparer.Compare
smallest, largest := manifest.KeyRange(cmp, pc.startLevel.files.Iter())
pc.startLevel.files = vers.Overlaps(0, cmp, smallest.UserKey,
pc.startLevel.files = vers.Overlaps(0, smallest.UserKey,
largest.UserKey, largest.IsExclusiveSentinel())
if pc.startLevel.files.Empty() {
panic("pebble: empty compaction")
Expand Down Expand Up @@ -1768,7 +1768,7 @@ func pickManualCompaction(
}
pc = newPickedCompaction(opts, vers, manual.level, defaultOutputLevel(manual.level, baseLevel), baseLevel)
manual.outputLevel = pc.outputLevel.level
pc.startLevel.files = vers.Overlaps(manual.level, opts.Comparer.Compare, manual.start, manual.end, false)
pc.startLevel.files = vers.Overlaps(manual.level, manual.start, manual.end, false)
if pc.startLevel.files.Empty() {
// Nothing to do
return nil, false
Expand Down Expand Up @@ -1854,8 +1854,7 @@ func (p *compactionPickerByScore) pickReadTriggeredCompaction(
func pickReadTriggeredCompactionHelper(
p *compactionPickerByScore, rc *readCompaction, env compactionEnv,
) (pc *pickedCompaction) {
cmp := p.opts.Comparer.Compare
overlapSlice := p.vers.Overlaps(rc.level, cmp, rc.start, rc.end, false /* exclusiveEnd */)
overlapSlice := p.vers.Overlaps(rc.level, rc.start, rc.end, false /* exclusiveEnd */)
if overlapSlice.Empty() {
// If there is no overlap, then the file with the key range
// must have been compacted away. So, we don't proceed to
Expand Down Expand Up @@ -1888,7 +1887,7 @@ func pickReadTriggeredCompactionHelper(

// Prevent read compactions which are too wide.
outputOverlaps := pc.version.Overlaps(
pc.outputLevel.level, pc.cmp, pc.smallest.UserKey,
pc.outputLevel.level, pc.smallest.UserKey,
pc.largest.UserKey, pc.largest.IsExclusiveSentinel())
if outputOverlaps.SizeSum() > pc.maxReadCompactionBytes {
return nil
Expand Down
22 changes: 9 additions & 13 deletions compaction_picker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -501,9 +501,8 @@ func TestCompactionPickerL0(t *testing.T) {
version := newVersion(opts, fileMetas)
version.L0Sublevels.InitCompactingFileInfo(inProgressL0Compactions(inProgressCompactions))
vs := &versionSet{
opts: opts,
cmp: DefaultComparer.Compare,
cmpName: DefaultComparer.Name,
opts: opts,
cmp: DefaultComparer,
}
vs.versions.Init(nil)
vs.append(version)
Expand Down Expand Up @@ -725,9 +724,8 @@ func TestCompactionPickerConcurrency(t *testing.T) {
version := newVersion(opts, fileMetas)
version.L0Sublevels.InitCompactingFileInfo(inProgressL0Compactions(inProgressCompactions))
vs := &versionSet{
opts: opts,
cmp: DefaultComparer.Compare,
cmpName: DefaultComparer.Name,
opts: opts,
cmp: DefaultComparer,
}
vs.versions.Init(nil)
vs.append(version)
Expand Down Expand Up @@ -843,9 +841,8 @@ func TestCompactionPickerPickReadTriggered(t *testing.T) {

vers = newVersion(opts, fileMetas)
vs := &versionSet{
opts: opts,
cmp: DefaultComparer.Compare,
cmpName: DefaultComparer.Name,
opts: opts,
cmp: DefaultComparer,
}
vs.versions.Init(nil)
vs.append(vers)
Expand Down Expand Up @@ -1049,7 +1046,7 @@ func TestPickedCompactionSetupInputs(t *testing.T) {
pc.outputLevel.level = pc.startLevel.level + 1
}
pc.version = newVersion(opts, files)
pc.startLevel.files = pc.version.Overlaps(pc.startLevel.level, pc.cmp,
pc.startLevel.files = pc.version.Overlaps(pc.startLevel.level,
[]byte(args[0].String()), []byte(args[1].String()), false /* exclusiveEnd */)

var isCompacting bool
Expand Down Expand Up @@ -1263,9 +1260,8 @@ func TestCompactionOutputFileSize(t *testing.T) {

vers = newVersion(opts, fileMetas)
vs := &versionSet{
opts: opts,
cmp: DefaultComparer.Compare,
cmpName: DefaultComparer.Name,
opts: opts,
cmp: DefaultComparer,
}
vs.versions.Init(nil)
vs.append(vers)
Expand Down
37 changes: 17 additions & 20 deletions compaction_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,7 @@ import (

func newVersion(opts *Options, files [numLevels][]*fileMetadata) *version {
return manifest.NewVersion(
opts.Comparer.Compare,
opts.Comparer.FormatKey,
opts.Comparer,
opts.FlushSplitBytes,
files)
}
Expand Down Expand Up @@ -506,9 +505,8 @@ func TestPickCompaction(t *testing.T) {

for _, tc := range testCases {
vs := &versionSet{
opts: opts,
cmp: DefaultComparer.Compare,
cmpName: DefaultComparer.Name,
opts: opts,
cmp: DefaultComparer,
}
vs.versions.Init(nil)
vs.append(tc.version)
Expand Down Expand Up @@ -566,7 +564,7 @@ func TestElideTombstone(t *testing.T) {
return err.Error()
}
if td.HasArg("verbose") {
return d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
return d.mu.versions.currentVersion().DebugString()
}
return d.mu.versions.currentVersion().String()
case "elide":
Expand Down Expand Up @@ -1213,8 +1211,8 @@ func TestManualCompaction(t *testing.T) {
ongoingCompaction.outputLevel = &ongoingCompaction.inputs[1]
// Mark files as compacting.
curr := d.mu.versions.currentVersion()
ongoingCompaction.startLevel.files = curr.Overlaps(startLevel, d.cmp, start, end, false)
ongoingCompaction.outputLevel.files = curr.Overlaps(outputLevel, d.cmp, start, end, false)
ongoingCompaction.startLevel.files = curr.Overlaps(startLevel, start, end, false)
ongoingCompaction.outputLevel.files = curr.Overlaps(outputLevel, start, end, false)
for _, cl := range ongoingCompaction.inputs {
iter := cl.files.Iter()
for f := iter.First(); f != nil; f = iter.Next() {
Expand Down Expand Up @@ -1268,7 +1266,7 @@ func TestManualCompaction(t *testing.T) {
d.mu.Lock()
s := d.mu.versions.currentVersion().String()
if verbose {
s = d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
s = d.mu.versions.currentVersion().DebugString()
}
d.mu.Unlock()
if td.HasArg("hide-file-num") {
Expand Down Expand Up @@ -1303,7 +1301,7 @@ func TestManualCompaction(t *testing.T) {

s := d.mu.versions.currentVersion().String()
if verbose {
s = d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
s = d.mu.versions.currentVersion().DebugString()
}
return s

Expand All @@ -1317,7 +1315,7 @@ func TestManualCompaction(t *testing.T) {
d.mu.Lock()
s := d.mu.versions.currentVersion().String()
if verbose {
s = d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
s = d.mu.versions.currentVersion().DebugString()
}
d.mu.Unlock()
return s
Expand All @@ -1329,7 +1327,7 @@ func TestManualCompaction(t *testing.T) {
d.mu.Lock()
s := d.mu.versions.currentVersion().String()
if verbose {
s = d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
s = d.mu.versions.currentVersion().DebugString()
}
d.mu.Unlock()
return s
Expand Down Expand Up @@ -1677,7 +1675,7 @@ func TestCompactionFindL0Limit(t *testing.T) {
}
}

vers = manifest.NewVersion(DefaultComparer.Compare, base.DefaultFormatter, flushSplitBytes, fileMetas)
vers = manifest.NewVersion(DefaultComparer, flushSplitBytes, fileMetas)
flushSplitKeys := vers.L0Sublevels.FlushSplitKeys()

var buf strings.Builder
Expand Down Expand Up @@ -1726,7 +1724,7 @@ func TestCompactionFindL0Limit(t *testing.T) {

func TestCompactionOutputLevel(t *testing.T) {
opts := (*Options)(nil).EnsureDefaults()
version := &version{}
version := manifest.TestingNewVersion(opts.Comparer)

datadriven.RunTest(t, "testdata/compaction_output_level",
func(t *testing.T, d *datadriven.TestData) (res string) {
Expand Down Expand Up @@ -2493,14 +2491,14 @@ func TestCompactionInuseKeyRangesRandomized(t *testing.T) {
})
}
v := newVersion(opts, files)
t.Log(v.DebugString(opts.Comparer.FormatKey))
t.Log(v.DebugString())
for i := 0; i < 1000; i++ {
l := rng.Intn(numLevels)
s := rng.Intn(endKeyspace)
maxWidth := rng.Intn(endKeyspace-s) + 1
e := rng.Intn(maxWidth) + s
sKey, eKey := makeUserKey(s), makeUserKey(e)
keyRanges := v.CalculateInuseKeyRanges(opts.Comparer.Compare, l, numLevels-1, sKey, eKey)
keyRanges := v.CalculateInuseKeyRanges(l, numLevels-1, sKey, eKey)

for level := l; level < numLevels; level++ {
for _, f := range files[level] {
Expand Down Expand Up @@ -3172,7 +3170,6 @@ func TestCompactionInvalidBounds(t *testing.T) {

func Test_calculateInuseKeyRanges(t *testing.T) {
opts := (*Options)(nil).EnsureDefaults()
cmp := base.DefaultComparer.Compare
newFileMeta := func(fileNum FileNum, size uint64, smallest, largest base.InternalKey) *fileMetadata {
m := (&fileMetadata{
FileNum: fileNum,
Expand Down Expand Up @@ -3401,7 +3398,7 @@ func Test_calculateInuseKeyRanges(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
if got := tt.v.CalculateInuseKeyRanges(cmp, tt.level, tt.depth, tt.smallest, tt.largest); !reflect.DeepEqual(got, tt.want) {
if got := tt.v.CalculateInuseKeyRanges(tt.level, tt.depth, tt.smallest, tt.largest); !reflect.DeepEqual(got, tt.want) {
t.Errorf("CalculateInuseKeyRanges() = %v, want %v", got, tt.want)
}
})
Expand Down Expand Up @@ -3468,7 +3465,7 @@ func TestMarkedForCompaction(t *testing.T) {
t = t.Add(time.Second)
return t
}
s := d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
s := d.mu.versions.currentVersion().DebugString()
return s

case "mark-for-compaction":
Expand Down Expand Up @@ -3500,7 +3497,7 @@ func TestMarkedForCompaction(t *testing.T) {
d.mu.compact.cond.Wait()
}

fmt.Fprintln(&buf, d.mu.versions.currentVersion().DebugString(base.DefaultFormatter))
fmt.Fprintln(&buf, d.mu.versions.currentVersion().DebugString())
s := strings.TrimSpace(buf.String())
buf.Reset()
opts.DisableAutomaticCompactions = true
Expand Down
2 changes: 1 addition & 1 deletion data_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1337,7 +1337,7 @@ func runLSMCmd(td *datadriven.TestData, d *DB) string {
d.mu.Lock()
defer d.mu.Unlock()
if td.HasArg("verbose") {
return d.mu.versions.currentVersion().DebugString(d.opts.Comparer.FormatKey)
return d.mu.versions.currentVersion().DebugString()
}
return d.mu.versions.currentVersion().String()
}
Expand Down
12 changes: 6 additions & 6 deletions db.go
Original file line number Diff line number Diff line change
Expand Up @@ -1742,7 +1742,7 @@ func (d *DB) Compact(start, end []byte, parallelize bool) error {
maxLevelWithFiles := 1
cur := d.mu.versions.currentVersion()
for level := 0; level < numLevels; level++ {
overlaps := cur.Overlaps(level, d.cmp, start, end, false)
overlaps := cur.Overlaps(level, start, end, false)
if !overlaps.Empty() {
maxLevelWithFiles = level + 1
}
Expand Down Expand Up @@ -1819,7 +1819,7 @@ func (d *DB) Compact(start, end []byte, parallelize bool) error {
func (d *DB) manualCompact(start, end []byte, level int, parallelize bool) error {
d.mu.Lock()
curr := d.mu.versions.currentVersion()
files := curr.Overlaps(level, d.cmp, start, end, false)
files := curr.Overlaps(level, start, end, false)
if files.Empty() {
d.mu.Unlock()
return nil
Expand Down Expand Up @@ -1866,7 +1866,7 @@ func (d *DB) splitManualCompaction(
if level == 0 {
endLevel = baseLevel
}
keyRanges := curr.CalculateInuseKeyRanges(d.cmp, level, endLevel, start, end)
keyRanges := curr.CalculateInuseKeyRanges(level, endLevel, start, end)
for _, keyRange := range keyRanges {
splitCompactions = append(splitCompactions, &manualCompaction{
level: level,
Expand Down Expand Up @@ -1931,7 +1931,7 @@ func (d *DB) downloadSpan(ctx context.Context, span DownloadSpan) error {
if vers.Levels[i].Empty() {
continue
}
overlap := vers.Overlaps(i, d.cmp, span.StartKey, span.EndKey, true /* exclusiveEnd */)
overlap := vers.Overlaps(i, span.StartKey, span.EndKey, true /* exclusiveEnd */)
foundExternalFile := false
overlap.Each(func(metadata *manifest.FileMetadata) {
objMeta, err := d.objProvider.Lookup(fileTypeTable, metadata.FileBacking.DiskFileNum)
Expand Down Expand Up @@ -2407,7 +2407,7 @@ func (d *DB) EstimateDiskUsageByBackingType(
// We can only use `Overlaps` to restrict `files` at L1+ since at L0 it
// expands the range iteratively until it has found a set of files that
// do not overlap any other L0 files outside that set.
overlaps := readState.current.Overlaps(level, d.opts.Comparer.Compare, start, end, false /* exclusiveEnd */)
overlaps := readState.current.Overlaps(level, start, end, false /* exclusiveEnd */)
iter = overlaps.Iter()
}
for file := iter.First(); file != nil; file = iter.Next() {
Expand Down Expand Up @@ -3087,5 +3087,5 @@ func (d *DB) checkVirtualBounds(m *fileMetadata) {
func (d *DB) DebugString() string {
d.mu.Lock()
defer d.mu.Unlock()
return d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
return d.mu.versions.currentVersion().DebugString()
}
Loading

0 comments on commit aed0b13

Please sign in to comment.