diff --git a/batch.go b/batch.go index ad419f147e..29686a6f70 100644 --- a/batch.go +++ b/batch.go @@ -2005,6 +2005,13 @@ func (b *flushableBatch) readyForFlush() bool { return true } +// computePossibleOverlaps is part of the flushable interface. +func (b *flushableBatch) computePossibleOverlaps( + fn func(bounded) shouldContinue, bounded ...bounded, +) { + computePossibleOverlapsGenericImpl[*flushableBatch](b, b.cmp, fn, bounded) +} + // Note: flushableBatchIter mirrors the implementation of batchIter. Keep the // two in sync. type flushableBatchIter struct { diff --git a/db.go b/db.go index 680bea5a25..ba261c39c5 100644 --- a/db.go +++ b/db.go @@ -1578,19 +1578,12 @@ func (d *DB) NewEventuallyFileOnlySnapshot(keyRanges []KeyRange) *EventuallyFile if err := d.closed.Load(); err != nil { panic(err) } - - internalKeyRanges := make([]internalKeyRange, len(keyRanges)) for i := range keyRanges { if i > 0 && d.cmp(keyRanges[i-1].End, keyRanges[i].Start) > 0 { panic("pebble: key ranges for eventually-file-only-snapshot not in order") } - internalKeyRanges[i] = internalKeyRange{ - smallest: base.MakeInternalKey(keyRanges[i].Start, InternalKeySeqNumMax, InternalKeyKindMax), - largest: base.MakeExclusiveSentinelKey(InternalKeyKindRangeDelete, keyRanges[i].End), - } } - - return d.makeEventuallyFileOnlySnapshot(keyRanges, internalKeyRanges) + return d.makeEventuallyFileOnlySnapshot(keyRanges) } // Close closes the DB. @@ -1748,25 +1741,17 @@ func (d *DB) Compact(start, end []byte, parallelize bool) error { return errors.Errorf("Compact start %s is not less than end %s", d.opts.Comparer.FormatKey(start), d.opts.Comparer.FormatKey(end)) } - iStart := base.MakeInternalKey(start, InternalKeySeqNumMax, InternalKeyKindMax) - iEnd := base.MakeInternalKey(end, 0, 0) - m := (&fileMetadata{}).ExtendPointKeyBounds(d.cmp, iStart, iEnd) - meta := []*fileMetadata{m} d.mu.Lock() maxLevelWithFiles := 1 cur := d.mu.versions.currentVersion() for level := 0; level < numLevels; level++ { - overlaps := cur.Overlaps(level, d.cmp, start, end, iEnd.IsExclusiveSentinel()) + overlaps := cur.Overlaps(level, d.cmp, start, end, false) if !overlaps.Empty() { maxLevelWithFiles = level + 1 } } - keyRanges := make([]internalKeyRange, len(meta)) - for i := range meta { - keyRanges[i] = internalKeyRange{smallest: m.Smallest, largest: m.Largest} - } // Determine if any memtable overlaps with the compaction range. We wait for // any such overlap to flush (initiating a flush if necessary). mem, err := func() (*flushableEntry, error) { @@ -1776,25 +1761,31 @@ func (d *DB) Compact(start, end []byte, parallelize bool) error { // overlaps. for i := len(d.mu.mem.queue) - 1; i >= 0; i-- { mem := d.mu.mem.queue[i] - if ingestMemtableOverlaps(d.cmp, mem, keyRanges) { - var err error + var anyOverlaps bool + mem.computePossibleOverlaps(func(b bounded) shouldContinue { + anyOverlaps = true + return stopIteration + }, KeyRange{Start: start, End: end}) + if !anyOverlaps { + continue + } + var err error + if mem.flushable == d.mu.mem.mutable { + // We have to hold both commitPipeline.mu and DB.mu when calling + // makeRoomForWrite(). Lock order requirements elsewhere force us to + // unlock DB.mu in order to grab commitPipeline.mu first. + d.mu.Unlock() + d.commit.mu.Lock() + d.mu.Lock() + defer d.commit.mu.Unlock() if mem.flushable == d.mu.mem.mutable { - // We have to hold both commitPipeline.mu and DB.mu when calling - // makeRoomForWrite(). Lock order requirements elsewhere force us to - // unlock DB.mu in order to grab commitPipeline.mu first. - d.mu.Unlock() - d.commit.mu.Lock() - d.mu.Lock() - defer d.commit.mu.Unlock() - if mem.flushable == d.mu.mem.mutable { - // Only flush if the active memtable is unchanged. - err = d.makeRoomForWrite(nil) - } + // Only flush if the active memtable is unchanged. + err = d.makeRoomForWrite(nil) } - mem.flushForced = true - d.maybeScheduleFlush() - return mem, err } + mem.flushForced = true + d.maybeScheduleFlush() + return mem, err } return nil, nil }() @@ -1811,7 +1802,7 @@ func (d *DB) Compact(start, end []byte, parallelize bool) error { for level := 0; level < maxLevelWithFiles; { for { if err := d.manualCompact( - iStart.UserKey, iEnd.UserKey, level, parallelize); err != nil { + start, end, level, parallelize); err != nil { if errors.Is(err, ErrCancelledCompaction) { continue } diff --git a/flushable.go b/flushable.go index 473dc6adf5..bc380746b4 100644 --- a/flushable.go +++ b/flushable.go @@ -7,6 +7,7 @@ package pebble import ( "context" "fmt" + "io" "sync/atomic" "time" @@ -30,6 +31,35 @@ type flushable interface { // memTable.readyForFlush for one implementation which needs to check whether // there are any outstanding write references. readyForFlush() bool + // computePossibleOverlaps determines whether the flushable's keys overlap + // with the bounds of any of the provided bounded items. If an item overlaps + // or might overlap but it's not possible to determine overlap cheaply, + // computePossibleOverlaps invokes the provided function with the object + // that might overlap. computePossibleOverlaps must not perform any I/O and + // implementations should invoke the provided function for items that would + // require I/O to determine overlap. + computePossibleOverlaps(overlaps func(bounded) shouldContinue, bounded ...bounded) +} + +type shouldContinue bool + +const ( + continueIteration shouldContinue = true + stopIteration = false +) + +type bounded interface { + // InternalKeyBounds returns a start key and an end key. Both bounds are + // inclusive. + InternalKeyBounds() (InternalKey, InternalKey) +} + +func sliceAsBounded[B bounded](s []B) []bounded { + ret := make([]bounded, len(s)) + for i := 0; i < len(s); i++ { + ret[i] = s[i] + } + return ret } // flushableEntry wraps a flushable and adds additional metadata and @@ -252,3 +282,66 @@ func (s *ingestedFlushable) readyForFlush() bool { // determine where to place the files in the lsm. return true } + +// computePossibleOverlaps is part of the flushable interface. +func (s *ingestedFlushable) computePossibleOverlaps( + fn func(bounded) shouldContinue, bounded ...bounded, +) { + for i := range bounded { + smallest, largest := bounded[i].InternalKeyBounds() + for j := 0; j < len(s.files); j++ { + if sstableKeyCompare(s.comparer.Compare, s.files[j].Largest, smallest) >= 0 { + // This file's largest key is larger than smallest. Either the + // file overlaps the bounds, or it lies strictly after the + // bounds. Either way we can stop iterating since the files are + // sorted. But first, determine if there's overlap and call fn + // if necessary. + if sstableKeyCompare(s.comparer.Compare, s.files[j].Smallest, largest) <= 0 { + // The file overlaps in key boundaries. The file doesn't necessarily + // contain any keys within the key range, but we would need to + // perform I/O to know for sure. The flushable interface dictates + // that we're not permitted to perform I/O here, so err towards + // assuming overlap. + if !fn(bounded[i]) { + return + } + } + break + } + } + } +} + +// computePossibleOverlapsGenericImpl is an implemention of the flushable +// interface's computePossibleOverlaps function for flushable implementations +// with only in-memory state that do not have special requirements and should +// read through the ordinary flushable iterators. +// +// This function must only be used with implementations that are infallible (eg, +// memtable iterators) and will panic if an error is encountered. +func computePossibleOverlapsGenericImpl[F flushable]( + f F, cmp Compare, fn func(bounded) shouldContinue, bounded []bounded, +) { + iter := f.newIter(nil) + rangeDelIter := f.newRangeDelIter(nil) + rkeyIter := f.newRangeKeyIter(nil) + for _, b := range bounded { + s, l := b.InternalKeyBounds() + kr := internalKeyRange{s, l} + if overlapWithIterator(iter, &rangeDelIter, rkeyIter, kr, cmp) { + if !fn(b) { + break + } + } + } + + for _, c := range [3]io.Closer{iter, rangeDelIter, rkeyIter} { + if c != nil { + if err := c.Close(); err != nil { + // This implementation must be used in circumstances where + // reading through the iterator is infallible. + panic(err) + } + } + } +} diff --git a/ingest.go b/ingest.go index 097d71ec74..c5680e00a2 100644 --- a/ingest.go +++ b/ingest.go @@ -53,6 +53,13 @@ func (k *KeyRange) Contains(cmp base.Compare, key InternalKey) bool { return (v < 0 || (v == 0 && key.IsExclusiveSentinel())) && cmp(k.Start, key.UserKey) <= 0 } +// InternalKeyBounds returns the key range as internal key bounds, with the end +// boundary represented as an exclusive range delete sentinel key. +func (k KeyRange) InternalKeyBounds() (InternalKey, InternalKey) { + return base.MakeInternalKey(k.Start, InternalKeySeqNumMax, InternalKeyKindMax), + base.MakeExclusiveSentinelKey(InternalKeyKindRangeDelete, k.End) +} + // OverlapsInternalKeyRange checks if the specified internal key range has an // overlap with the KeyRange. Note that we aren't checking for full containment // of smallest-largest within k, rather just that there's some intersection @@ -625,33 +632,6 @@ func ingestLink( return nil } -func ingestMemtableOverlaps(cmp Compare, mem flushable, keyRanges []internalKeyRange) bool { - iter := mem.newIter(nil) - rangeDelIter := mem.newRangeDelIter(nil) - rkeyIter := mem.newRangeKeyIter(nil) - - closeIters := func() error { - err := iter.Close() - if rangeDelIter != nil { - err = firstError(err, rangeDelIter.Close()) - } - if rkeyIter != nil { - err = firstError(err, rkeyIter.Close()) - } - return err - } - - for _, kr := range keyRanges { - if overlapWithIterator(iter, &rangeDelIter, rkeyIter, kr, cmp) { - closeIters() - return true - } - } - - // Assume overlap if any iterator errored out. - return closeIters() != nil -} - func ingestUpdateSeqNum( cmp Compare, format base.FormatKey, seqNum uint64, loadResult ingestLoadResult, ) error { @@ -1369,23 +1349,30 @@ func (d *DB) ingest( return IngestOperationStats{}, err } - // metaFlushableOverlaps is a slice parallel to meta indicating which of the - // ingested sstables overlap some table in the flushable queue. It's used to - // approximate ingest-into-L0 stats when using flushable ingests. - metaFlushableOverlaps := make([]bool, loadResult.fileCount) + // metaFlushableOverlaps is a map indicating which of the ingested sstables + // overlap some table in the flushable queue. It's used to approximate + // ingest-into-L0 stats when using flushable ingests. + metaFlushableOverlaps := make(map[FileNum]bool, loadResult.fileCount) var mem *flushableEntry var mut *memTable // asFlushable indicates whether the sstable was ingested as a flushable. var asFlushable bool - iterOps := IterOptions{ - CategoryAndQoS: sstable.CategoryAndQoS{ - Category: "pebble-ingest", - QoSLevel: sstable.LatencySensitiveQoSLevel, - }, - } prepare := func(seqNum uint64) { // Note that d.commit.mu is held by commitPipeline when calling prepare. + // Determine the set of bounds we care about for the purpose of checking + // for overlap among the flushables. If there's an excise span, we need + // to check for overlap with its bounds as well. + overlapBounds := make([]bounded, 0, loadResult.fileCount+1) + for _, metas := range [3][]*fileMetadata{loadResult.localMeta, loadResult.sharedMeta, loadResult.externalMeta} { + for _, m := range metas { + overlapBounds = append(overlapBounds, m) + } + } + if exciseSpan.Valid() { + overlapBounds = append(overlapBounds, &exciseSpan) + } + d.mu.Lock() defer d.mu.Unlock() @@ -1396,59 +1383,32 @@ func (d *DB) ingest( for i := len(d.mu.mem.queue) - 1; i >= 0; i-- { m := d.mu.mem.queue[i] - iter := m.newIter(&iterOps) - rangeDelIter := m.newRangeDelIter(&iterOps) - rkeyIter := m.newRangeKeyIter(&iterOps) - - checkForOverlap := func(i int, meta *fileMetadata) { - if metaFlushableOverlaps[i] { - // This table already overlapped a more recent flushable. - return - } - kr := internalKeyRange{ - smallest: meta.Smallest, - largest: meta.Largest, - } - if overlapWithIterator(iter, &rangeDelIter, rkeyIter, kr, d.cmp) { - // If this is the first table to overlap a flushable, save - // the flushable. This ingest must be ingested or flushed - // after it. - if mem == nil { - mem = m - } - metaFlushableOverlaps[i] = true - } - } - for i := range loadResult.localMeta { - checkForOverlap(i, loadResult.localMeta[i]) - } - for i := range loadResult.sharedMeta { - checkForOverlap(len(loadResult.localMeta)+i, loadResult.sharedMeta[i]) - } - for i := range loadResult.externalMeta { - checkForOverlap(len(loadResult.localMeta)+len(loadResult.sharedMeta)+i, loadResult.externalMeta[i]) - } - if exciseSpan.Valid() { - kr := internalKeyRange{ - smallest: base.MakeInternalKey(exciseSpan.Start, InternalKeySeqNumMax, InternalKeyKindMax), - largest: base.MakeExclusiveSentinelKey(InternalKeyKindRangeDelete, exciseSpan.End), + m.computePossibleOverlaps(func(b bounded) shouldContinue { + // If this is the first table to overlap a flushable, save + // the flushable. This ingest must be ingested or flushed + // after it. + if mem == nil { + mem = m } - if overlapWithIterator(iter, &rangeDelIter, rkeyIter, kr, d.cmp) { - if mem == nil { - mem = m - } + + switch v := b.(type) { + case *fileMetadata: + // NB: False positives are possible if `m` is a flushable + // ingest that overlaps the file `v` in bounds but doesn't + // contain overlapping data. This is considered acceptable + // because it's rare (in CockroachDB a bound overlap likely + // indicates a data overlap), and blocking the commit + // pipeline while we perform I/O to check for overlap may be + // more disruptive than enqueueing this ingestion on the + // flushable queue and switching to a new memtable. + metaFlushableOverlaps[v.FileNum] = true + case *KeyRange: + // An excise span; not a file. + default: + panic("unreachable") } - } - err := iter.Close() - if rangeDelIter != nil { - err = firstError(err, rangeDelIter.Close()) - } - if rkeyIter != nil { - err = firstError(err, rkeyIter.Close()) - } - if err != nil { - d.opts.Logger.Errorf("ingest error reading flushable for log %s: %s", m.logNum, err) - } + return continueIteration + }, overlapBounds...) } if mem == nil { @@ -1592,7 +1552,7 @@ func (d *DB) ingest( if e.Level == 0 { stats.ApproxIngestedIntoL0Bytes += e.Meta.Size } - if i < len(metaFlushableOverlaps) && metaFlushableOverlaps[i] { + if metaFlushableOverlaps[e.Meta.FileNum] { stats.MemtableOverlappingFiles++ } } @@ -1614,7 +1574,7 @@ func (d *DB) ingest( // before entering the commit pipeline, we can use that overlap to // improve our approximation by incorporating overlap with L0, not // just memtables. - if metaFlushableOverlaps[i] { + if metaFlushableOverlaps[f.FileNum] { stats.ApproxIngestedIntoL0Bytes += f.Size stats.MemtableOverlappingFiles++ } diff --git a/ingest_test.go b/ingest_test.go index 40fcf7aec4..77d1dae2ed 100644 --- a/ingest_test.go +++ b/ingest_test.go @@ -1912,12 +1912,17 @@ func TestIngestMemtableOverlaps(t *testing.T) { case "overlaps": var buf bytes.Buffer for _, data := range strings.Split(d.Input, "\n") { - var keyRanges []internalKeyRange + var keyRanges []bounded for _, part := range strings.Fields(data) { meta := parseMeta(part) - keyRanges = append(keyRanges, internalKeyRange{smallest: meta.Smallest, largest: meta.Largest}) + keyRanges = append(keyRanges, meta) } - fmt.Fprintf(&buf, "%t\n", ingestMemtableOverlaps(mem.cmp, mem, keyRanges)) + var overlaps bool + mem.computePossibleOverlaps(func(bounded) shouldContinue { + overlaps = true + return stopIteration + }, keyRanges...) + fmt.Fprintf(&buf, "%t\n", overlaps) } return buf.String() @@ -2563,7 +2568,6 @@ func TestIngestFlushQueuedMemTable(t *testing.T) { } ingest := func(keys ...string) { - t.Helper() f, err := mem.Create("ext") require.NoError(t, err) @@ -2577,7 +2581,7 @@ func TestIngestFlushQueuedMemTable(t *testing.T) { stats, err := d.IngestWithStats([]string{"ext"}) require.NoError(t, err) require.Equal(t, stats.ApproxIngestedIntoL0Bytes, stats.Bytes) - require.Equal(t, stats.MemtableOverlappingFiles, 1) + require.Equal(t, 1, stats.MemtableOverlappingFiles) require.Less(t, uint64(0), stats.Bytes) } diff --git a/internal/manifest/version.go b/internal/manifest/version.go index fe086d1aed..6a20828e7e 100644 --- a/internal/manifest/version.go +++ b/internal/manifest/version.go @@ -275,6 +275,11 @@ type FileMetadata struct { PrefixReplacement *PrefixReplacement } +// InternalKeyBounds returns the set of overall table bounds. +func (m *FileMetadata) InternalKeyBounds() (InternalKey, InternalKey) { + return m.Smallest, m.Largest +} + // PrefixReplacement represents a read-time replacement of a key prefix. type PrefixReplacement struct { ContentPrefix, SyntheticPrefix []byte diff --git a/mem_table.go b/mem_table.go index e728e94c71..405efe6b16 100644 --- a/mem_table.go +++ b/mem_table.go @@ -313,6 +313,11 @@ func (m *memTable) empty() bool { return m.skl.Size() == memTableEmptySize } +// computePossibleOverlaps is part of the flushable interface. +func (m *memTable) computePossibleOverlaps(fn func(bounded) shouldContinue, bounded ...bounded) { + computePossibleOverlapsGenericImpl[*memTable](m, m.cmp, fn, bounded) +} + // A keySpanFrags holds a set of fragmented keyspan.Spans with a particular key // kind at a particular moment for a memtable. // diff --git a/mem_table_test.go b/mem_table_test.go index 62ea9c60f2..ae555e5ec0 100644 --- a/mem_table_test.go +++ b/mem_table_test.go @@ -13,6 +13,7 @@ import ( "sync/atomic" "testing" "time" + "unicode" "github.com/cockroachdb/datadriven" "github.com/cockroachdb/errors" @@ -400,6 +401,70 @@ func TestMemTableReserved(t *testing.T) { require.Equal(t, int(m.reserved), int(b.memTableSize)+int(prevReserved)) } +func TestMemTable(t *testing.T) { + var m *memTable + var buf bytes.Buffer + batches := map[string]*Batch{} + + summary := func() string { + return fmt.Sprintf("%d of %d bytes available", + m.availBytes(), m.totalBytes()) + } + + datadriven.RunTest(t, "testdata/mem_table", func(t *testing.T, td *datadriven.TestData) string { + buf.Reset() + switch td.Cmd { + case "new": + var o memTableOptions + td.MaybeScanArgs(t, "size", &o.size) + m = newMemTable(o) + return "" + case "prepare": + var name string + td.ScanArgs(t, "name", &name) + b := newBatch(nil) + if err := runBatchDefineCmd(td, b); err != nil { + return err.Error() + } + batches[name] = b + if err := m.prepare(b); err != nil { + return err.Error() + } + return summary() + case "apply": + var name string + var seqNum uint64 + td.ScanArgs(t, "name", &name) + td.ScanArgs(t, "seq", &seqNum) + if err := m.apply(batches[name], seqNum); err != nil { + return err.Error() + } + delete(batches, name) + return summary() + case "computePossibleOverlaps": + stopAfterFirst := td.HasArg("stop-after-first") + + var keyRanges []bounded + for _, l := range strings.Split(td.Input, "\n") { + s := strings.FieldsFunc(l, func(r rune) bool { return unicode.IsSpace(r) || r == '-' }) + keyRanges = append(keyRanges, KeyRange{Start: []byte(s[0]), End: []byte(s[1])}) + } + + m.computePossibleOverlaps(func(b bounded) shouldContinue { + fmt.Fprintf(&buf, "%s\n", b) + if stopAfterFirst { + return stopIteration + } + return continueIteration + }, keyRanges...) + + return buf.String() + default: + return fmt.Sprintf("unrecognized command %q", td.Cmd) + } + }) +} + func buildMemTable(b *testing.B) (*memTable, [][]byte) { m := newMemTable(memTableOptions{}) var keys [][]byte diff --git a/snapshot.go b/snapshot.go index cdbf028d19..1714765a32 100644 --- a/snapshot.go +++ b/snapshot.go @@ -275,9 +275,7 @@ type EventuallyFileOnlySnapshot struct { closed chan struct{} } -func (d *DB) makeEventuallyFileOnlySnapshot( - keyRanges []KeyRange, internalKeyRanges []internalKeyRange, -) *EventuallyFileOnlySnapshot { +func (d *DB) makeEventuallyFileOnlySnapshot(keyRanges []KeyRange) *EventuallyFileOnlySnapshot { isFileOnly := true d.mu.Lock() @@ -285,11 +283,10 @@ func (d *DB) makeEventuallyFileOnlySnapshot( seqNum := d.mu.versions.visibleSeqNum.Load() // Check if any of the keyRanges overlap with a memtable. for i := range d.mu.mem.queue { - mem := d.mu.mem.queue[i] - if ingestMemtableOverlaps(d.cmp, mem, internalKeyRanges) { + d.mu.mem.queue[i].computePossibleOverlaps(func(bounded) shouldContinue { isFileOnly = false - break - } + return stopIteration + }, sliceAsBounded(keyRanges)...) } es := &EventuallyFileOnlySnapshot{ db: d, diff --git a/testdata/mem_table b/testdata/mem_table new file mode 100644 index 0000000000..4edf7234ce --- /dev/null +++ b/testdata/mem_table @@ -0,0 +1,35 @@ +new size=10000 +---- + +prepare name=batch1 +set foo foo +set bar bar +---- +8475 of 10000 bytes available + +apply name=batch1 seq=1 +---- +8475 of 10000 bytes available + +computePossibleOverlaps +a-f +a-bar +a-foo +bar-foo +foo-zoo +goo-zoo +---- +{a f} +{a foo} +{bar foo} +{foo zoo} + +computePossibleOverlaps stop-after-first +a-f +a-bar +a-foo +bar-foo +foo-zoo +goo-zoo +---- +{a f} diff --git a/testdata/metrics b/testdata/metrics index dda818b26b..e2ddd5ea10 100644 --- a/testdata/metrics +++ b/testdata/metrics @@ -466,15 +466,14 @@ MemTables: 1 (1.0MB) zombie: 1 (1.0MB) Zombie tables: 0 (0B) Backing tables: 0 (0B) Virtual tables: 0 (0B) -Block cache: 12 entries (2.4KB) hit rate: 31.1% -Table cache: 3 entries (2.4KB) hit rate: 57.9% +Block cache: 12 entries (2.4KB) hit rate: 24.5% +Table cache: 1 entries (808B) hit rate: 60.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 Filter utility: 0.0% Ingestions: 0 as flushable: 2 (2.1KB in 3 tables) Iter category stats: - _unknown, latency: {BlockBytes:88 BlockBytesInCache:88} b, latency: {BlockBytes:44 BlockBytesInCache:0} c, non-latency: {BlockBytes:44 BlockBytesInCache:44} pebble-compaction, non-latency: {BlockBytes:411 BlockBytesInCache:154} @@ -528,15 +527,14 @@ MemTables: 1 (1.0MB) zombie: 1 (1.0MB) Zombie tables: 0 (0B) Backing tables: 0 (0B) Virtual tables: 0 (0B) -Block cache: 12 entries (2.4KB) hit rate: 31.1% -Table cache: 3 entries (2.4KB) hit rate: 57.9% +Block cache: 12 entries (2.4KB) hit rate: 24.5% +Table cache: 1 entries (808B) hit rate: 60.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 Filter utility: 0.0% Ingestions: 0 as flushable: 2 (2.1KB in 3 tables) Iter category stats: - _unknown, latency: {BlockBytes:88 BlockBytesInCache:88} b, latency: {BlockBytes:44 BlockBytesInCache:0} c, non-latency: {BlockBytes:44 BlockBytesInCache:44} pebble-compaction, non-latency: {BlockBytes:411 BlockBytesInCache:154} @@ -605,14 +603,13 @@ Zombie tables: 0 (0B) Backing tables: 2 (1.3KB) Virtual tables: 2 (102B) Block cache: 21 entries (4.1KB) hit rate: 0.0% -Table cache: 5 entries (3.9KB) hit rate: 0.0% +Table cache: 3 entries (2.4KB) hit rate: 0.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 Filter utility: 0.0% Ingestions: 1 as flushable: 2 (2.1KB in 3 tables) Iter category stats: - _unknown, latency: {BlockBytes:0 BlockBytesInCache:0} b, latency: {BlockBytes:44 BlockBytesInCache:0} c, non-latency: {BlockBytes:44 BlockBytesInCache:44} pebble-compaction, non-latency: {BlockBytes:411 BlockBytesInCache:154} @@ -713,7 +710,6 @@ Table iters: 0 Filter utility: 0.0% Ingestions: 2 as flushable: 2 (2.1KB in 3 tables) Iter category stats: - _unknown, latency: {BlockBytes:0 BlockBytesInCache:0} b, latency: {BlockBytes:44 BlockBytesInCache:0} c, non-latency: {BlockBytes:44 BlockBytesInCache:44} pebble-compaction, non-latency: {BlockBytes:941 BlockBytesInCache:640}