diff --git a/compaction.go b/compaction.go index ac2d0690e46..d13087b7e51 100644 --- a/compaction.go +++ b/compaction.go @@ -733,11 +733,8 @@ func (c *compaction) newInputIters( } }() iterOpts := IterOptions{ - CategoryAndQoS: sstable.CategoryAndQoS{ - Category: "pebble-compaction", - QoSLevel: sstable.NonLatencySensitiveQoSLevel, - }, - logger: c.logger, + Category: categoryCompaction, + logger: c.logger, } // Populate iters, rangeDelIters and rangeKeyIters with the appropriate @@ -1246,11 +1243,8 @@ func (d *DB) runIngestFlush(c *compaction) (*manifest.VersionEdit, error) { comparer: d.opts.Comparer, newIters: d.newIters, opts: IterOptions{ - logger: d.opts.Logger, - CategoryAndQoS: sstable.CategoryAndQoS{ - Category: "pebble-ingest", - QoSLevel: sstable.LatencySensitiveQoSLevel, - }, + logger: d.opts.Logger, + Category: categoryIngest, }, v: c.version, } diff --git a/db.go b/db.go index 3adb2d5e072..17d6baf6082 100644 --- a/db.go +++ b/db.go @@ -579,10 +579,7 @@ func (d *DB) getInternal(key []byte, b *Batch, s *Snapshot) ([]byte, io.Closer, snapshot: seqNum, iterOpts: IterOptions{ // TODO(sumeer): replace with a parameter provided by the caller. - CategoryAndQoS: sstable.CategoryAndQoS{ - Category: "pebble-get", - QoSLevel: sstable.LatencySensitiveQoSLevel, - }, + Category: categoryGet, logger: d.opts.Logger, snapshotForHideObsoletePoints: seqNum, }, @@ -1262,7 +1259,7 @@ func finishInitializingIter(ctx context.Context, buf *iterAlloc) *Iterator { // iteration is invalid in those cases. func (d *DB) ScanInternal( ctx context.Context, - categoryAndQoS sstable.CategoryAndQoS, + category sstable.Category, lower, upper []byte, visitPointKey func(key *InternalKey, value LazyValue, iterInfo IteratorLevel) error, visitRangeDel func(start, end []byte, seqNum SeqNum) error, @@ -1271,7 +1268,7 @@ func (d *DB) ScanInternal( visitExternalFile func(sst *ExternalFile) error, ) error { scanInternalOpts := &scanInternalOptions{ - CategoryAndQoS: categoryAndQoS, + category: category, visitPointKey: visitPointKey, visitRangeDel: visitRangeDel, visitRangeKey: visitRangeKey, @@ -1369,7 +1366,7 @@ func finishInitializingInternalIter( } i.initializeBoundBufs(i.opts.LowerBound, i.opts.UpperBound) - if err := i.constructPointIter(i.opts.CategoryAndQoS, memtables, buf); err != nil { + if err := i.constructPointIter(i.opts.category, memtables, buf); err != nil { return nil, err } @@ -1411,7 +1408,7 @@ func (i *Iterator) constructPointIter( if collector := i.tc.dbOpts.sstStatsCollector; collector != nil { internalOpts.iterStatsAccumulator = collector.Accumulator( uint64(uintptr(unsafe.Pointer(i))), - i.opts.CategoryAndQoS, + i.opts.Category, ) } if i.opts.RangeKeyMasking.Filter != nil { diff --git a/get_iter_test.go b/get_iter_test.go index 68be1e64ef0..271190cabbc 100644 --- a/get_iter_test.go +++ b/get_iter_test.go @@ -13,7 +13,6 @@ import ( "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/manifest" "github.com/cockroachdb/pebble/internal/testkeys" - "github.com/cockroachdb/pebble/sstable" ) func TestGetIter(t *testing.T) { @@ -455,10 +454,7 @@ func TestGetIter(t *testing.T) { get.version = v get.snapshot = ikey.SeqNum() + 1 get.iterOpts = IterOptions{ - CategoryAndQoS: sstable.CategoryAndQoS{ - Category: "pebble-get", - QoSLevel: sstable.LatencySensitiveQoSLevel, - }, + Category: categoryGet, logger: testLogger{t}, snapshotForHideObsoletePoints: get.snapshot, } diff --git a/ingest.go b/ingest.go index 3fe0aae7275..81d8baffa16 100644 --- a/ingest.go +++ b/ingest.go @@ -1752,11 +1752,8 @@ func (d *DB) excise( } var err error iters, err = d.newIters(ctx, m, &IterOptions{ - CategoryAndQoS: sstable.CategoryAndQoS{ - Category: "pebble-ingest", - QoSLevel: sstable.LatencySensitiveQoSLevel, - }, - layer: manifest.Level(level), + Category: categoryIngest, + layer: manifest.Level(level), }, internalIterOpts{}, iterPointKeys|iterRangeDeletions|iterRangeKeys) itersLoaded = true return err @@ -2154,11 +2151,8 @@ func (d *DB) ingestApply( comparer: d.opts.Comparer, newIters: d.newIters, opts: IterOptions{ - logger: d.opts.Logger, - CategoryAndQoS: sstable.CategoryAndQoS{ - Category: "pebble-ingest", - QoSLevel: sstable.LatencySensitiveQoSLevel, - }, + logger: d.opts.Logger, + Category: categoryIngest, }, v: current, } diff --git a/ingest_test.go b/ingest_test.go index e8f5da5032a..eded55aa632 100644 --- a/ingest_test.go +++ b/ingest_test.go @@ -1133,7 +1133,7 @@ func testIngestSharedImpl( w := sstable.NewRawWriter(objstorageprovider.NewFileWritable(f), writeOpts) var sharedSSTs []SharedSSTMeta - err = from.ScanInternal(context.TODO(), sstable.CategoryAndQoS{}, startKey, endKey, + err = from.ScanInternal(context.TODO(), sstable.CategoryUnknown, startKey, endKey, func(key *InternalKey, value LazyValue, _ IteratorLevel) error { val, _, err := value.Value(nil) require.NoError(t, err) @@ -1634,7 +1634,7 @@ func TestConcurrentExcise(t *testing.T) { w := sstable.NewRawWriter(objstorageprovider.NewFileWritable(f), writeOpts) var sharedSSTs []SharedSSTMeta - err = from.ScanInternal(context.TODO(), sstable.CategoryAndQoS{}, startKey, endKey, + err = from.ScanInternal(context.TODO(), sstable.CategoryUnknown, startKey, endKey, func(key *InternalKey, value LazyValue, _ IteratorLevel) error { val, _, err := value.Value(nil) require.NoError(t, err) @@ -2071,7 +2071,7 @@ func TestIngestExternal(t *testing.T) { w := sstable.NewRawWriter(objstorageprovider.NewFileWritable(f), writeOpts) var externalFiles []ExternalFile - err = from.ScanInternal(context.TODO(), sstable.CategoryAndQoS{}, startKey, endKey, + err = from.ScanInternal(context.TODO(), sstable.CategoryUnknown, startKey, endKey, func(key *InternalKey, value LazyValue, _ IteratorLevel) error { val, _, err := value.Value(nil) require.NoError(t, err) @@ -2356,11 +2356,8 @@ func TestIngestTargetLevel(t *testing.T) { comparer: d.opts.Comparer, newIters: d.newIters, opts: IterOptions{ - logger: d.opts.Logger, - CategoryAndQoS: sstable.CategoryAndQoS{ - Category: "pebble-ingest", - QoSLevel: sstable.LatencySensitiveQoSLevel, - }, + logger: d.opts.Logger, + Category: categoryIngest, }, v: d.mu.versions.currentVersion(), } diff --git a/level_iter.go b/level_iter.go index 7f4b6f853c0..168ce1325c3 100644 --- a/level_iter.go +++ b/level_iter.go @@ -169,7 +169,7 @@ func (l *levelIter) init( l.tableOpts.PointKeyFilters = l.filtersBuf[:0:1] } l.tableOpts.UseL6Filters = opts.UseL6Filters - l.tableOpts.CategoryAndQoS = opts.CategoryAndQoS + l.tableOpts.Category = opts.Category l.tableOpts.layer = l.layer l.tableOpts.snapshotForHideObsoletePoints = opts.snapshotForHideObsoletePoints l.comparer = comparer diff --git a/metamorphic/ops.go b/metamorphic/ops.go index 3d3438ffdbf..ac940f52923 100644 --- a/metamorphic/ops.go +++ b/metamorphic/ops.go @@ -1914,7 +1914,7 @@ func (r *replicateOp) runSharedReplicate( ) { var sharedSSTs []pebble.SharedSSTMeta var err error - err = source.ScanInternal(context.TODO(), sstable.CategoryAndQoS{}, r.start, r.end, + err = source.ScanInternal(context.TODO(), sstable.CategoryUnknown, r.start, r.end, func(key *pebble.InternalKey, value pebble.LazyValue, _ pebble.IteratorLevel) error { val, _, err := value.Value(nil) if err != nil { @@ -1977,7 +1977,7 @@ func (r *replicateOp) runExternalReplicate( ) { var externalSSTs []pebble.ExternalFile var err error - err = source.ScanInternal(context.TODO(), sstable.CategoryAndQoS{}, r.start, r.end, + err = source.ScanInternal(context.TODO(), sstable.CategoryUnknown, r.start, r.end, func(key *pebble.InternalKey, value pebble.LazyValue, _ pebble.IteratorLevel) error { val, _, err := value.Value(nil) if err != nil { diff --git a/metrics.go b/metrics.go index a6fe1bf7143..60b2dd5e5fc 100644 --- a/metrics.go +++ b/metrics.go @@ -147,6 +147,10 @@ func (m *LevelMetrics) WriteAmp() float64 { return float64(m.BytesFlushed+m.BytesCompacted) / float64(m.BytesIn) } +var categoryCompaction = sstable.RegisterCategory("pebble-compaction", sstable.NonLatencySensitiveQoSLevel) +var categoryIngest = sstable.RegisterCategory("pebble-ingest", sstable.LatencySensitiveQoSLevel) +var categoryGet = sstable.RegisterCategory("pebble-get", sstable.LatencySensitiveQoSLevel) + // Metrics holds metrics for various subsystems of the DB such as the Cache, // Compactions, WAL, and per-Level metrics. // diff --git a/metrics_test.go b/metrics_test.go index cb55f58bfc2..1657be234f5 100644 --- a/metrics_test.go +++ b/metrics_test.go @@ -107,6 +107,13 @@ func exampleMetrics() Metrics { return m } +func init() { + // Register some categories for the purposes of the test. + sstable.RegisterCategory("a", sstable.NonLatencySensitiveQoSLevel) + sstable.RegisterCategory("b", sstable.LatencySensitiveQoSLevel) + sstable.RegisterCategory("c", sstable.NonLatencySensitiveQoSLevel) +} + func TestMetrics(t *testing.T) { if runtime.GOARCH == "386" { t.Skip("skipped on 32-bit due to slightly varied output") @@ -306,18 +313,13 @@ func TestMetrics(t *testing.T) { return err.Error() } } - var categoryAndQoS sstable.CategoryAndQoS + category := sstable.CategoryUnknown if td.HasArg("category") { var s string td.ScanArgs(t, "category", &s) - categoryAndQoS.Category = sstable.Category(s) - } - if td.HasArg("qos") { - var qos string - td.ScanArgs(t, "qos", &qos) - categoryAndQoS.QoSLevel = sstable.StringToQoSForTesting(qos) + category = sstable.StringToCategoryForTesting(s) } - iter, _ := d.NewIter(&IterOptions{CategoryAndQoS: categoryAndQoS}) + iter, _ := d.NewIter(&IterOptions{Category: category}) // Some iterators (eg. levelIter) do not instantiate the underlying // iterator until the first positioning call. Position the iterator // so that levelIters will have loaded an sstable. @@ -342,7 +344,7 @@ func TestMetrics(t *testing.T) { m.TableCache = cache.Metrics{} m.BlockCache = cache.Metrics{} // Empirically, the unknown stats are also non-deterministic. - if len(m.CategoryStats) > 0 && m.CategoryStats[0].Category == "_unknown" { + if len(m.CategoryStats) > 0 && m.CategoryStats[0].Category == sstable.CategoryUnknown { m.CategoryStats[0].CategoryStats = sstable.CategoryStats{} } } @@ -352,7 +354,7 @@ func TestMetrics(t *testing.T) { fmt.Fprintf(&buf, "Iter category stats:\n") for _, stats := range m.CategoryStats { fmt.Fprintf(&buf, "%20s, %11s: %+v\n", stats.Category, - redact.StringWithoutMarkers(stats.QoSLevel), stats.CategoryStats) + redact.StringWithoutMarkers(stats.Category.QoSLevel()), stats.CategoryStats) } } return buf.String() diff --git a/options.go b/options.go index 42c76596b32..e871e9ced7a 100644 --- a/options.go +++ b/options.go @@ -188,9 +188,9 @@ type IterOptions struct { // existing is not low or if we just expect a one-time Seek (where loading the // data block directly is better). UseL6Filters bool - // CategoryAndQoS is used for categorized iterator stats. This should not be + // Category is used for categorized iterator stats. This should not be // changed by calling SetOptions. - sstable.CategoryAndQoS + Category sstable.Category DebugRangeKeyStack bool @@ -261,9 +261,10 @@ func (o *IterOptions) SpanIterOptions() keyspan.SpanIterOptions { // scanInternalOptions is similar to IterOptions, meant for use with // scanInternalIterator. type scanInternalOptions struct { - sstable.CategoryAndQoS IterOptions + category sstable.Category + visitPointKey func(key *InternalKey, value LazyValue, iterInfo IteratorLevel) error visitRangeDel func(start, end []byte, seqNum SeqNum) error visitRangeKey func(start, end []byte, keys []rangekey.Key) error diff --git a/scan_internal.go b/scan_internal.go index 810295c0d8d..a066359736b 100644 --- a/scan_internal.go +++ b/scan_internal.go @@ -830,7 +830,7 @@ func (opts *scanInternalOptions) skipLevelForOpts() int { // constructPointIter constructs a merging iterator and sets i.iter to it. func (i *scanInternalIterator) constructPointIter( - categoryAndQoS sstable.CategoryAndQoS, memtables flushableList, buf *iterAlloc, + category sstable.Category, memtables flushableList, buf *iterAlloc, ) error { // Merging levels and levels from iterAlloc. mlevels := buf.mlevels[:0] @@ -897,7 +897,7 @@ func (i *scanInternalIterator) constructPointIter( levels = levels[:numLevelIters] rangeDelLevels = rangeDelLevels[:numLevelIters] i.opts.IterOptions.snapshotForHideObsoletePoints = i.seqNum - i.opts.IterOptions.CategoryAndQoS = categoryAndQoS + i.opts.IterOptions.Category = category addLevelIterForFiles := func(files manifest.LevelIterator, level manifest.Layer) { li := &levels[levelsIndex] rli := &rangeDelLevels[levelsIndex] diff --git a/scan_internal_test.go b/scan_internal_test.go index e8e6255a9e3..5839742f55f 100644 --- a/scan_internal_test.go +++ b/scan_internal_test.go @@ -213,7 +213,7 @@ func TestScanInternal(t *testing.T) { type scanInternalReader interface { ScanInternal( ctx context.Context, - categoryAndQoS sstable.CategoryAndQoS, + category sstable.Category, lower, upper []byte, visitPointKey func(key *InternalKey, value LazyValue, iterInfo IteratorLevel) error, visitRangeDel func(start, end []byte, seqNum base.SeqNum) error, @@ -561,7 +561,7 @@ func TestScanInternal(t *testing.T) { } } } - err := reader.ScanInternal(context.TODO(), sstable.CategoryAndQoS{}, lower, upper, + err := reader.ScanInternal(context.TODO(), sstable.CategoryUnknown, lower, upper, func(key *InternalKey, value LazyValue, _ IteratorLevel) error { v := value.InPlaceValue() fmt.Fprintf(&b, "%s (%s)\n", key, v) diff --git a/snapshot.go b/snapshot.go index 1be35931c47..24a75b8038c 100644 --- a/snapshot.go +++ b/snapshot.go @@ -75,7 +75,7 @@ func (s *Snapshot) NewIterWithContext(ctx context.Context, o *IterOptions) (*Ite // point keys deleted by range dels and keys masked by range keys. func (s *Snapshot) ScanInternal( ctx context.Context, - categoryAndQoS sstable.CategoryAndQoS, + category sstable.Category, lower, upper []byte, visitPointKey func(key *InternalKey, value LazyValue, iterInfo IteratorLevel) error, visitRangeDel func(start, end []byte, seqNum base.SeqNum) error, @@ -87,7 +87,7 @@ func (s *Snapshot) ScanInternal( panic(ErrClosed) } scanInternalOpts := &scanInternalOptions{ - CategoryAndQoS: categoryAndQoS, + category: category, visitPointKey: visitPointKey, visitRangeDel: visitRangeDel, visitRangeKey: visitRangeKey, @@ -473,7 +473,7 @@ func (es *EventuallyFileOnlySnapshot) NewIterWithContext( // point keys deleted by range dels and keys masked by range keys. func (es *EventuallyFileOnlySnapshot) ScanInternal( ctx context.Context, - categoryAndQoS sstable.CategoryAndQoS, + category sstable.Category, lower, upper []byte, visitPointKey func(key *InternalKey, value LazyValue, iterInfo IteratorLevel) error, visitRangeDel func(start, end []byte, seqNum base.SeqNum) error, @@ -486,7 +486,7 @@ func (es *EventuallyFileOnlySnapshot) ScanInternal( } var sOpts snapshotIterOpts opts := &scanInternalOptions{ - CategoryAndQoS: categoryAndQoS, + category: category, IterOptions: IterOptions{ KeyTypes: IterKeyTypePointsAndRanges, LowerBound: lower, diff --git a/sstable/category_stats.go b/sstable/category_stats.go index 2db32ca61e8..79d84e8ae74 100644 --- a/sstable/category_stats.go +++ b/sstable/category_stats.go @@ -8,6 +8,7 @@ import ( "cmp" "slices" "sync" + "sync/atomic" "time" "unsafe" @@ -21,13 +22,89 @@ import ( // // Examples of categories that can be useful in the CockroachDB context are: // sql-user, sql-stats, raft, rangefeed, mvcc-gc, range-snapshot. -type Category string +type Category uint8 + +// CategoryUnknown is the unknown category. It has the latency-sensitive QoS +// level. +const CategoryUnknown Category = 0 + +// CategoryMax is the maximum value of a category, and is also the maximum +// number of categories that can be registered. +const CategoryMax = 30 + +func (c Category) String() string { + return categories[c].name +} + +// QoSLevel returns the QoSLevel associated with this Category. +func (c Category) QoSLevel() QoSLevel { + return categories[c].qosLevel +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (c Category) SafeFormat(p redact.SafePrinter, verb rune) { + p.SafeString(redact.SafeString(c.String())) +} + +// RegisterCategory registers a new category. Each category has a name and an +// associated QoS level. The category name must be unique. +// +// Only CategoryMax categories can be registered in total. +func RegisterCategory(name string, qosLevel QoSLevel) Category { + if categoriesList != nil { + panic("ReigsterCategory called after Categories()") + } + c := Category(numRegisteredCategories.Add(1)) + if c > CategoryMax { + panic("too many categories") + } + categories[c].name = name + categories[c].qosLevel = qosLevel + return c +} + +// Categories returns all registered categories, including CategoryUnknown. +// +// Can only be called after all categories have been registered. Calling +// RegisterCategory() after Categories() will result in a panic. +func Categories() []Category { + categoriesListOnce.Do(func() { + categoriesList = make([]Category, numRegisteredCategories.Load()+1) + for i := range categoriesList { + categoriesList[i] = Category(i) + } + }) + return categoriesList +} + +var categories = [CategoryMax + 1]struct { + name string + qosLevel QoSLevel +}{ + CategoryUnknown: {name: "", qosLevel: LatencySensitiveQoSLevel}, +} + +var numRegisteredCategories atomic.Uint32 + +var categoriesList []Category +var categoriesListOnce sync.Once + +// StringToCategoryForTesting returns the Category for the string, or panics if +// the string is not known. +func StringToCategoryForTesting(s string) Category { + for i := range categories { + if categories[i].name == s { + return Category(i) + } + } + panic(errors.AssertionFailedf("unknown Category %s", s)) +} // QoSLevel describes whether the read is latency-sensitive or not. Each // category must map to a single QoSLevel. While category strings are opaque // to Pebble, the QoSLevel may be internally utilized in Pebble to better // optimize future reads. -type QoSLevel int +type QoSLevel uint8 const ( // LatencySensitiveQoSLevel is the default when QoSLevel is not specified, @@ -62,12 +139,6 @@ func StringToQoSForTesting(s string) QoSLevel { panic(errors.AssertionFailedf("unknown QoS %s", s)) } -// CategoryAndQoS specifies both the Category and the QoSLevel. -type CategoryAndQoS struct { - Category Category - QoSLevel QoSLevel -} - // CategoryStats provides stats about a category of reads. type CategoryStats struct { // BlockBytes is the bytes in the loaded blocks. If the block was @@ -93,7 +164,6 @@ func (s *CategoryStats) aggregate(a CategoryStats) { // CategoryStatsAggregate is the aggregate for the given category. type CategoryStatsAggregate struct { Category Category - QoSLevel QoSLevel CategoryStats CategoryStats } @@ -125,7 +195,6 @@ type CategoryStatsCollector struct { // contention on the category stats mutex has been observed. type shardedCategoryStats struct { Category Category - QoSLevel QoSLevel shards [numCategoryStatsShards]struct { categoryStatsWithMu // Pad each shard to 64 bytes so they don't share a cache line. @@ -138,7 +207,6 @@ type shardedCategoryStats struct { func (s *shardedCategoryStats) getStats() CategoryStatsAggregate { agg := CategoryStatsAggregate{ Category: s.Category, - QoSLevel: s.QoSLevel, } for i := range s.shards { s.shards[i].mu.Lock() @@ -150,13 +218,12 @@ func (s *shardedCategoryStats) getStats() CategoryStatsAggregate { // Accumulator returns a stats accumulator for the given category. The provided // p is used to detrmine which shard to write stats to. -func (c *CategoryStatsCollector) Accumulator(p uint64, caq CategoryAndQoS) IterStatsAccumulator { - v, ok := c.statsMap.Load(caq.Category) +func (c *CategoryStatsCollector) Accumulator(p uint64, category Category) IterStatsAccumulator { + v, ok := c.statsMap.Load(category) if !ok { c.mu.Lock() - v, _ = c.statsMap.LoadOrStore(caq.Category, &shardedCategoryStats{ - Category: caq.Category, - QoSLevel: caq.QoSLevel, + v, _ = c.statsMap.LoadOrStore(category, &shardedCategoryStats{ + Category: category, }) c.mu.Unlock() } @@ -172,9 +239,6 @@ func (c *CategoryStatsCollector) GetStats() []CategoryStatsAggregate { var stats []CategoryStatsAggregate c.statsMap.Range(func(_, v any) bool { s := v.(*shardedCategoryStats).getStats() - if len(s.Category) == 0 { - s.Category = "_unknown" - } stats = append(stats, s) return true }) diff --git a/table_cache.go b/table_cache.go index d0d4e361d9b..d9cfd8a3fce 100644 --- a/table_cache.go +++ b/table_cache.go @@ -574,7 +574,7 @@ func (c *tableCacheShard) newPointIter( iterStatsAccum := internalOpts.iterStatsAccumulator if iterStatsAccum == nil && opts != nil && dbOpts.sstStatsCollector != nil { iterStatsAccum = dbOpts.sstStatsCollector.Accumulator( - uint64(uintptr(unsafe.Pointer(v.reader))), opts.CategoryAndQoS) + uint64(uintptr(unsafe.Pointer(v.reader))), opts.Category) } if internalOpts.compaction { iter, err = cr.NewCompactionIter(transforms, iterStatsAccum, rp, internalOpts.bufferPool) diff --git a/testdata/metrics b/testdata/metrics index 360a8a56d6f..4d031971543 100644 --- a/testdata/metrics +++ b/testdata/metrics @@ -38,7 +38,7 @@ batch set a 1 ---- -iter-new a category=a qos=non-latency +iter-new a category=a ---- flush @@ -48,7 +48,7 @@ L0.0: # iter b references both a memtable and sstable 5. -iter-new b category=b qos=latency +iter-new b category=b ---- metrics @@ -103,7 +103,7 @@ L0.0: # iter c references both a memtable and sstables 5 and 7. -iter-new c category=c qos=non-latency +iter-new c category=c ---- compact a-z @@ -144,10 +144,10 @@ Filter utility: 0.0% Ingestions: 0 as flushable: 0 (0B in 0 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} c, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} disk-usage ---- @@ -191,10 +191,10 @@ Filter utility: 0.0% Ingestions: 0 as flushable: 0 (0B in 0 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} c, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} # Closing iter c will release one of the zombie sstables. The other # zombie sstable is still referenced by iter b. @@ -235,10 +235,10 @@ Filter utility: 0.0% Ingestions: 0 as flushable: 0 (0B in 0 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} c, non-latency: {BlockBytes:44 BlockBytesInCache:44 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} disk-usage ---- @@ -282,10 +282,10 @@ Filter utility: 0.0% Ingestions: 0 as flushable: 0 (0B in 0 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} c, non-latency: {BlockBytes:44 BlockBytesInCache:44 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} disk-usage ---- @@ -355,10 +355,10 @@ Filter utility: 0.0% Ingestions: 0 as flushable: 0 (0B in 0 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} c, non-latency: {BlockBytes:44 BlockBytesInCache:44 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:88 BlockBytesInCache:44 BlockReadDuration:10ms} additional-metrics ---- @@ -412,10 +412,10 @@ Filter utility: 0.0% Ingestions: 0 as flushable: 0 (0B in 0 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:301 BlockBytesInCache:44 BlockReadDuration:60ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} c, non-latency: {BlockBytes:44 BlockBytesInCache:44 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:301 BlockBytesInCache:44 BlockReadDuration:60ms} additional-metrics ---- @@ -518,11 +518,11 @@ Filter utility: 0.0% Ingestions: 2 as flushable: 2 (1.7KB in 3 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:301 BlockBytesInCache:44 BlockReadDuration:60ms} + pebble-ingest, latency: {BlockBytes:64 BlockBytesInCache:0 BlockReadDuration:10ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} c, non-latency: {BlockBytes:44 BlockBytesInCache:44 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:301 BlockBytesInCache:44 BlockReadDuration:60ms} - pebble-ingest, latency: {BlockBytes:64 BlockBytesInCache:0 BlockReadDuration:10ms} batch set g g @@ -583,11 +583,11 @@ Filter utility: 0.0% Ingestions: 2 as flushable: 2 (1.7KB in 3 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:301 BlockBytesInCache:44 BlockReadDuration:60ms} + pebble-ingest, latency: {BlockBytes:64 BlockBytesInCache:0 BlockReadDuration:10ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} c, non-latency: {BlockBytes:44 BlockBytesInCache:44 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:301 BlockBytesInCache:44 BlockReadDuration:60ms} - pebble-ingest, latency: {BlockBytes:64 BlockBytesInCache:0 BlockReadDuration:10ms} build ext1 set z z @@ -662,11 +662,11 @@ Filter utility: 0.0% Ingestions: 3 as flushable: 2 (1.7KB in 3 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:301 BlockBytesInCache:44 BlockReadDuration:60ms} + pebble-ingest, latency: {BlockBytes:200 BlockBytesInCache:0 BlockReadDuration:30ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} c, non-latency: {BlockBytes:44 BlockBytesInCache:44 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:301 BlockBytesInCache:44 BlockReadDuration:60ms} - pebble-ingest, latency: {BlockBytes:200 BlockBytesInCache:0 BlockReadDuration:30ms} # Virtualize a virtual sstable. build ext1 @@ -766,11 +766,11 @@ Filter utility: 0.0% Ingestions: 4 as flushable: 2 (1.7KB in 3 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: + pebble-compaction, non-latency: {BlockBytes:677 BlockBytesInCache:376 BlockReadDuration:70ms} + pebble-ingest, latency: {BlockBytes:272 BlockBytesInCache:72 BlockReadDuration:30ms} a, non-latency: {BlockBytes:0 BlockBytesInCache:0 BlockReadDuration:0s} b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} c, non-latency: {BlockBytes:44 BlockBytesInCache:44 BlockReadDuration:0s} - pebble-compaction, non-latency: {BlockBytes:677 BlockBytesInCache:376 BlockReadDuration:70ms} - pebble-ingest, latency: {BlockBytes:272 BlockBytesInCache:72 BlockReadDuration:30ms} # Create a DB where lower levels are written as shared tables. All ingests also # become shared tables.