Skip to content

Commit

Permalink
manifest: improve Annotator interface with generics
Browse files Browse the repository at this point in the history
Refactors `manifest.Annotator` to use generics and a simplified API.
This eliminates the need to perform pointer manipulation and unsafe
typecasting when defining a new Annotator. The goal of this change is to improve the `Annotator` interface while not changing any existing behavior.

`BenchmarkNumFilesAnnotator` shows roughly the same performance as master when compared
to the equivalent implementation of `orderStatistic`:
```
pkg: github.com/cockroachdb/pebble/internal/manifest
                     │     old     │             new              │
                     │   sec/op    │   sec/op     vs base         │
NumFilesAnnotator-10   1.635µ ± 1%   1.572µ ± 7%  ~ (p=0.065 n=6)

                     │    old     │              new              │
                     │    B/op    │    B/op     vs base           │
NumFilesAnnotator-10   536.0 ± 0%   536.0 ± 0%  ~ (p=1.000 n=6) ¹
¹ all samples are equal

                     │    old     │              new              │
                     │ allocs/op  │ allocs/op   vs base           │
NumFilesAnnotator-10   7.000 ± 0%   7.000 ± 0%  ~ (p=1.000 n=6) ¹
¹ all samples are equal
```
  • Loading branch information
anish-shanbhag committed Jul 22, 2024
1 parent 7920d96 commit a5e666e
Show file tree
Hide file tree
Showing 11 changed files with 436 additions and 536 deletions.
209 changes: 62 additions & 147 deletions compaction_picker.go
Original file line number Diff line number Diff line change
Expand Up @@ -639,39 +639,13 @@ func compensatedSize(f *fileMetadata) uint64 {
return f.Size + fileCompensation(f)
}

// compensatedSizeAnnotator implements manifest.Annotator, annotating B-Tree
// nodes with the sum of the files' compensated sizes. Its annotation type is
// a *uint64. Compensated sizes may change once a table's stats are loaded
// asynchronously, so its values are marked as cacheable only if a file's
// stats have been loaded.
type compensatedSizeAnnotator struct {
}

var _ manifest.Annotator = compensatedSizeAnnotator{}

func (a compensatedSizeAnnotator) Zero(dst interface{}) interface{} {
if dst == nil {
return new(uint64)
}
v := dst.(*uint64)
*v = 0
return v
}

func (a compensatedSizeAnnotator) Accumulate(
f *fileMetadata, dst interface{},
) (v interface{}, cacheOK bool) {
vptr := dst.(*uint64)
*vptr = *vptr + compensatedSize(f)
return vptr, f.StatsValid()
}

func (a compensatedSizeAnnotator) Merge(src interface{}, dst interface{}) interface{} {
srcV := src.(*uint64)
dstV := dst.(*uint64)
*dstV = *dstV + *srcV
return dstV
}
// compensatedSizeAnnotator is a manifest.Annotator that annotates B-Tree
// nodes with the sum of the files' compensated sizes. Compensated sizes may
// change once a table's stats are loaded asynchronously, so its values are
// marked as cacheable only if a file's stats have been loaded.
var compensatedSizeAnnotator = manifest.SumAnnotator(func(f *fileMetadata) (uint64, bool) {
return compensatedSize(f), f.StatsValid()
})

// totalCompensatedSize computes the compensated size over a file metadata
// iterator. Note that this function is linear in the files available to the
Expand Down Expand Up @@ -912,10 +886,6 @@ func calculateSizeAdjust(inProgressCompactions []compactionInfo) [numLevels]leve
return sizeAdjust
}

func levelCompensatedSize(lm manifest.LevelMetadata) uint64 {
return *lm.Annotation(compensatedSizeAnnotator{}).(*uint64)
}

