Skip to content

Commit 58846b6

Browse files
committed
db: aggregate blob file compression stats
We rename `manifest.Annotator` to `TableAnnotator` and add a `BlobFileAnnotator` type. Internally, we use a generic `annotator` type to share code. We use a `BlobFileAnnotator` to report compression statistics for blob files.
1 parent 2c04bce commit 58846b6

File tree

14 files changed

+445
-264
lines changed

14 files changed

+445
-264
lines changed

compaction_picker.go

Lines changed: 39 additions & 43 deletions
Original file line numberDiff line numberDiff line change
@@ -1557,62 +1557,58 @@ func (p *compactionPickerByScore) addScoresToPickedCompactionMetrics(
15571557
}
15581558
}
15591559

1560-
// elisionOnlyAnnotator is a manifest.Annotator that annotates B-Tree
1560+
// elisionOnlyAnnotator is a manifest.TableAnnotator that annotates B-Tree
15611561
// nodes with the *fileMetadata of a file meeting the obsolete keys criteria
15621562
// for an elision-only compaction within the subtree. If multiple files meet
15631563
// the criteria, it chooses whichever file has the lowest LargestSeqNum. The
15641564
// lowest LargestSeqNum file will be the first eligible for an elision-only
15651565
// compaction once snapshots less than or equal to its LargestSeqNum are closed.
1566-
var elisionOnlyAnnotator = &manifest.Annotator[manifest.TableMetadata]{
1567-
Aggregator: manifest.PickFileAggregator{
1568-
Filter: func(f *manifest.TableMetadata) (eligible bool, cacheOK bool) {
1569-
if f.IsCompacting() {
1570-
return false, true
1571-
}
1566+
var elisionOnlyAnnotator = manifest.NewTableAnnotator[manifest.TableMetadata](manifest.PickFileAggregator{
1567+
Filter: func(f *manifest.TableMetadata) (eligible bool, cacheOK bool) {
1568+
if f.IsCompacting() {
1569+
return false, true
1570+
}
15721571

1573-
backingProps, backingPropsValid := f.TableBacking.Properties()
1574-
stats, statsValid := f.Stats()
1575-
if !backingPropsValid || !statsValid {
1576-
return false, false
1577-
}
1572+
backingProps, backingPropsValid := f.TableBacking.Properties()
1573+
stats, statsValid := f.Stats()
1574+
if !backingPropsValid || !statsValid {
1575+
return false, false
1576+
}
15781577

1579-
// Bottommost files are large and not worthwhile to compact just
1580-
// to remove a few tombstones. Consider a file eligible only if
1581-
// either its own range deletions delete at least 10% of its data or
1582-
// its deletion tombstones make at least 10% of its entries.
1583-
//
1584-
// TODO(jackson): This does not account for duplicate user keys
1585-
// which may be collapsed. Ideally, we would have 'obsolete keys'
1586-
// statistics that would include tombstones, the keys that are
1587-
// dropped by tombstones and duplicated user keys. See #847.
1588-
//
1589-
// Note that tables that contain exclusively range keys (i.e. no point keys,
1590-
// `NumEntries` and `RangeDeletionsBytesEstimate` are both zero) are excluded
1591-
// from elision-only compactions.
1592-
// TODO(travers): Consider an alternative heuristic for elision of range-keys.
1593-
eligible = stats.RangeDeletionsBytesEstimate*10 >= f.Size || backingProps.NumDeletions*10 > backingProps.NumEntries
1594-
return eligible, true
1595-
},
1596-
Compare: func(f1 *manifest.TableMetadata, f2 *manifest.TableMetadata) bool {
1597-
return f1.LargestSeqNum < f2.LargestSeqNum
1598-
},
1578+
// Bottommost files are large and not worthwhile to compact just
1579+
// to remove a few tombstones. Consider a file eligible only if
1580+
// either its own range deletions delete at least 10% of its data or
1581+
// its deletion tombstones make at least 10% of its entries.
1582+
//
1583+
// TODO(jackson): This does not account for duplicate user keys
1584+
// which may be collapsed. Ideally, we would have 'obsolete keys'
1585+
// statistics that would include tombstones, the keys that are
1586+
// dropped by tombstones and duplicated user keys. See #847.
1587+
//
1588+
// Note that tables that contain exclusively range keys (i.e. no point keys,
1589+
// `NumEntries` and `RangeDeletionsBytesEstimate` are both zero) are excluded
1590+
// from elision-only compactions.
1591+
// TODO(travers): Consider an alternative heuristic for elision of range-keys.
1592+
eligible = stats.RangeDeletionsBytesEstimate*10 >= f.Size || backingProps.NumDeletions*10 > backingProps.NumEntries
1593+
return eligible, true
15991594
},
1600-
}
1595+
Compare: func(f1 *manifest.TableMetadata, f2 *manifest.TableMetadata) bool {
1596+
return f1.LargestSeqNum < f2.LargestSeqNum
1597+
},
1598+
})
16011599

1602-
// markedForCompactionAnnotator is a manifest.Annotator that annotates B-Tree
1600+
// markedForCompactionAnnotator is a manifest.TableAnnotator that annotates B-Tree
16031601
// nodes with the *fileMetadata of a file that is marked for compaction
16041602
// within the subtree. If multiple files meet the criteria, it chooses
16051603
// whichever file has the lowest LargestSeqNum.
1606-
var markedForCompactionAnnotator = &manifest.Annotator[manifest.TableMetadata]{
1607-
Aggregator: manifest.PickFileAggregator{
1608-
Filter: func(f *manifest.TableMetadata) (eligible bool, cacheOK bool) {
1609-
return f.MarkedForCompaction, true
1610-
},
1611-
Compare: func(f1 *manifest.TableMetadata, f2 *manifest.TableMetadata) bool {
1612-
return f1.LargestSeqNum < f2.LargestSeqNum
1613-
},
1604+
var markedForCompactionAnnotator = manifest.NewTableAnnotator[manifest.TableMetadata](manifest.PickFileAggregator{
1605+
Filter: func(f *manifest.TableMetadata) (eligible bool, cacheOK bool) {
1606+
return f.MarkedForCompaction, true
16141607
},
1615-
}
1608+
Compare: func(f1 *manifest.TableMetadata, f2 *manifest.TableMetadata) bool {
1609+
return f1.LargestSeqNum < f2.LargestSeqNum
1610+
},
1611+
})
16161612

16171613
// pickedCompactionFromCandidateFile creates a pickedCompaction from a *fileMetadata
16181614
// with various checks to ensure that the file still exists in the expected level

db.go

Lines changed: 31 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -504,15 +504,15 @@ type DB struct {
504504
validating bool
505505
}
506506

507-
// annotators contains various instances of manifest.Annotator which
507+
// annotators contains various instances of manifest.TableAnnotator which
508508
// should be protected from concurrent access.
509509
annotators struct {
510510
// totalFileSize is the sum of the size of all files in the
511511
// database. This includes local, remote, and external sstables --
512512
// along with blob files.
513-
totalFileSize *manifest.Annotator[uint64]
514-
remoteSize *manifest.Annotator[uint64]
515-
externalSize *manifest.Annotator[uint64]
513+
totalFileSize *manifest.TableAnnotator[uint64]
514+
remoteSize *manifest.TableAnnotator[uint64]
515+
externalSize *manifest.TableAnnotator[uint64]
516516
}
517517
}
518518

@@ -2165,14 +2165,18 @@ func (d *DB) Metrics() *Metrics {
21652165
if d.mu.compact.flushing {
21662166
metrics.Flush.NumInProgress = 1
21672167
}
2168+
2169+
metrics.Table.PendingStatsCollectionCount = int64(len(d.mu.tableStats.pending))
2170+
metrics.Table.InitialStatsCollectionComplete = d.mu.tableStats.loadedInitial
2171+
21682172
for i := 0; i < numLevels; i++ {
21692173
metrics.Levels[i].Additional.ValueBlocksSize = *valueBlockSizeAnnotator.LevelAnnotation(vers.Levels[i])
21702174
compressionMetrics := compressionStatsAnnotator.LevelAnnotation(vers.Levels[i])
21712175
metrics.Table.Compression.MergeWith(compressionMetrics)
21722176
}
21732177

2174-
metrics.Table.PendingStatsCollectionCount = int64(len(d.mu.tableStats.pending))
2175-
metrics.Table.InitialStatsCollectionComplete = d.mu.tableStats.loadedInitial
2178+
blobCompressionMetrics := blobCompressionStatsAnnotator.Annotation(&vers.BlobFiles)
2179+
metrics.BlobFiles.Compression.MergeWith(blobCompressionMetrics)
21762180

21772181
d.mu.Unlock()
21782182

@@ -2398,31 +2402,29 @@ func (d *DB) SSTables(opts ...SSTablesOption) ([][]SSTableInfo, error) {
23982402
// referenced blob files.
23992403
func (d *DB) makeFileSizeAnnotator(
24002404
filter func(f *manifest.TableMetadata) bool,
2401-
) *manifest.Annotator[uint64] {
2402-
return &manifest.Annotator[uint64]{
2403-
Aggregator: manifest.SumAggregator{
2404-
AccumulateFunc: func(f *manifest.TableMetadata) (uint64, bool) {
2405-
if filter(f) {
2406-
return f.Size + f.EstimatedReferenceSize(), true
2407-
}
2408-
return 0, true
2409-
},
2410-
AccumulatePartialOverlapFunc: func(f *manifest.TableMetadata, bounds base.UserKeyBounds) uint64 {
2411-
if filter(f) {
2412-
overlappingFileSize, err := d.fileCache.estimateSize(f, bounds.Start, bounds.End.Key)
2413-
if err != nil {
2414-
return 0
2415-
}
2416-
overlapFraction := float64(overlappingFileSize) / float64(f.Size)
2417-
// Scale the blob reference size proportionally to the file
2418-
// overlap from the bounds to approximate only the blob
2419-
// references that overlap with the requested bounds.
2420-
return overlappingFileSize + uint64(float64(f.EstimatedReferenceSize())*overlapFraction)
2405+
) *manifest.TableAnnotator[uint64] {
2406+
return manifest.NewTableAnnotator[uint64](manifest.SumAggregator{
2407+
AccumulateFunc: func(f *manifest.TableMetadata) (uint64, bool) {
2408+
if filter(f) {
2409+
return f.Size + f.EstimatedReferenceSize(), true
2410+
}
2411+
return 0, true
2412+
},
2413+
AccumulatePartialOverlapFunc: func(f *manifest.TableMetadata, bounds base.UserKeyBounds) uint64 {
2414+
if filter(f) {
2415+
overlappingFileSize, err := d.fileCache.estimateSize(f, bounds.Start, bounds.End.Key)
2416+
if err != nil {
2417+
return 0
24212418
}
2422-
return 0
2423-
},
2419+
overlapFraction := float64(overlappingFileSize) / float64(f.Size)
2420+
// Scale the blob reference size proportionally to the file
2421+
// overlap from the bounds to approximate only the blob
2422+
// references that overlap with the requested bounds.
2423+
return overlappingFileSize + uint64(float64(f.EstimatedReferenceSize())*overlapFraction)
2424+
}
2425+
return 0
24242426
},
2425-
}
2427+
})
24262428
}
24272429

24282430
// EstimateDiskUsage returns the estimated filesystem space used in bytes for

0 commit comments

Comments
 (0)