Skip to content

Commit c1d3686

Browse files
committed
metrics: expose new compression stats
Update the compression metrics to use the new compression statistics.
1 parent 70fa14f commit c1d3686

19 files changed

+293
-155
lines changed

compaction.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3512,7 +3512,7 @@ func (c *tableCompaction) makeVersionEdit(result compact.Result) (*manifest.Vers
35123512
// If the file didn't contain any range deletions, we can fill its
35133513
// table stats now, avoiding unnecessarily loading the table later.
35143514
maybeSetStatsFromProperties(
3515-
fileMeta.PhysicalMeta(), &t.WriterMeta.Properties.CommonProperties, c.logger,
3515+
fileMeta.PhysicalMeta(), &t.WriterMeta.Properties, c.logger,
35163516
)
35173517

35183518
if t.WriterMeta.HasPointKeys {

db.go

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -2185,12 +2185,8 @@ func (d *DB) Metrics() *Metrics {
21852185
}
21862186
for i := 0; i < numLevels; i++ {
21872187
metrics.Levels[i].Additional.ValueBlocksSize = *valueBlockSizeAnnotator.LevelAnnotation(vers.Levels[i])
2188-
compressionTypes := compressionTypeAnnotator.LevelAnnotation(vers.Levels[i])
2189-
metrics.Table.CompressedCountUnknown += int64(compressionTypes.unknown)
2190-
metrics.Table.CompressedCountSnappy += int64(compressionTypes.snappy)
2191-
metrics.Table.CompressedCountZstd += int64(compressionTypes.zstd)
2192-
metrics.Table.CompressedCountMinLZ += int64(compressionTypes.minlz)
2193-
metrics.Table.CompressedCountNone += int64(compressionTypes.none)
2188+
compressionMetrics := compressionStatsAnnotator.LevelAnnotation(vers.Levels[i])
2189+
metrics.Compression.MergeWith(compressionMetrics)
21942190
}
21952191

21962192
metrics.Table.PendingStatsCollectionCount = int64(len(d.mu.tableStats.pending))

ingest.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -316,7 +316,7 @@ func ingestLoad1(
316316
// disallowing removal of an open file. Under MemFS, if we don't populate
317317
// meta.Stats here, the file will be loaded into the file cache for
318318
// calculating stats before we can remove the original link.
319-
maybeSetStatsFromProperties(meta.PhysicalMeta(), &props.CommonProperties, opts.Logger)
319+
maybeSetStatsFromProperties(meta.PhysicalMeta(), &props, opts.Logger)
320320

321321
{
322322
iter, err := r.NewIter(sstable.NoTransforms, nil /* lower */, nil /* upper */, sstable.AssertNoBlobHandles)

ingest_test.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -249,6 +249,7 @@ func TestIngestLoadRand(t *testing.T) {
249249

250250
for _, m := range lr.local {
251251
m.CreationTime = 0
252+
m.Stats.CompressionStats = block.CompressionStats{}
252253
}
253254
t.Log(strings.Join(pretty.Diff(expected, lr.local), "\n"))
254255
require.Equal(t, expected, lr.local)

internal/manifest/table_metadata.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1167,6 +1167,9 @@ type TableStats struct {
11671167
// CompressionType is the compression profile used for the table (or nil if
11681168
// the profile name is not recognized).
11691169
CompressionType *block.CompressionProfile
1170+
// CompressionStats contains compression statistics; not available for older
1171+
// table versions.
1172+
CompressionStats block.CompressionStats
11701173
// TombstoneDenseBlocksRatio is the ratio of data blocks in this table that
11711174
// fulfills at least one of the following:
11721175
// 1. The block contains at least options.Experimental.NumDeletionsThreshold

internal/manifest/table_metadata_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -148,7 +148,7 @@ func TestTableMetadataSize(t *testing.T) {
148148
}
149149
structSize := unsafe.Sizeof(TableMetadata{})
150150

151-
const tableMetadataSize = 320
151+
const tableMetadataSize = 352
152152
if structSize != tableMetadataSize {
153153
t.Errorf("TableMetadata struct size (%d bytes) is not expected size (%d bytes)",
154154
structSize, tableMetadataSize)

metrics.go

Lines changed: 67 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ import (
1717
"github.com/cockroachdb/pebble/internal/ascii/table"
1818
"github.com/cockroachdb/pebble/internal/base"
1919
"github.com/cockroachdb/pebble/internal/cache"
20+
"github.com/cockroachdb/pebble/internal/compression"
2021
"github.com/cockroachdb/pebble/internal/manifest"
2122
"github.com/cockroachdb/pebble/internal/manual"
2223
"github.com/cockroachdb/pebble/objstorage/objstorageprovider/sharedcache"
@@ -347,18 +348,6 @@ type Metrics struct {
347348
BackingTableCount uint64
348349
// The sum of the sizes of the BackingTableCount sstables that are backing virtual tables.
349350
BackingTableSize uint64
350-
// The number of sstables that are compressed with an unknown compression
351-
// algorithm.
352-
CompressedCountUnknown int64
353-
// The number of sstables that are compressed with the default compression
354-
// algorithm, snappy.
355-
CompressedCountSnappy int64
356-
// The number of sstables that are compressed with zstd.
357-
CompressedCountZstd int64
358-
// The number of sstables that are compressed with minlz.
359-
CompressedCountMinLZ int64
360-
// The number of sstables that are uncompressed.
361-
CompressedCountNone int64
362351

363352
// Local file sizes.
364353
Local struct {
@@ -450,6 +439,8 @@ type Metrics struct {
450439
}
451440
}
452441

442+
Compression CompressionMetrics
443+
453444
FileCache FileCacheMetrics
454445

455446
// Count of the number of open sstable iterators.
@@ -498,6 +489,46 @@ type Metrics struct {
498489
manualMemory manual.Metrics
499490
}
500491

492+
// CompressionMetrics contains compression metrics for sstables and blob files.
493+
type CompressionMetrics struct {
494+
// NoCompressionBytes is the total number of bytes in files that do are not
495+
// compressed. Data can be uncompressed when 1) compression is disabled; 2)
496+
// for certain special types of blocks; and 3) for blocks that are not
497+
// compressible.
498+
NoCompressionBytes uint64
499+
// CompressedBytesWithoutStats is the total number of bytes in files that do not
500+
// encode compression statistics (or for which there are no statistics yet).
501+
CompressedBytesWithoutStats uint64
502+
Snappy CompressionStatsForSetting
503+
MinLZ CompressionStatsForSetting
504+
Zstd CompressionStatsForSetting
505+
}
506+
507+
type CompressionStatsForSetting = block.CompressionStatsForSetting
508+
509+
func (cm *CompressionMetrics) Add(stats *block.CompressionStats) {
510+
for s, cs := range stats.All() {
511+
switch s.Algorithm {
512+
case compression.NoCompression:
513+
cm.NoCompressionBytes += cs.UncompressedBytes
514+
case compression.SnappyAlgorithm:
515+
cm.Snappy.Add(cs)
516+
case compression.MinLZ:
517+
cm.MinLZ.Add(cs)
518+
case compression.Zstd:
519+
cm.Zstd.Add(cs)
520+
}
521+
}
522+
}
523+
524+
func (cm *CompressionMetrics) MergeWith(o *CompressionMetrics) {
525+
cm.NoCompressionBytes += o.NoCompressionBytes
526+
cm.CompressedBytesWithoutStats += o.CompressedBytesWithoutStats
527+
cm.Snappy.Add(o.Snappy)
528+
cm.MinLZ.Add(o.MinLZ)
529+
cm.Zstd.Add(o.Zstd)
530+
}
531+
501532
var (
502533
// FsyncLatencyBuckets are prometheus histogram buckets suitable for a histogram
503534
// that records latencies for fsyncs.
@@ -751,9 +782,18 @@ var (
751782
table.String("range dels", 15, table.AlignRight, func(i keysInfo) string { return i.rangeDels }),
752783
)
753784
compressionTableHeader = `COMPRESSION`
754-
compressionTable = table.Define[pair[string, int64]](
755-
table.String("algorithm", 10, table.AlignRight, func(p pair[string, int64]) string { return p.k }),
756-
table.Count("tables", 10, table.AlignRight, func(p pair[string, int64]) int64 { return p.v }),
785+
compressionTable = table.Define[pair[string, CompressionStatsForSetting]](
786+
table.String("algorithm", 13, table.AlignRight, func(p pair[string, CompressionStatsForSetting]) string { return p.k }),
787+
table.Bytes("on disk bytes", 13, table.AlignRight, func(p pair[string, CompressionStatsForSetting]) uint64 { return p.v.CompressedBytes }),
788+
table.String("CR", 13, table.AlignRight, func(p pair[string, CompressionStatsForSetting]) string {
789+
if p.v.UncompressedBytes == p.v.CompressedBytes {
790+
return ""
791+
}
792+
if p.v.UncompressedBytes == 0 {
793+
return "?"
794+
}
795+
return crhumanize.Float(p.v.CompressionRatio(), 2 /* precision */).String()
796+
}),
757797
)
758798
)
759799

@@ -974,15 +1014,19 @@ func (m *Metrics) String() string {
9741014
cur = cur.NewlineReturn()
9751015

9761016
cur = cur.WriteString(compressionTableHeader).NewlineReturn()
977-
compressionContents := []pair[string, int64]{
978-
{k: "none", v: m.Table.CompressedCountNone},
979-
{k: "snappy", v: m.Table.CompressedCountSnappy},
980-
{k: "minlz", v: m.Table.CompressedCountMinLZ},
981-
{k: "zstd", v: m.Table.CompressedCountZstd},
982-
}
983-
if m.Table.CompressedCountUnknown > 0 {
984-
compressionContents = append(compressionContents, pair[string, int64]{k: "???", v: m.Table.CompressedCountUnknown})
1017+
compressionContents := []pair[string, CompressionStatsForSetting]{
1018+
{k: "none", v: CompressionStatsForSetting{
1019+
CompressedBytes: m.Compression.NoCompressionBytes,
1020+
UncompressedBytes: m.Compression.NoCompressionBytes,
1021+
}},
1022+
{k: "snappy", v: m.Compression.Snappy},
1023+
{k: "minlz", v: m.Compression.MinLZ},
1024+
{k: "zstd", v: m.Compression.Zstd},
1025+
{k: "unknown", v: CompressionStatsForSetting{CompressedBytes: m.Compression.CompressedBytesWithoutStats}},
9851026
}
1027+
compressionContents = slices.DeleteFunc(compressionContents, func(p pair[string, CompressionStatsForSetting]) bool {
1028+
return p.v.CompressedBytes == 0
1029+
})
9861030
compressionTable.Render(cur, table.RenderOptions{Orientation: table.Horizontally}, slices.Values(compressionContents))
9871031

9881032
return wb.String()

metrics_test.go

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@ import (
2222
"github.com/cockroachdb/pebble/internal/cache"
2323
"github.com/cockroachdb/pebble/internal/humanize"
2424
"github.com/cockroachdb/pebble/internal/manifest"
25-
"github.com/cockroachdb/pebble/internal/manual"
2625
"github.com/cockroachdb/pebble/internal/testkeys"
2726
"github.com/cockroachdb/pebble/objstorage/remote"
2827
"github.com/cockroachdb/pebble/sstable/block"
@@ -130,10 +129,6 @@ func exampleMetrics() Metrics {
130129
m.Table.ZombieCount = 18
131130
m.Table.BackingTableCount = 1
132131
m.Table.BackingTableSize = 2 * MB
133-
m.Table.CompressedCountMinLZ = 32
134-
m.Table.CompressedCountSnappy = 33
135-
m.Table.CompressedCountZstd = 34
136-
m.Table.CompressedCountNone = 35
137132
m.Table.Local.LiveSize = 28 * GB
138133
m.Table.Local.LiveCount = 10_000
139134
m.Table.Local.ObsoleteSize = 29 * MB
@@ -160,6 +155,15 @@ func exampleMetrics() Metrics {
160155
m.BlobFiles.Local.ZombieSize = 30 * MB
161156
m.BlobFiles.Local.ZombieCount = 14
162157

158+
m.Compression.NoCompressionBytes = 100 * MB
159+
m.Compression.CompressedBytesWithoutStats = 500 * MB
160+
m.Compression.Snappy.CompressedBytes = 1 * GB
161+
m.Compression.Snappy.UncompressedBytes = 2 * GB
162+
m.Compression.MinLZ.CompressedBytes = 1 * GB
163+
m.Compression.MinLZ.UncompressedBytes = 3 * GB
164+
m.Compression.Zstd.CompressedBytes = 10 * GB
165+
m.Compression.Zstd.UncompressedBytes = 50 * GB
166+
163167
m.FileCache.Size = 1 * MB
164168
m.FileCache.TableCount = 180
165169
m.FileCache.BlobFileCount = 181
@@ -418,9 +422,6 @@ func TestMetrics(t *testing.T) {
418422
d.mu.Unlock()
419423

420424
m := d.Metrics()
421-
// Don't show memory usage as that can depend on architecture, invariants
422-
// tag, etc.
423-
m.manualMemory = manual.Metrics{}
424425
// Some subset of cases show non-determinism in cache hits/misses.
425426
if td.HasArg("zero-cache-hits-misses") {
426427
// Avoid non-determinism.
@@ -432,7 +433,7 @@ func TestMetrics(t *testing.T) {
432433
}
433434
}
434435
var buf strings.Builder
435-
fmt.Fprintf(&buf, "%s", m.StringForTests())
436+
fmt.Fprintf(&buf, "%s\n", m.StringForTests())
436437
if len(m.CategoryStats) > 0 {
437438
fmt.Fprintf(&buf, "\nIter category stats:\n")
438439
for _, stats := range m.CategoryStats {

sstable/block/compression_stats.go

Lines changed: 43 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,7 @@ import (
1111
"slices"
1212
"strings"
1313

14+
"github.com/cockroachdb/crlib/crmath"
1415
"github.com/cockroachdb/errors"
1516
"github.com/cockroachdb/pebble/internal/compression"
1617
"github.com/cockroachdb/pebble/internal/invariants"
@@ -37,13 +38,28 @@ type CompressionStatsForSetting struct {
3738
UncompressedBytes uint64
3839
}
3940

41+
// CompressionRatio returns the compression ratio for the setting.
42+
func (cs CompressionStatsForSetting) CompressionRatio() float64 {
43+
return float64(cs.UncompressedBytes) / float64(cs.CompressedBytes)
44+
}
45+
4046
func (cs *CompressionStatsForSetting) Add(other CompressionStatsForSetting) {
4147
cs.CompressedBytes += other.CompressedBytes
4248
cs.UncompressedBytes += other.UncompressedBytes
4349
}
4450

45-
// add updates the stats to reflect a block that was compressed with the given setting.
46-
func (c *CompressionStats) add(setting compression.Setting, stats CompressionStatsForSetting) {
51+
func (c *CompressionStats) IsEmpty() bool {
52+
return c.noCompressionBytes == 0 && c.fastest.CompressedBytes == 0 && len(c.others) == 0
53+
}
54+
55+
func (c *CompressionStats) Reset() {
56+
c.noCompressionBytes = 0
57+
c.fastest = CompressionStatsForSetting{}
58+
clear(c.others)
59+
}
60+
61+
// addOne updates the stats to reflect a block that was compressed with the given setting.
62+
func (c *CompressionStats) addOne(setting compression.Setting, stats CompressionStatsForSetting) {
4763
switch setting {
4864
case compression.None:
4965
c.noCompressionBytes += stats.UncompressedBytes
@@ -62,10 +78,10 @@ func (c *CompressionStats) add(setting compression.Setting, stats CompressionSta
6278
}
6379
}
6480

65-
// MergeWith updates the receiver stats to include the other stats.
66-
func (c *CompressionStats) MergeWith(other *CompressionStats) {
81+
// Add updates the receiver stats to include the other stats.
82+
func (c *CompressionStats) Add(other *CompressionStats) {
6783
for s, cs := range other.All() {
68-
c.add(s, cs)
84+
c.addOne(s, cs)
6985
}
7086
}
7187

@@ -128,11 +144,32 @@ func (c CompressionStats) String() string {
128144
return buf.String()
129145
}
130146

147+
// Scale the stats by (size/backingSize). Used to obtain an approximation of the
148+
// stats for a virtual table.
149+
func (c *CompressionStats) Scale(size uint64, backingSize uint64) {
150+
// Make sure the sizes are sane, just in case.
151+
size = max(size, 1)
152+
backingSize = max(backingSize, size)
153+
154+
c.noCompressionBytes = crmath.ScaleUint64(c.noCompressionBytes, size, backingSize)
155+
c.fastest.CompressedBytes = crmath.ScaleUint64(c.fastest.CompressedBytes, size, backingSize)
156+
c.fastest.UncompressedBytes = crmath.ScaleUint64(c.fastest.UncompressedBytes, size, backingSize)
157+
158+
for s, cs := range c.others {
159+
cs.CompressedBytes = crmath.ScaleUint64(cs.CompressedBytes, size, backingSize)
160+
cs.UncompressedBytes = crmath.ScaleUint64(cs.UncompressedBytes, size, backingSize)
161+
c.others[s] = cs
162+
}
163+
}
164+
131165
// ParseCompressionStats parses the output of CompressionStats.String back into CompressionStats.
132166
//
133167
// If the string contains statistics for unknown compression settings, these are
134168
// accumulated under a special "unknown" setting.
135169
func ParseCompressionStats(s string) (CompressionStats, error) {
170+
if s == "" {
171+
return CompressionStats{}, nil
172+
}
136173
var stats CompressionStats
137174
for _, a := range strings.Split(s, ",") {
138175
b := strings.Split(a, ":")
@@ -147,7 +184,7 @@ func ParseCompressionStats(s string) (CompressionStats, error) {
147184
if _, err := fmt.Sscanf(b[1], "%d/%d", &cs.CompressedBytes, &cs.UncompressedBytes); err != nil {
148185
return CompressionStats{}, errors.Errorf("cannot parse compression stats %q", s)
149186
}
150-
stats.add(setting, cs)
187+
stats.addOne(setting, cs)
151188
}
152189
return stats, nil
153190
}

sstable/block/compression_stats_test.go

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -15,27 +15,27 @@ import (
1515
func TestCompressionStatsString(t *testing.T) {
1616
var stats CompressionStats
1717

18-
stats.add(compression.None, CompressionStatsForSetting{CompressedBytes: 100, UncompressedBytes: 100})
18+
stats.addOne(compression.None, CompressionStatsForSetting{CompressedBytes: 100, UncompressedBytes: 100})
1919
require.Equal(t, "NoCompression:100/100", stats.String())
2020

21-
stats.add(compression.Snappy, CompressionStatsForSetting{CompressedBytes: 100, UncompressedBytes: 200})
21+
stats.addOne(compression.Snappy, CompressionStatsForSetting{CompressedBytes: 100, UncompressedBytes: 200})
2222
require.Equal(t, "NoCompression:100/100,Snappy:100/200", stats.String())
2323

24-
stats.add(compression.Snappy, CompressionStatsForSetting{CompressedBytes: 100, UncompressedBytes: 200})
24+
stats.addOne(compression.Snappy, CompressionStatsForSetting{CompressedBytes: 100, UncompressedBytes: 200})
2525
require.Equal(t, "NoCompression:100/100,Snappy:200/400", stats.String())
2626

27-
stats.add(compression.MinLZFastest, CompressionStatsForSetting{CompressedBytes: 1000, UncompressedBytes: 4000})
27+
stats.addOne(compression.MinLZFastest, CompressionStatsForSetting{CompressedBytes: 1000, UncompressedBytes: 4000})
2828
require.Equal(t, "MinLZ1:1000/4000,NoCompression:100/100,Snappy:200/400", stats.String())
2929

30-
stats.add(compression.ZstdLevel1, CompressionStatsForSetting{CompressedBytes: 10000, UncompressedBytes: 80000})
30+
stats.addOne(compression.ZstdLevel1, CompressionStatsForSetting{CompressedBytes: 10000, UncompressedBytes: 80000})
3131
require.Equal(t, "MinLZ1:1000/4000,NoCompression:100/100,Snappy:200/400,ZSTD1:10000/80000", stats.String())
3232

3333
stats = CompressionStats{}
34-
stats.add(compression.MinLZFastest, CompressionStatsForSetting{CompressedBytes: 1000, UncompressedBytes: 4000})
34+
stats.addOne(compression.MinLZFastest, CompressionStatsForSetting{CompressedBytes: 1000, UncompressedBytes: 4000})
3535
require.Equal(t, "MinLZ1:1000/4000", stats.String())
3636

3737
stats = CompressionStats{}
38-
stats.add(compression.Snappy, CompressionStatsForSetting{CompressedBytes: 1000, UncompressedBytes: 4000})
38+
stats.addOne(compression.Snappy, CompressionStatsForSetting{CompressedBytes: 1000, UncompressedBytes: 4000})
3939
require.Equal(t, "Snappy:1000/4000", stats.String())
4040
}
4141

@@ -49,7 +49,7 @@ func TestCompressionStatsRoundtrip(t *testing.T) {
4949
if settings[i] != compression.None {
5050
uncompressed += compressed * rand.Uint64N(20) / 10
5151
}
52-
stats.add(settings[i], CompressionStatsForSetting{CompressedBytes: compressed, UncompressedBytes: uncompressed})
52+
stats.addOne(settings[i], CompressionStatsForSetting{CompressedBytes: compressed, UncompressedBytes: uncompressed})
5353
str := stats.String()
5454
stats2, err := ParseCompressionStats(str)
5555
require.NoError(t, err)

0 commit comments

Comments
 (0)