func (p *compactionPickerByScore) calculateLevelScores(
inProgressCompactions []compactionInfo,
) [numLevels]candidateLevelInfo {
Expand All @@ -932,7 +902,7 @@ func (p *compactionPickerByScore) calculateLevelScores(
}
sizeAdjust := calculateSizeAdjust(inProgressCompactions)
for level := 1; level < numLevels; level++ {
compensatedLevelSize := levelCompensatedSize(p.vers.Levels[level]) + sizeAdjust[level].compensated()
compensatedLevelSize := *compensatedSizeAnnotator.LevelAnnotation(p.vers.Levels[level]) + sizeAdjust[level].compensated()
scores[level].compensatedScore = float64(compensatedLevelSize) / float64(p.levelMaxBytes[level])
scores[level].uncompensatedScore = float64(p.vers.Levels[level].Size()+sizeAdjust[level].actual()) / float64(p.levelMaxBytes[level])
}
Expand Down Expand Up @@ -1393,109 +1363,56 @@ func (p *compactionPickerByScore) addScoresToPickedCompactionMetrics(
}
}

// elisionOnlyAnnotator implements the manifest.Annotator interface,
// annotating B-Tree nodes with the *fileMetadata of a file meeting the
// obsolete keys criteria for an elision-only compaction within the subtree.
// If multiple files meet the criteria, it chooses whichever file has the
// lowest LargestSeqNum. The lowest LargestSeqNum file will be the first
// eligible for an elision-only compaction once snapshots less than or equal
// to its LargestSeqNum are closed.
type elisionOnlyAnnotator struct{}

var _ manifest.Annotator = elisionOnlyAnnotator{}

func (a elisionOnlyAnnotator) Zero(interface{}) interface{} {
return nil
}

func (a elisionOnlyAnnotator) Accumulate(f *fileMetadata, dst interface{}) (interface{}, bool) {
if f.IsCompacting() {
return dst, true
}
if !f.StatsValid() {
return dst, false
}
// Bottommost files are large and not worthwhile to compact just
// to remove a few tombstones. Consider a file ineligible if its
// own range deletions delete less than 10% of its data and its
// deletion tombstones make up less than 10% of its entries.
//
// TODO(jackson): This does not account for duplicate user keys
// which may be collapsed. Ideally, we would have 'obsolete keys'
// statistics that would include tombstones, the keys that are
// dropped by tombstones and duplicated user keys. See #847.
//
// Note that tables that contain exclusively range keys (i.e. no point keys,
// `NumEntries` and `RangeDeletionsBytesEstimate` are both zero) are excluded
// from elision-only compactions.
// TODO(travers): Consider an alternative heuristic for elision of range-keys.
if f.Stats.RangeDeletionsBytesEstimate*10 < f.Size &&
f.Stats.NumDeletions*10 <= f.Stats.NumEntries {
return dst, true
}
if dst == nil {
return f, true
} else if dstV := dst.(*fileMetadata); dstV.LargestSeqNum > f.LargestSeqNum {
return f, true
}
return dst, true
}

func (a elisionOnlyAnnotator) Merge(v interface{}, accum interface{}) interface{} {
if v == nil {
return accum
}
// If we haven't accumulated an eligible file yet, or f's LargestSeqNum is
// less than the accumulated file's, use f.
if accum == nil {
return v
}
f := v.(*fileMetadata)
accumV := accum.(*fileMetadata)
if accumV == nil || accumV.LargestSeqNum > f.LargestSeqNum {
return f
}
return accumV
}

// markedForCompactionAnnotator implements the manifest.Annotator interface,
// annotating B-Tree nodes with the *fileMetadata of a file that is marked for
// compaction within the subtree. If multiple files meet the criteria, it
// chooses whichever file has the lowest LargestSeqNum.
type markedForCompactionAnnotator struct{}

var _ manifest.Annotator = markedForCompactionAnnotator{}

func (a markedForCompactionAnnotator) Zero(interface{}) interface{} {
return nil
}

func (a markedForCompactionAnnotator) Accumulate(
f *fileMetadata, dst interface{},
) (interface{}, bool) {
if !f.MarkedForCompaction {
// Not marked for compaction; return dst.
return dst, true
}
return markedMergeHelper(f, dst)
}

func (a markedForCompactionAnnotator) Merge(v interface{}, accum interface{}) interface{} {
if v == nil {
return accum
}
accum, _ = markedMergeHelper(v.(*fileMetadata), accum)
return accum
}

// REQUIRES: f is non-nil, and f.MarkedForCompaction=true.
func markedMergeHelper(f *fileMetadata, dst interface{}) (interface{}, bool) {
if dst == nil {
return f, true
} else if dstV := dst.(*fileMetadata); dstV.LargestSeqNum > f.LargestSeqNum {
return f, true
}
return dst, true
// elisionOnlyAnnotator is a manifest.Annotator that annotates B-Tree
// nodes with the *fileMetadata of a file meeting the obsolete keys criteria
// for an elision-only compaction within the subtree. If multiple files meet
// the criteria, it chooses whichever file has the lowest LargestSeqNum. The
// lowest LargestSeqNum file will be the first eligible for an elision-only
// compaction once snapshots less than or equal to its LargestSeqNum are closed.
var elisionOnlyAnnotator = &manifest.Annotator[fileMetadata]{
Aggregator: manifest.PickFileAggregator{
Filter: func(f *fileMetadata) (eligible bool, cacheOK bool) {
if f.IsCompacting() {
return false, true
}
if !f.StatsValid() {
return false, false
}
// Bottommost files are large and not worthwhile to compact just
// to remove a few tombstones. Consider a file eligible only if
// either its own range deletions delete at least 10% of its data or
// its deletion tombstones make at least 10% of its entries.
//
// TODO(jackson): This does not account for duplicate user keys
// which may be collapsed. Ideally, we would have 'obsolete keys'
// statistics that would include tombstones, the keys that are
// dropped by tombstones and duplicated user keys. See #847.
//
// Note that tables that contain exclusively range keys (i.e. no point keys,
// `NumEntries` and `RangeDeletionsBytesEstimate` are both zero) are excluded
// from elision-only compactions.
// TODO(travers): Consider an alternative heuristic for elision of range-keys.
return f.Stats.RangeDeletionsBytesEstimate*10 >= f.Size || f.Stats.NumDeletions*10 > f.Stats.NumEntries, true
},
Compare: func(f1 *fileMetadata, f2 *fileMetadata) bool {
return f1.LargestSeqNum < f2.LargestSeqNum
},
},
}

// markedForCompactionAnnotator is a manifest.Annotator that annotates B-Tree
// nodes with the *fileMetadata of a file that is marked for compaction
// within the subtree. If multiple files meet the criteria, it chooses
// whichever file has the lowest LargestSeqNum.
var markedForCompactionAnnotator = &manifest.Annotator[fileMetadata]{
Aggregator: manifest.PickFileAggregator{
Filter: func(f *fileMetadata) (eligible bool, cacheOK bool) {
return f.MarkedForCompaction, true
},
Compare: func(f1 *fileMetadata, f2 *fileMetadata) bool {
return f1.LargestSeqNum < f2.LargestSeqNum
},
},
}

// pickElisionOnlyCompaction looks for compactions of sstables in the
Expand All @@ -1506,11 +1423,10 @@ func (p *compactionPickerByScore) pickElisionOnlyCompaction(
if p.opts.private.disableElisionOnlyCompactions {
return nil
}
v := p.vers.Levels[numLevels-1].Annotation(elisionOnlyAnnotator{})
if v == nil {
candidate := elisionOnlyAnnotator.LevelAnnotation(p.vers.Levels[numLevels-1])
if candidate == nil {
return nil
}
candidate := v.(*fileMetadata)
if candidate.IsCompacting() || candidate.LargestSeqNum >= env.earliestSnapshotSeqNum {
return nil
}
Expand Down Expand Up @@ -1542,12 +1458,11 @@ func (p *compactionPickerByScore) pickElisionOnlyCompaction(
// the input level.
func (p *compactionPickerByScore) pickRewriteCompaction(env compactionEnv) (pc *pickedCompaction) {
for l := numLevels - 1; l >= 0; l-- {
v := p.vers.Levels[l].Annotation(markedForCompactionAnnotator{})
if v == nil {
candidate := markedForCompactionAnnotator.LevelAnnotation(p.vers.Levels[l])
if candidate == nil {
// Try the next level.
continue
}
candidate := v.(*fileMetadata)
if candidate.IsCompacting() {
// Try the next level.
continue
Expand Down
2 changes: 1 addition & 1 deletion compaction_picker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -563,7 +563,7 @@ func TestCompactionPickerL0(t *testing.T) {
}
f.MarkedForCompaction = true
picker.vers.Stats.MarkedForCompaction++
picker.vers.Levels[l].InvalidateAnnotation(markedForCompactionAnnotator{})
markedForCompactionAnnotator.InvalidateLevelAnnotation(picker.vers.Levels[l])
return fmt.Sprintf("marked L%d.%s", l, f.FileNum)
}
}
Expand Down
2 changes: 1 addition & 1 deletion compaction_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2505,7 +2505,7 @@ func TestMarkedForCompaction(t *testing.T) {
}
f.MarkedForCompaction = true
vers.Stats.MarkedForCompaction++
vers.Levels[l].InvalidateAnnotation(markedForCompactionAnnotator{})
markedForCompactionAnnotator.InvalidateLevelAnnotation(vers.Levels[l])
return fmt.Sprintf("marked L%d.%s", l, f.FileNum)
}
}
Expand Down
16 changes: 8 additions & 8 deletions db.go
Original file line number Diff line number Diff line change
Expand Up @@ -1995,8 +1995,8 @@ func (d *DB) Metrics() *Metrics {
metrics.private.optionsFileSize = d.optionsFileSize

// TODO(jackson): Consider making these metrics optional.
metrics.Keys.RangeKeySetsCount = countRangeKeySetFragments(vers)
metrics.Keys.TombstoneCount = countTombstones(vers)
metrics.Keys.RangeKeySetsCount = *rangeKeySetsAnnotator.MultiLevelAnnotation(vers.RangeKeyLevels[:])
metrics.Keys.TombstoneCount = *tombstonesAnnotator.MultiLevelAnnotation(vers.Levels[:])

d.mu.versions.logLock()
metrics.private.manifestFileSize = uint64(d.mu.versions.manifest.Size())
Expand All @@ -2014,12 +2014,12 @@ func (d *DB) Metrics() *Metrics {
metrics.Flush.NumInProgress = 1
}
for i := 0; i < numLevels; i++ {
metrics.Levels[i].Additional.ValueBlocksSize = valueBlocksSizeForLevel(vers, i)
unknown, snappy, none, zstd := compressionTypesForLevel(vers, i)
metrics.Table.CompressedCountUnknown += int64(unknown)
metrics.Table.CompressedCountSnappy += int64(snappy)
metrics.Table.CompressedCountZstd += int64(zstd)
metrics.Table.CompressedCountNone += int64(none)
metrics.Levels[i].Additional.ValueBlocksSize = *valueBlockSizeAnnotator.LevelAnnotation(vers.Levels[i])
compressionTypes := compressionTypeAnnotator.LevelAnnotation(vers.Levels[i])
metrics.Table.CompressedCountUnknown += int64(compressionTypes.unknown)
metrics.Table.CompressedCountSnappy += int64(compressionTypes.snappy)
metrics.Table.CompressedCountZstd += int64(compressionTypes.zstd)
metrics.Table.CompressedCountNone += int64(compressionTypes.none)
}

d.mu.Unlock()
Expand Down
2 changes: 1 addition & 1 deletion format_major_version.go
Original file line number Diff line number Diff line change
Expand Up @@ -517,7 +517,7 @@ func (d *DB) markFilesLocked(findFn findFilesFunc) error {
// annotations will be out of date. Clear the compaction-picking
// annotation, so that it's recomputed the next time the compaction
// picker looks for a file marked for compaction.
vers.Levels[l].InvalidateAnnotation(markedForCompactionAnnotator{})
markedForCompactionAnnotator.InvalidateLevelAnnotation(vers.Levels[l])
}

// The 'marked-for-compaction' bit is persisted in the MANIFEST file
Expand Down
Loading

0 comments on commit a5e666e

Please sign in to comment.